|
@@ -23,6 +23,8 @@ import static org.junit.Assert.assertTrue;
|
|
|
import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
+import java.util.List;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
@@ -30,6 +32,8 @@ import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Executors;
|
|
|
import java.util.concurrent.Future;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.ChecksumException;
|
|
@@ -38,6 +42,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|
@@ -51,6 +58,8 @@ import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
+
|
|
|
/**
|
|
|
* This class tests the DFS positional read functionality in a single node
|
|
|
* mini-cluster.
|
|
@@ -542,6 +551,143 @@ public class TestPread {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Scenario: 1. Write a file with RF=2, DN1 and DN2<br>
|
|
|
+ * 2. Open the stream, Consider Locations are [DN1, DN2] in LocatedBlock.<br>
|
|
|
+ * 3. Move block from DN2 to DN3.<br>
|
|
|
+ * 4. Let block gets replicated to another DN3<br>
|
|
|
+ * 5. Stop DN1 also.<br>
|
|
|
+ * 6. Current valid Block locations in NameNode [DN1, DN3]<br>
|
|
|
+ * 7. Consider next calls to getBlockLocations() always returns DN3 as last
|
|
|
+ * location.<br>
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testPreadFailureWithChangedBlockLocations() throws Exception {
|
|
|
+ doPreadTestWithChangedLocations();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Scenario: 1. Write a file with RF=2, DN1 and DN2<br>
|
|
|
+ * 2. Open the stream, Consider Locations are [DN1, DN2] in LocatedBlock.<br>
|
|
|
+ * 3. Move block from DN2 to DN3.<br>
|
|
|
+ * 4. Let block gets replicated to another DN3<br>
|
|
|
+ * 5. Stop DN1 also.<br>
|
|
|
+ * 6. Current valid Block locations in NameNode [DN1, DN3]<br>
|
|
|
+ * 7. Consider next calls to getBlockLocations() always returns DN3 as last
|
|
|
+ * location.<br>
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testPreadHedgedFailureWithChangedBlockLocations()
|
|
|
+ throws Exception {
|
|
|
+ isHedgedRead = true;
|
|
|
+ doPreadTestWithChangedLocations();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void doPreadTestWithChangedLocations()
|
|
|
+ throws IOException, TimeoutException, InterruptedException {
|
|
|
+ GenericTestUtils.setLogLevel(DFSClient.LOG, Level.DEBUG);
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 2);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
|
+ if (isHedgedRead) {
|
|
|
+ conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, 2);
|
|
|
+ }
|
|
|
+ try (MiniDFSCluster cluster =
|
|
|
+ new MiniDFSCluster.Builder(conf).numDataNodes(3).build()) {
|
|
|
+ DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
+ Path p = new Path("/test");
|
|
|
+ String data = "testingmissingblock";
|
|
|
+ DFSTestUtil.writeFile(dfs, p, data);
|
|
|
+
|
|
|
+ FSDataInputStream in = dfs.open(p);
|
|
|
+ List<LocatedBlock> blocks = DFSTestUtil.getAllBlocks(in);
|
|
|
+ LocatedBlock lb = blocks.get(0);
|
|
|
+ DFSTestUtil.waitForReplication(cluster, lb.getBlock(), 1, 2, 0);
|
|
|
+ blocks = DFSTestUtil.getAllBlocks(in);
|
|
|
+ DatanodeInfo[] locations = null;
|
|
|
+ for (LocatedBlock locatedBlock : blocks) {
|
|
|
+ locations = locatedBlock.getLocations();
|
|
|
+ DFSClient.LOG
|
|
|
+ .info(locatedBlock.getBlock() + " " + Arrays.toString(locations));
|
|
|
+ }
|
|
|
+ final DatanodeInfo validDownLocation = locations[0];
|
|
|
+ final DFSClient client = dfs.getClient();
|
|
|
+ DFSClient dfsClient = Mockito.spy(client);
|
|
|
+ // Keep the valid location as last in the locations list for second
|
|
|
+ // requests
|
|
|
+ // onwards.
|
|
|
+ final AtomicInteger count = new AtomicInteger(0);
|
|
|
+ Mockito.doAnswer(new Answer<LocatedBlocks>() {
|
|
|
+ @Override
|
|
|
+ public LocatedBlocks answer(InvocationOnMock invocation)
|
|
|
+ throws Throwable {
|
|
|
+ if (count.compareAndSet(0, 1)) {
|
|
|
+ return (LocatedBlocks) invocation.callRealMethod();
|
|
|
+ }
|
|
|
+ Object obj = invocation.callRealMethod();
|
|
|
+ LocatedBlocks locatedBlocks = (LocatedBlocks) obj;
|
|
|
+ LocatedBlock lb = locatedBlocks.get(0);
|
|
|
+ DatanodeInfo[] locations = lb.getLocations();
|
|
|
+ if (!(locations[0].getName().equals(validDownLocation.getName()))) {
|
|
|
+ // Latest location which is currently down, should be first
|
|
|
+ DatanodeInfo l = locations[0];
|
|
|
+ locations[0] = locations[locations.length - 1];
|
|
|
+ locations[locations.length - 1] = l;
|
|
|
+ }
|
|
|
+ return locatedBlocks;
|
|
|
+ }
|
|
|
+ }).when(dfsClient).getLocatedBlocks(p.toString(), 0);
|
|
|
+
|
|
|
+ // Findout target node to move the block to.
|
|
|
+ DatanodeInfo[] nodes =
|
|
|
+ cluster.getNameNodeRpc().getDatanodeReport(DatanodeReportType.LIVE);
|
|
|
+ DatanodeInfo toMove = null;
|
|
|
+ List<DatanodeInfo> locationsList = Arrays.asList(locations);
|
|
|
+ for (DatanodeInfo node : nodes) {
|
|
|
+ if (locationsList.contains(node)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ toMove = node;
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ // STEP 2: Open stream
|
|
|
+ DFSInputStream din = dfsClient.open(p.toString());
|
|
|
+ // STEP 3: Move replica
|
|
|
+ final DatanodeInfo source = locations[1];
|
|
|
+ final DatanodeInfo destination = toMove;
|
|
|
+ DFSTestUtil.replaceBlock(lb.getBlock(), source, locations[1], toMove);
|
|
|
+ // Wait for replica to get deleted
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ try {
|
|
|
+ LocatedBlocks lbs = dfsClient.getLocatedBlocks(p.toString(), 0);
|
|
|
+ LocatedBlock lb = lbs.get(0);
|
|
|
+ List<DatanodeInfo> locations = Arrays.asList(lb.getLocations());
|
|
|
+ DFSClient.LOG
|
|
|
+ .info("Source :" + source + ", destination: " + destination);
|
|
|
+ DFSClient.LOG.info("Got updated locations :" + locations);
|
|
|
+ return locations.contains(destination)
|
|
|
+ && !locations.contains(source);
|
|
|
+ } catch (IOException e) {
|
|
|
+ DFSClient.LOG.error("Problem in getting block locations", e);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }, 1000, 10000);
|
|
|
+ DFSTestUtil.waitForReplication(cluster, lb.getBlock(), 1, 2, 0);
|
|
|
+ // STEP 4: Stop first node in new locations
|
|
|
+ cluster.stopDataNode(validDownLocation.getName());
|
|
|
+ DFSClient.LOG.info("Starting read");
|
|
|
+ byte[] buf = new byte[1024];
|
|
|
+ int n = din.read(0, buf, 0, data.length());
|
|
|
+ assertEquals(data.length(), n);
|
|
|
+ assertEquals("Data should be read", data, new String(buf, 0, n));
|
|
|
+ DFSClient.LOG.info("Read completed");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
new TestPread().testPreadDFS();
|
|
|
}
|