|
@@ -43,6 +43,7 @@ import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.commons.lang3.time.DateUtils;
|
|
import org.apache.commons.lang3.time.DateUtils;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -60,9 +61,11 @@ import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
|
|
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.DistributedFileSystem;
|
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.CacheDirective;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
|
|
@@ -1605,4 +1608,65 @@ public class TestCacheDirectives {
|
|
waitForCachedBlocks(namenode, expected, 0,
|
|
waitForCachedBlocks(namenode, expected, 0,
|
|
"testAddingCacheDirectiveInfosWhenCachingIsDisabled:2");
|
|
"testAddingCacheDirectiveInfosWhenCachingIsDisabled:2");
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test(timeout=120000)
|
|
|
|
+ public void testExpiryTimeConsistency() throws Exception {
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, 1);
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
|
|
|
+ MiniDFSCluster dfsCluster =
|
|
|
|
+ new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
|
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleHATopology())
|
|
|
|
+ .build();
|
|
|
|
+ dfsCluster.transitionToActive(0);
|
|
|
|
+
|
|
|
|
+ DistributedFileSystem fs = dfsCluster.getFileSystem(0);
|
|
|
|
+ final NameNode ann = dfsCluster.getNameNode(0);
|
|
|
|
+
|
|
|
|
+ final Path filename = new Path("/file");
|
|
|
|
+ final short replication = (short) 3;
|
|
|
|
+ DFSTestUtil.createFile(fs, filename, 1, replication, 0x0BAC);
|
|
|
|
+ fs.addCachePool(new CachePoolInfo("pool"));
|
|
|
|
+ long id = fs.addCacheDirective(
|
|
|
|
+ new CacheDirectiveInfo.Builder().setPool("pool").setPath(filename)
|
|
|
|
+ .setExpiration(CacheDirectiveInfo.Expiration.newRelative(86400000))
|
|
|
|
+ .setReplication(replication).build());
|
|
|
|
+ fs.modifyCacheDirective(new CacheDirectiveInfo.Builder()
|
|
|
|
+ .setId(id)
|
|
|
|
+ .setExpiration(CacheDirectiveInfo.Expiration.newRelative(172800000))
|
|
|
|
+ .build());
|
|
|
|
+ final NameNode sbn = dfsCluster.getNameNode(1);
|
|
|
|
+ final CacheManager annCachemanager = ann.getNamesystem().getCacheManager();
|
|
|
|
+ final CacheManager sbnCachemanager = sbn.getNamesystem().getCacheManager();
|
|
|
|
+ HATestUtil.waitForStandbyToCatchUp(ann, sbn);
|
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
|
+ boolean isConsistence = false;
|
|
|
|
+ ann.getNamesystem().readLock();
|
|
|
|
+ try {
|
|
|
|
+ sbn.getNamesystem().readLock();
|
|
|
|
+ try {
|
|
|
|
+ Iterator<CacheDirective> annDirectivesIt = annCachemanager.
|
|
|
|
+ getCacheDirectives().iterator();
|
|
|
|
+ Iterator<CacheDirective> sbnDirectivesIt = sbnCachemanager.
|
|
|
|
+ getCacheDirectives().iterator();
|
|
|
|
+ if (annDirectivesIt.hasNext() && sbnDirectivesIt.hasNext()) {
|
|
|
|
+ CacheDirective annDirective = annDirectivesIt.next();
|
|
|
|
+ CacheDirective sbnDirective = sbnDirectivesIt.next();
|
|
|
|
+ if (annDirective.getExpiryTimeString().
|
|
|
|
+ equals(sbnDirective.getExpiryTimeString())) {
|
|
|
|
+ isConsistence = true;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ sbn.getNamesystem().readUnlock();
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ ann.getNamesystem().readUnlock();
|
|
|
|
+ }
|
|
|
|
+ if (!isConsistence) {
|
|
|
|
+ LOG.info("testEexpiryTimeConsistency:"
|
|
|
|
+ + "ANN CacheDirective Status is inconsistent with SBN");
|
|
|
|
+ }
|
|
|
|
+ return isConsistence;
|
|
|
|
+ }, 500, 120000);
|
|
|
|
+ }
|
|
}
|
|
}
|