|
@@ -17,6 +17,7 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
+import net.jcip.annotations.NotThreadSafe;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
|
|
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
@@ -34,6 +35,8 @@ import java.nio.channels.FileChannel;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
|
+import java.util.concurrent.locks.ReadWriteLock;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -80,8 +83,10 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.test.MetricsAsserts;
|
|
|
import org.apache.log4j.Logger;
|
|
|
import org.junit.After;
|
|
|
+import org.junit.AfterClass;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
+import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.apache.log4j.LogManager;
|
|
@@ -91,6 +96,7 @@ import com.google.common.primitives.Ints;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY;
|
|
|
|
|
|
+@NotThreadSafe
|
|
|
public class TestFsDatasetCache {
|
|
|
private static final Log LOG = LogFactory.getLog(TestFsDatasetCache.class);
|
|
|
|
|
@@ -110,13 +116,39 @@ public class TestFsDatasetCache {
|
|
|
private static DataNode dn;
|
|
|
private static FsDatasetSpi<?> fsd;
|
|
|
private static DatanodeProtocolClientSideTranslatorPB spyNN;
|
|
|
+ /**
|
|
|
+ * Used to pause DN BPServiceActor threads. BPSA threads acquire the
|
|
|
+ * shared read lock. The test acquires the write lock for exclusive access.
|
|
|
+ */
|
|
|
+ private static ReadWriteLock lock = new ReentrantReadWriteLock(true);
|
|
|
private static final PageRounder rounder = new PageRounder();
|
|
|
private static CacheManipulator prevCacheManipulator;
|
|
|
+ private static DataNodeFaultInjector oldInjector;
|
|
|
|
|
|
static {
|
|
|
LogManager.getLogger(FsDatasetCache.class).setLevel(Level.DEBUG);
|
|
|
}
|
|
|
|
|
|
+ @BeforeClass
|
|
|
+ public static void setUpClass() throws Exception {
|
|
|
+ oldInjector = DataNodeFaultInjector.get();
|
|
|
+ DataNodeFaultInjector.set(new DataNodeFaultInjector() {
|
|
|
+ @Override
|
|
|
+ public void startOfferService() throws Exception {
|
|
|
+ lock.readLock().lock();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void endOfferService() throws Exception {
|
|
|
+ lock.readLock().unlock();
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ @AfterClass
|
|
|
+ public static void tearDownClass() throws Exception {
|
|
|
+ DataNodeFaultInjector.set(oldInjector);
|
|
|
+ }
|
|
|
+
|
|
|
@Before
|
|
|
public void setUp() throws Exception {
|
|
|
conf = new HdfsConfiguration();
|
|
@@ -143,7 +175,6 @@ public class TestFsDatasetCache {
|
|
|
fsd = dn.getFSDataset();
|
|
|
|
|
|
spyNN = InternalDataNodeTestUtils.spyOnBposToNN(dn, nn);
|
|
|
-
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -164,18 +195,23 @@ public class TestFsDatasetCache {
|
|
|
}
|
|
|
|
|
|
private static void setHeartbeatResponse(DatanodeCommand[] cmds)
|
|
|
- throws IOException {
|
|
|
- NNHAStatusHeartbeat ha = new NNHAStatusHeartbeat(HAServiceState.ACTIVE,
|
|
|
- fsImage.getLastAppliedOrWrittenTxId());
|
|
|
- HeartbeatResponse response =
|
|
|
- new HeartbeatResponse(cmds, ha, null,
|
|
|
- ThreadLocalRandom.current().nextLong() | 1L);
|
|
|
- doReturn(response).when(spyNN).sendHeartbeat(
|
|
|
- (DatanodeRegistration) any(),
|
|
|
- (StorageReport[]) any(), anyLong(), anyLong(),
|
|
|
- anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
|
|
|
- anyBoolean(), any(SlowPeerReports.class),
|
|
|
- any(SlowDiskReports.class));
|
|
|
+ throws Exception {
|
|
|
+ lock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ NNHAStatusHeartbeat ha = new NNHAStatusHeartbeat(HAServiceState.ACTIVE,
|
|
|
+ fsImage.getLastAppliedOrWrittenTxId());
|
|
|
+ HeartbeatResponse response =
|
|
|
+ new HeartbeatResponse(cmds, ha, null,
|
|
|
+ ThreadLocalRandom.current().nextLong() | 1L);
|
|
|
+ doReturn(response).when(spyNN).sendHeartbeat(
|
|
|
+ (DatanodeRegistration) any(),
|
|
|
+ (StorageReport[]) any(), anyLong(), anyLong(),
|
|
|
+ anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
|
|
|
+ anyBoolean(), any(SlowPeerReports.class),
|
|
|
+ any(SlowDiskReports.class));
|
|
|
+ } finally {
|
|
|
+ lock.writeLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private static DatanodeCommand[] cacheBlock(HdfsBlockLocation loc) {
|