浏览代码

HDDS-227. Use Grpc as the default transport protocol for Standalone pipeline. Contributed by chencan.

Márton Elek 6 年之前
父节点
当前提交
a813fd0215

+ 1 - 5
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java

@@ -60,7 +60,6 @@ public class XceiverClientManager implements Closeable {
   private final Configuration conf;
   private final Cache<Long, XceiverClientSpi> clientCache;
   private final boolean useRatis;
-  private final boolean useGrpc;
 
   private static XceiverClientMetrics metrics;
   /**
@@ -78,8 +77,6 @@ public class XceiverClientManager implements Closeable {
     this.useRatis = conf.getBoolean(
         ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
         ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
-    this.useGrpc = conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_DEFAULT);
     this.conf = conf;
     this.clientCache = CacheBuilder.newBuilder()
         .expireAfterAccess(staleThresholdMs, TimeUnit.MILLISECONDS)
@@ -153,8 +150,7 @@ public class XceiverClientManager implements Closeable {
               client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf);
               break;
             case STAND_ALONE:
-              client = useGrpc ? new XceiverClientGrpc(pipeline, conf) :
-                  new XceiverClient(pipeline, conf);
+              client = new XceiverClientGrpc(pipeline, conf);
               break;
             case CHAINED:
             default:

+ 0 - 4
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java

@@ -49,10 +49,6 @@ public final class ScmConfigKeys {
       = "dfs.container.ratis.enabled";
   public static final boolean DFS_CONTAINER_RATIS_ENABLED_DEFAULT
       = false;
-  public static final String DFS_CONTAINER_GRPC_ENABLED_KEY
-      = "dfs.container.grpc.enabled";
-  public static final boolean DFS_CONTAINER_GRPC_ENABLED_DEFAULT
-      = false;
   public static final String DFS_CONTAINER_RATIS_RPC_TYPE_KEY
       = "dfs.container.ratis.rpc.type";
   public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT

+ 0 - 9
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -94,15 +94,6 @@
       the replication pipeline supported by ozone.
     </description>
   </property>
-  <property>
-    <name>dfs.container.grpc.enabled</name>
-    <value>false</value>
-    <tag>OZONE, MANAGEMENT, PIPELINE, RATIS</tag>
-    <description>Ozone supports different kinds of replication pipelines
-      protocols. grpc is one of the replication pipeline protocol supported by
-      ozone.
-    </description>
-  </property>
   <property>
     <name>dfs.container.ratis.ipc</name>
     <value>9858</value>

+ 2 - 9
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java

@@ -25,12 +25,10 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
-import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
 import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
@@ -76,17 +74,12 @@ public class OzoneContainer {
     this.config = conf;
     this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
     this.containerSet = new ContainerSet();
-    boolean useGrpc = this.config.getBoolean(
-        ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_DEFAULT);
     buildContainerSet();
     hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet,
         context);
     server = new XceiverServerSpi[]{
-        useGrpc ? new XceiverServerGrpc(datanodeDetails, this.config, this
-            .hddsDispatcher) :
-            new XceiverServer(datanodeDetails,
-                this.config, this.hddsDispatcher),
+        new XceiverServerGrpc(datanodeDetails, this.config, this
+            .hddsDispatcher),
         XceiverServerRatis.newXceiverServerRatis(datanodeDetails, this
             .config, hddsDispatcher)
     };

+ 10 - 41
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java

@@ -26,10 +26,9 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.hdds.scm.XceiverClient;
+import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -77,7 +76,7 @@ public class TestOzoneContainer {
       container.getDispatcher().setScmId(UUID.randomUUID().toString());
       container.start();
 
-      XceiverClient client = new XceiverClient(pipeline, conf);
+      XceiverClientGrpc client = new XceiverClientGrpc(pipeline, conf);
       client.connect();
       createContainerForTesting(client, containerID);
     } finally {
@@ -119,7 +118,7 @@ public class TestOzoneContainer {
       cluster.waitForClusterToBeReady();
 
       // This client talks to ozone container via datanode.
-      XceiverClient client = new XceiverClient(pipeline, conf);
+      XceiverClientGrpc client = new XceiverClientGrpc(pipeline, conf);
 
       runTestOzoneContainerViaDataNode(containerID, client);
     } finally {
@@ -215,7 +214,7 @@ public class TestOzoneContainer {
   @Test
   public void testBothGetandPutSmallFile() throws Exception {
     MiniOzoneCluster cluster = null;
-    XceiverClient client = null;
+    XceiverClientGrpc client = null;
     try {
       OzoneConfiguration conf = newOzoneConfiguration();
 
@@ -269,7 +268,7 @@ public class TestOzoneContainer {
   @Test
   public void testCloseContainer() throws Exception {
     MiniOzoneCluster cluster = null;
-    XceiverClient client = null;
+    XceiverClientGrpc client = null;
     ContainerProtos.ContainerCommandResponseProto response;
     ContainerProtos.ContainerCommandRequestProto
         writeChunkRequest, putKeyRequest, request;
@@ -365,7 +364,7 @@ public class TestOzoneContainer {
   @Test
   public void testDeleteContainer() throws Exception {
     MiniOzoneCluster cluster = null;
-    XceiverClient client = null;
+    XceiverClientGrpc client = null;
     ContainerProtos.ContainerCommandResponseProto response;
     ContainerProtos.ContainerCommandRequestProto request,
         writeChunkRequest, putKeyRequest;
@@ -485,7 +484,7 @@ public class TestOzoneContainer {
   @Test
   public void testXcieverClientAsync() throws Exception {
     MiniOzoneCluster cluster = null;
-    XceiverClient client = null;
+    XceiverClientGrpc client = null;
     try {
       OzoneConfiguration conf = newOzoneConfiguration();
 
@@ -503,38 +502,8 @@ public class TestOzoneContainer {
     }
   }
 
-  @Test
-  public void testInvalidRequest() throws Exception {
-    MiniOzoneCluster cluster = null;
-    XceiverClient client;
-    ContainerProtos.ContainerCommandRequestProto request;
-    try {
-      OzoneConfiguration conf = newOzoneConfiguration();
-
-      client = createClientForTesting(conf);
-      cluster = MiniOzoneCluster.newBuilder(conf)
-              .setRandomContainerPort(false)
-              .build();
-      cluster.waitForClusterToBeReady();
-      client.connect();
-
-      // Send a request without traceId.
-      long containerID = ContainerTestHelper.getTestContainerID();
-      request = ContainerTestHelper
-          .getRequestWithoutTraceId(client.getPipeline(), containerID);
-      client.sendCommand(request);
-      Assert.fail("IllegalArgumentException expected");
-    } catch(IllegalArgumentException iae){
-      GenericTestUtils.assertExceptionContains("Invalid trace ID", iae);
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  private static XceiverClient createClientForTesting(OzoneConfiguration conf)
-      throws Exception {
+  private static XceiverClientGrpc createClientForTesting(
+      OzoneConfiguration conf) throws Exception {
     // Start ozone container Via Datanode create.
     Pipeline pipeline =
         ContainerTestHelper.createSingleNodePipeline();
@@ -543,7 +512,7 @@ public class TestOzoneContainer {
             .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
     // This client talks to ozone container via datanode.
-    return new XceiverClient(pipeline, conf);
+    return new XceiverClientGrpc(pipeline, conf);
   }
 
   private static void createContainerForTesting(XceiverClientSpi client,

+ 0 - 25
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.scm;
 
 import com.google.common.cache.Cache;
 import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
@@ -36,12 +35,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
 
 import static org.apache.hadoop.hdds.scm
     .ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY;
@@ -49,23 +43,12 @@ import static org.apache.hadoop.hdds.scm
 /**
  * Test for XceiverClientManager caching and eviction.
  */
-@RunWith(Parameterized.class)
 public class TestXceiverClientManager {
   private static OzoneConfiguration config;
   private static MiniOzoneCluster cluster;
   private static StorageContainerLocationProtocolClientSideTranslatorPB
       storageContainerLocationClient;
   private static String containerOwner = "OZONE";
-  private static boolean shouldUseGrpc;
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> withGrpc() {
-    return Arrays.asList(new Object[][] {{false}, {true}});
-  }
-
-  public TestXceiverClientManager(boolean useGrpc) {
-    shouldUseGrpc = useGrpc;
-  }
 
   @Rule
   public ExpectedException exception = ExpectedException.none();
@@ -73,8 +56,6 @@ public class TestXceiverClientManager {
   @Before
   public void init() throws Exception {
     config = new OzoneConfiguration();
-    config.setBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        shouldUseGrpc);
     cluster = MiniOzoneCluster.newBuilder(config)
         .setNumDatanodes(3)
         .build();
@@ -94,8 +75,6 @@ public class TestXceiverClientManager {
   @Test
   public void testCaching() throws IOException {
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.setBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        shouldUseGrpc);
     XceiverClientManager clientManager = new XceiverClientManager(conf);
 
     ContainerWithPipeline container1 = storageContainerLocationClient
@@ -129,8 +108,6 @@ public class TestXceiverClientManager {
   public void testFreeByReference() throws IOException {
     OzoneConfiguration conf = new OzoneConfiguration();
     conf.setInt(SCM_CONTAINER_CLIENT_MAX_SIZE_KEY, 1);
-    conf.setBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        shouldUseGrpc);
     XceiverClientManager clientManager = new XceiverClientManager(conf);
     Cache<Long, XceiverClientSpi> cache =
         clientManager.getClientCache();
@@ -186,8 +163,6 @@ public class TestXceiverClientManager {
   public void testFreeByEviction() throws IOException {
     OzoneConfiguration conf = new OzoneConfiguration();
     conf.setInt(SCM_CONTAINER_CLIENT_MAX_SIZE_KEY, 1);
-    conf.setBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        shouldUseGrpc);
     XceiverClientManager clientManager = new XceiverClientManager(conf);
     Cache<Long, XceiverClientSpi> cache =
         clientManager.getClientCache();

+ 0 - 16
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java

@@ -29,7 +29,6 @@ import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -72,8 +71,6 @@ import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -83,7 +80,6 @@ import java.io.IOException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.Collection;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
@@ -102,7 +98,6 @@ import static org.junit.Assert.fail;
 /**
  * Test Ozone Key Lifecycle.
  */
-@RunWith(Parameterized.class)
 public class TestKeys {
   /**
    * Set the timeout for every test.
@@ -117,16 +112,7 @@ public class TestKeys {
   private static long currentTime;
   private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
   private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
-  private static boolean shouldUseGrpc;
 
-  @Parameterized.Parameters
-  public static Collection<Object[]> withGrpc() {
-    return Arrays.asList(new Object[][] {{false}, {true}});
-  }
-
-  public TestKeys(boolean useGrpc) {
-    shouldUseGrpc = useGrpc;
-  }
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -141,8 +127,6 @@ public class TestKeys {
     conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
         1000, TimeUnit.MILLISECONDS);
     conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS);
-    conf.setBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
-        shouldUseGrpc);
 
     path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);