HDFS-16698. Add a metric to sense possible MaxDirectoryItemsExceededException in time.

This commit is contained in:
zengqiang.xu 2022-07-27 14:52:00 +08:00
parent 4c8cd61961
commit a2fef1296f
5 changed files with 58 additions and 5 deletions

View File

@ -781,6 +781,11 @@ public class NamenodeBeanMetrics
return 0;
}
@Override
public int getMaxDirectoryItemsAlarmNums() {
return 0;
}
@Override
public long getTotalSyncCount() {
return 0;

View File

@ -83,6 +83,7 @@ import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ForkJoinPool;
import java.util.concurrent.RecursiveAction;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hadoop.fs.CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
@ -156,6 +157,8 @@ public class FSDirectory implements Closeable {
private volatile boolean skipQuotaCheck = false; //skip while consuming edits
private final int maxComponentLength;
private final int maxDirItems;
private final int maxDirItemsAlarmThreshold;
private final AtomicInteger maxDirItemsAlarmNum;
private final int lsLimit; // max list limit
private final int contentCountLimit; // max content summary counts per run
private final long contentSleepMicroSec;
@ -379,6 +382,8 @@ public class FSDirectory implements Closeable {
this.maxDirItems = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY,
DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_DEFAULT);
this.maxDirItemsAlarmThreshold = (int) Math.ceil(this.maxDirItems * 0.8);
this.maxDirItemsAlarmNum = new AtomicInteger(0);
this.inodeXAttrsLimit = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY,
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT);
@ -1283,16 +1288,18 @@ public class FSDirectory implements Closeable {
throws MaxDirectoryItemsExceededException {
final int count = parent.getChildrenList(CURRENT_STATE_ID).size();
if (count >= maxDirItems) {
final MaxDirectoryItemsExceededException e
= new MaxDirectoryItemsExceededException(parentPath, maxDirItems,
count);
final MaxDirectoryItemsExceededException e = new MaxDirectoryItemsExceededException(
parentPath, maxDirItems, count);
if (namesystem.isImageLoaded()) {
throw e;
} else {
// Do not throw if edits log is still being processed
NameNode.LOG.error("FSDirectory.verifyMaxDirItems: "
+ e.getLocalizedMessage());
NameNode.LOG.error("FSDirectory.verifyMaxDirItems: {}.", e.getLocalizedMessage());
}
} else if (count >= maxDirItemsAlarmThreshold) {
LOG.warn("File count exceeding alarm threshold for {}: {}/{}",
parentPath, count, this.maxDirItems);
this.maxDirItemsAlarmNum.incrementAndGet();
}
}
@ -1579,6 +1586,10 @@ public class FSDirectory implements Closeable {
return getInodeMapSize();
}
int getMaxDirItemsAlarmNum() {
return this.maxDirItemsAlarmNum.get();
}
/**
* Reset the entire namespace tree.
*/

View File

@ -8888,6 +8888,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return fsImage.editLog.getTotalSyncCount();
}
/**
* Returns total number of alarms that SubDirectoryItems
* of one directory have reached alarm threshold.
*/
@Override
@Metric({"MaxDirectoryItemsAlarmNums",
"Total number of alarms that SubDirectoryItems more than alarm threshold"})
public int getMaxDirectoryItemsAlarmNums() {
return this.dir.getMaxDirItemsAlarmNum();
}
/**
* Return total time spent doing sync operations on FSEditLog.
*/

View File

@ -224,6 +224,12 @@ public interface FSNamesystemMBean {
*/
int getFsLockQueueLength();
/**
* Returns total number of alarms that SubDirectoryItems
* of one directory have reached alarm threshold.
*/
public int getMaxDirectoryItemsAlarmNums();
/**
* Return total number of Sync Operations on FSEditLog.
*/

View File

@ -114,6 +114,26 @@ public class TestFsLimits {
mkdirs("/4444", MaxDirectoryItemsExceededException.class);
}
@Test
public void testMaxDirItemsAlarm() throws Exception {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 10);
mkdirs("/1", null);
mkdirs("/22", null);
mkdirs("/333", null);
mkdirs("/4444", null);
mkdirs("/55555", null);
mkdirs("/666666", null);
mkdirs("/7777777", null);
mkdirs("/88888888", null);
assertEquals(0, fs.getMaxDirectoryItemsAlarmNums());
mkdirs("/999999999", null);
assertEquals(1, fs.getMaxDirectoryItemsAlarmNums());
mkdirs("/1010101010", null);
assertEquals(2, fs.getMaxDirectoryItemsAlarmNums());
mkdirs("/11111111111", MaxDirectoryItemsExceededException.class);
assertEquals(2, fs.getMaxDirectoryItemsAlarmNums());
}
@Test
public void testMaxDirItemsRename() throws Exception {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);