Selaa lähdekoodia

HDFS-4473. Don't create domain socket unless we need it. Contributed by Colin Patrick McCabe.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-347@1443283 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 12 vuotta sitten
vanhempi
commit
3d46863e12

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-347.txt

@@ -39,3 +39,5 @@ HDFS-4433. Make TestPeerCache not flaky (Colin Patrick McCabe via todd)
 HDFS-4438. TestDomainSocket fails when system umask is set to 0002. (Colin Patrick McCabe via atm)
 
 HDFS-4440. Avoid annoying log message when dfs.domain.socket.path is not set. (Colin Patrick McCabe via atm)
+
+HDFS-4473. Don't create domain socket unless we need it. (Colin Patrick McCabe via atm)

+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -559,13 +559,18 @@ public class DataNode extends Configured
         new DataXceiverServer(tcpPeerServer, conf, this));
     this.threadGroup.setDaemon(true); // auto destroy when empty
 
-    DomainPeerServer domainPeerServer =
-              getDomainPeerServer(conf, streamingAddr.getPort());
-    if (domainPeerServer != null) {
-      this.localDataXceiverServer = new Daemon(threadGroup, 
-          new DataXceiverServer(domainPeerServer, conf, this));
-      LOG.info("Listening on UNIX domain socket: " +
-          domainPeerServer.getBindPath());
+    if (conf.getBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
+              DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT) ||
+        conf.getBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+              DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
+      DomainPeerServer domainPeerServer =
+                getDomainPeerServer(conf, streamingAddr.getPort());
+      if (domainPeerServer != null) {
+        this.localDataXceiverServer = new Daemon(threadGroup,
+            new DataXceiverServer(domainPeerServer, conf, this));
+        LOG.info("Listening on UNIX domain socket: " +
+            domainPeerServer.getBindPath());
+      }
     }
   }
 

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java

@@ -26,7 +26,17 @@ import org.junit.BeforeClass;
 public class TestParallelRead extends TestParallelReadUtil {
   @BeforeClass
   static public void setupCluster() throws Exception {
-    setupCluster(DEFAULT_REPLICATION_FACTOR, new HdfsConfiguration());
+    // This is a test of the normal (TCP) read path.  For this reason, we turn
+    // off both short-circuit local reads and UNIX domain socket data traffic.
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+                    false);
+    // dfs.domain.socket.path should be ignored because the previous two keys
+    // were set to false.  This is a regression test for HDFS-4473.
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, "/will/not/be/created");
+
+    setupCluster(DEFAULT_REPLICATION_FACTOR, conf);
   }
 
   @AfterClass

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java

@@ -293,6 +293,9 @@ public class TestShortCircuitLocalRead {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        "/tmp/testSkipWithVerifyChecksum._PORT");
+    DomainSocket.disableBindPathValidation();
     if (simulatedStorage) {
       SimulatedFSDataset.setFactory(conf);
     }
@@ -337,6 +340,8 @@ public class TestShortCircuitLocalRead {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        "/tmp/testHandleTruncatedBlockFile._PORT");
     conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
     final Path TEST_PATH = new Path("/a");
     final Path TEST_PATH2 = new Path("/b");
@@ -432,6 +437,8 @@ public class TestShortCircuitLocalRead {
     // Setup create a file
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        "/tmp/TestShortCircuitLocalRead._PORT");
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
         checksum);