You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2014/12/24 20:35:32 UTC

[18/50] hadoop git commit: HDFS-7431. log message for InvalidMagicNumberException may be incorrect. Contributed by Yi Liu.

HDFS-7431. log message for InvalidMagicNumberException may be incorrect. Contributed by Yi Liu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2e5a7939
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e5a7939
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e5a7939

Branch: refs/heads/HDFS-EC
Commit: 2e5a7939fa77ec7eef8bff6d11de4b97be41d100
Parents: d8d86fe
Author: cnauroth <cn...@apache.org>
Authored: Thu Dec 18 15:03:52 2014 -0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Wed Dec 24 11:22:15 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../sasl/InvalidMagicNumberException.java       | 14 +++++++++-
 .../sasl/SaslDataTransferServer.java            |  3 ++-
 .../hdfs/server/datanode/DataXceiver.java       | 15 ++++++++---
 .../datatransfer/sasl/TestSaslDataTransfer.java | 28 +++++++++++++++++---
 5 files changed, 54 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e5a7939/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e68aec2..73286cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -618,6 +618,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7494. Checking of closed in DFSInputStream#pread() should be protected
     by synchronization (Ted Yu via Colin P. McCabe)
 
+    HDFS-7431. log message for InvalidMagicNumberException may be incorrect.
+    (Yi Liu via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e5a7939/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/InvalidMagicNumberException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/InvalidMagicNumberException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/InvalidMagicNumberException.java
index eb494a8..cc9b0c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/InvalidMagicNumberException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/InvalidMagicNumberException.java
@@ -31,14 +31,26 @@ import org.apache.hadoop.classification.InterfaceAudience;
 public class InvalidMagicNumberException extends IOException {
 
   private static final long serialVersionUID = 1L;
+  private final boolean handshake4Encryption;
 
   /**
    * Creates a new InvalidMagicNumberException.
    *
    * @param magicNumber expected value
    */
-  public InvalidMagicNumberException(int magicNumber) {
+  public InvalidMagicNumberException(final int magicNumber, 
+      final boolean handshake4Encryption) {
     super(String.format("Received %x instead of %x from client.",
         magicNumber, SASL_TRANSFER_MAGIC_NUMBER));
+    this.handshake4Encryption = handshake4Encryption;
+  }
+  
+  /**
+   * Return true if it's handshake for encryption
+   * 
+   * @return boolean true if it's handshake for encryption
+   */
+  public boolean isHandshake4Encryption() {
+    return handshake4Encryption;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e5a7939/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
index 1d2b30b..f060beb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
@@ -357,7 +357,8 @@ public class SaslDataTransferServer {
 
     int magicNumber = in.readInt();
     if (magicNumber != SASL_TRANSFER_MAGIC_NUMBER) {
-      throw new InvalidMagicNumberException(magicNumber);
+      throw new InvalidMagicNumberException(magicNumber, 
+          dnConf.getEncryptDataTransfer());
     }
     try {
       // step 1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e5a7939/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index bbc23e6..3a2723f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -192,10 +192,17 @@ class DataXceiver extends Receiver implements Runnable {
           HdfsConstants.SMALL_BUFFER_SIZE);
         socketOut = saslStreams.out;
       } catch (InvalidMagicNumberException imne) {
-        LOG.info("Failed to read expected encryption handshake from client " +
-            "at " + peer.getRemoteAddressString() + ". Perhaps the client " +
-            "is running an older version of Hadoop which does not support " +
-            "encryption");
+        if (imne.isHandshake4Encryption()) {
+          LOG.info("Failed to read expected encryption handshake from client " +
+              "at " + peer.getRemoteAddressString() + ". Perhaps the client " +
+              "is running an older version of Hadoop which does not support " +
+              "encryption");
+        } else {
+          LOG.info("Failed to read expected SASL data transfer protection " +
+              "handshake from client at " + peer.getRemoteAddressString() + 
+              ". Perhaps the client is running an older version of Hadoop " +
+              "which does not support SASL data transfer protection");
+        }
         return;
       }
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e5a7939/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
index 0d860b4..52e5384 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
@@ -25,16 +25,22 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -110,12 +116,28 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
   public void testServerSaslNoClientSasl() throws Exception {
     HdfsConfiguration clusterConf = createSecureConfig(
       "authentication,integrity,privacy");
+    // Set short retry timeouts so this test runs faster
+    clusterConf.setInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, 10);
     startCluster(clusterConf);
     HdfsConfiguration clientConf = new HdfsConfiguration(clusterConf);
     clientConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "");
-    exception.expect(IOException.class);
-    exception.expectMessage("could only be replicated to 0 nodes");
-    doTest(clientConf);
+
+    LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(
+        LogFactory.getLog(DataNode.class));
+    try {
+      doTest(clientConf);
+      Assert.fail("Should fail if SASL data transfer protection is not " +
+          "configured or not supported in client");
+    } catch (IOException e) {
+      GenericTestUtils.assertMatches(e.getMessage(), 
+          "could only be replicated to 0 nodes");
+    } finally {
+      logs.stopCapturing();
+    }
+
+    GenericTestUtils.assertMatches(logs.getOutput(),
+        "Failed to read expected SASL data transfer protection " +
+        "handshake from client at");
   }
 
   @Test