Просмотр исходного кода

HDFS-8070. Pre-HDFS-7915 DFSClient cannot use short circuit on post-HDFS-7915 DataNode (cmccabe)

(cherry picked from commit a8898445dc9b5cdb7230e2e23a57393c9f378ff0)
(cherry picked from commit 3d0385c3c8fab72be02453a158c7692212ab3b5e)
Colin Patrick Mccabe 10 лет назад
Родитель
Сommit
181281cfca

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -126,6 +126,9 @@ Release 2.6.1 - UNRELEASED
     HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
     fail to tell the DFSClient about it because of a network error (cmccabe)
 
+    HDFS-8070. Pre-HDFS-7915 DFSClient cannot use short circuit on
+    post-HDFS-7915 DataNode (cmccabe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -74,6 +74,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     public void injectRequestFileDescriptorsFailure() throws IOException {
       // do nothing
     }
+    public boolean getSupportsReceiptVerification() {
+      return true;
+    }
   }
 
   @VisibleForTesting
@@ -531,7 +534,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1,
+        failureInjector.getSupportsReceiptVerification());
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java

@@ -184,7 +184,8 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion(), true);
+          slotId, proto.getMaxVersion(),
+          proto.getSupportsReceiptVerification());
     } finally {
       if (traceScope != null) traceScope.close();
     }

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java

@@ -677,4 +677,47 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
+
+  public static class TestPreReceiptVerificationFailureInjector
+      extends BlockReaderFactory.FailureInjector {
+    @Override
+    public boolean getSupportsReceiptVerification() {
+      return false;
+    }
+  }
+
+  // Regression test for HDFS-8070
+  @Test(timeout=60000)
+  public void testPreReceiptVerificationDfsClientCanDoScr() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf(
+        "testPreReceiptVerificationDfsClientCanDoScr", sockDir);
+    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        1000000000L);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    fs.getClient().getConf().brfFailureInjector =
+        new TestPreReceiptVerificationFailureInjector();
+    final Path TEST_PATH1 = new Path("/test_file1");
+    DFSTestUtil.createFile(fs, TEST_PATH1, 4096, (short)1, 0xFADE2);
+    final Path TEST_PATH2 = new Path("/test_file2");
+    DFSTestUtil.createFile(fs, TEST_PATH2, 4096, (short)1, 0xFADE2);
+    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
+    DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
+    ShortCircuitRegistry registry =
+        cluster.getDataNodes().get(0).getShortCircuitRegistry();
+    registry.visit(new ShortCircuitRegistry.Visitor() {
+      @Override
+      public void accept(HashMap<ShmId, RegisteredShm> segments,
+                         HashMultimap<ExtendedBlockId, Slot> slots) {
+        Assert.assertEquals(1, segments.size());
+        Assert.assertEquals(2, slots.size());
+      }
+    });
+    cluster.shutdown();
+    sockDir.close();
+  }
 }