HDFS-9396. Total files and directories on jmx and web UI on standby is uninitialized. Contributed by Kihwal Lee.
(cherry picked from commit a2a104f3d3
)
This commit is contained in:
parent
bb5dd7f772
commit
8726a5e020
|
@ -1444,6 +1444,9 @@ Release 2.8.0 - UNRELEASED
|
||||||
HDFS-9245. Fix findbugs warnings in hdfs-nfs/WriteCtx.
|
HDFS-9245. Fix findbugs warnings in hdfs-nfs/WriteCtx.
|
||||||
(Mingliang Liu via xyao)
|
(Mingliang Liu via xyao)
|
||||||
|
|
||||||
|
HDFS-9396. Total files and directories on jmx and web UI on standby is
|
||||||
|
uninitialized. (kihwal)
|
||||||
|
|
||||||
Release 2.7.3 - UNRELEASED
|
Release 2.7.3 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -1218,13 +1218,7 @@ public class FSDirectory implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
long totalInodes() {
|
long totalInodes() {
|
||||||
readLock();
|
return getInodeMapSize();
|
||||||
try {
|
|
||||||
return rootDir.getDirectoryWithQuotaFeature().getSpaceConsumed()
|
|
||||||
.getNameSpace();
|
|
||||||
} finally {
|
|
||||||
readUnlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -24,8 +24,10 @@ import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -119,4 +121,58 @@ public class TestHAMetrics {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testHAInodeCount() throws Exception {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, Integer.MAX_VALUE);
|
||||||
|
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||||
|
.nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(1)
|
||||||
|
.build();
|
||||||
|
FileSystem fs = null;
|
||||||
|
try {
|
||||||
|
cluster.waitActive();
|
||||||
|
|
||||||
|
FSNamesystem nn0 = cluster.getNamesystem(0);
|
||||||
|
FSNamesystem nn1 = cluster.getNamesystem(1);
|
||||||
|
|
||||||
|
cluster.transitionToActive(0);
|
||||||
|
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||||
|
DFSTestUtil.createFile(fs, new Path("/testHAInodeCount1"),
|
||||||
|
10, (short)1, 1L);
|
||||||
|
DFSTestUtil.createFile(fs, new Path("/testHAInodeCount2"),
|
||||||
|
10, (short)1, 1L);
|
||||||
|
DFSTestUtil.createFile(fs, new Path("/testHAInodeCount3"),
|
||||||
|
10, (short)1, 1L);
|
||||||
|
DFSTestUtil.createFile(fs, new Path("/testHAInodeCount4"),
|
||||||
|
10, (short)1, 1L);
|
||||||
|
|
||||||
|
// 1 dir and 4 files
|
||||||
|
assertEquals(5, nn0.getFilesTotal());
|
||||||
|
// The SBN still has one dir, which is "/".
|
||||||
|
assertEquals(1, nn1.getFilesTotal());
|
||||||
|
|
||||||
|
// Save fsimage so that nn does not build up namesystem by replaying
|
||||||
|
// edits, but load from the image.
|
||||||
|
((DistributedFileSystem)fs).setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||||
|
((DistributedFileSystem)fs).saveNamespace();
|
||||||
|
|
||||||
|
// Flip the two namenodes and restart the standby, which will load
|
||||||
|
// the fsimage.
|
||||||
|
cluster.transitionToStandby(0);
|
||||||
|
cluster.transitionToActive(1);
|
||||||
|
cluster.restartNameNode(0);
|
||||||
|
assertEquals(nn0.getHAState(), "standby");
|
||||||
|
|
||||||
|
// The restarted standby should report the correct count
|
||||||
|
nn0 = cluster.getNamesystem(0);
|
||||||
|
assertEquals(5, nn0.getFilesTotal());
|
||||||
|
} finally {
|
||||||
|
IOUtils.cleanup(LOG, fs);
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue