Ver código fonte

HDFS-4390. Bypass UNIX domain socket unit tests when they cannot be run. Contributed by Colin Patrick McCabe.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-347@1432350 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 12 anos atrás
pai
commit
d2737575ad

+ 1 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TestDomainSocket.java

@@ -30,12 +30,10 @@ import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 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 org.junit.AfterClass;
 import org.junit.Assert;
@@ -49,7 +47,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket.DomainChannel;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.Shell.ExitCodeException;
 
 import com.google.common.io.Files;
 
@@ -68,7 +65,7 @@ public class TestDomainSocket {
   }
   
   @Before
-  public void checkPrecondition() {
+  public void before() {
     Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null);
   }
     

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

@@ -13,3 +13,7 @@ HDFS-4356. BlockReaderLocal should use passed file descriptors rather than paths
 
 HDFS-4388. DomainSocket should throw AsynchronousCloseException when appropriate.
 (Colin Patrick McCabe via todd)
+
+HDFS-4390. Bypass UNIX domain socket unit tests when they cannot be run.
+(Colin Patrick McCabe via todd)
+

+ 0 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java

@@ -28,7 +28,6 @@ import java.net.Socket;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.unix.DomainSocket;
 
 /**
  * A helper class to setup the cluster, and get to BlockReader and DataNode for a block.
@@ -168,13 +166,4 @@ public class BlockReaderTestUtil {
     int ipcport = nodes[0].getIpcPort();
     return cluster.getDataNode(ipcport);
   }
-
-  public boolean haveRequiredResources() {
-    if (conf.get(DFSConfigKeys.DFS_DATANODE_DOMAIN_SOCKET_PATH_KEY) != null) {
-      // To use UNIX Domain sockets, we must have the native code loaded.
-      return DomainSocket.getLoadingFailureReason() == null;
-    } else {
-      return true;
-    }
-  }
 }

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java

@@ -401,19 +401,16 @@ public class TestParallelReadUtil {
    */
   @Test
   public void testParallelReadCopying() throws IOException {
-    Assume.assumeTrue(util.haveRequiredResources());
     runTestWorkload(new CopyingReadWorkerHelper());
   }
 
   @Test
   public void testParallelReadByteBuffer() throws IOException {
-    Assume.assumeTrue(util.haveRequiredResources());
     runTestWorkload(new DirectReadWorkerHelper());
   }
 
   @Test
   public void testParallelReadMixed() throws IOException {
-    Assume.assumeTrue(util.haveRequiredResources());
     runTestWorkload(new MixedWorkloadHelper());
   }
 }

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

@@ -22,13 +22,17 @@ import java.io.File;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
 
 public class TestParallelShortCircuitRead extends TestParallelReadUtil {
   private static TemporarySocketDirectory sockDir;
 
   @BeforeClass
   static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
     sockDir = new TemporarySocketDirectory();
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DATANODE_DOMAIN_SOCKET_PATH_KEY,
@@ -40,9 +44,15 @@ public class TestParallelShortCircuitRead extends TestParallelReadUtil {
     setupCluster(1, conf);
   }
 
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
   @AfterClass
   static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
     sockDir.close();
     TestParallelReadUtil.teardownCluster();
   }
-}
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadNoChecksum.java

@@ -22,13 +22,17 @@ import java.io.File;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
 
 public class TestParallelShortCircuitReadNoChecksum extends TestParallelReadUtil {
   private static TemporarySocketDirectory sockDir;
 
   @BeforeClass
   static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
     sockDir = new TemporarySocketDirectory();
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DATANODE_DOMAIN_SOCKET_PATH_KEY,
@@ -40,8 +44,14 @@ public class TestParallelShortCircuitReadNoChecksum extends TestParallelReadUtil
     setupCluster(1, conf);
   }
 
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
   @AfterClass
   static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
     sockDir.close();
     TestParallelReadUtil.teardownCluster();
   }

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

@@ -22,13 +22,17 @@ import java.io.File;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.BeforeClass;
+import static org.hamcrest.CoreMatchers.*;
 
 public class TestParallelUnixDomainRead extends TestParallelReadUtil {
   private static TemporarySocketDirectory sockDir;
 
   @BeforeClass
   static public void setupCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
     sockDir = new TemporarySocketDirectory();
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DATANODE_DOMAIN_SOCKET_PATH_KEY,
@@ -38,9 +42,15 @@ public class TestParallelUnixDomainRead extends TestParallelReadUtil {
     setupCluster(1, conf);
   }
 
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+
   @AfterClass
   static public void teardownCluster() throws Exception {
+    if (DomainSocket.getLoadingFailureReason() != null) return;
     sockDir.close();
     TestParallelReadUtil.teardownCluster();
   }
-}
+}

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

@@ -47,8 +47,11 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.hamcrest.CoreMatchers.*;
 
 /**
  * Test for short circuit read functionality using {@link BlockReaderLocal}.
@@ -71,6 +74,11 @@ public class TestShortCircuitLocalRead {
     sockDir.close();
   }
 
+  @Before
+  public void before() {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+  }
+  
   static final long seed = 0xDEADBEEFL;
   static final int blockSize = 5120;
   boolean simulatedStorage = false;