diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java index 7b17f333fc..738a588510 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java @@ -37,6 +37,8 @@ import java.io.IOException; import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; /** @@ -56,11 +58,16 @@ public static XceiverClientRatis newXceiverClientRatis( } private final Pipeline pipeline; - private final RaftClient client; + private final RpcType rpcType; + private final AtomicReference client = new AtomicReference<>(); /** Constructs a client. */ - XceiverClientRatis(Pipeline pipeline, RpcType rpcType) { + private XceiverClientRatis(Pipeline pipeline, RpcType rpcType) { this.pipeline = pipeline; + this.rpcType = rpcType; + } + + static RaftClient newRaftClient(Pipeline pipeline, RpcType rpcType) { final List peers = pipeline.getMachines().stream() .map(dn -> dn.getXferAddr()) .map(addr -> new RaftPeer(new RaftPeerId(addr), addr)) @@ -70,7 +77,7 @@ public static XceiverClientRatis newXceiverClientRatis( final ClientFactory factory = ClientFactory.cast(rpcType.newFactory( properties, null)); - client = RaftClient.newBuilder() + return RaftClient.newBuilder() .setClientRpc(factory.newRaftClientRpc()) .setServers(peers) .setLeaderId(new RaftPeerId(pipeline.getLeader().getXferAddr())) @@ -85,23 +92,32 @@ public Pipeline getPipeline() { @Override public void connect() throws Exception { - // do nothing. + if (!client.compareAndSet(null, newRaftClient(pipeline, rpcType))) { + throw new IllegalStateException("Client is already connected."); + } } @Override public void close() { - try { - client.close(); - } catch (IOException e) { - throw new IllegalStateException(e); + final RaftClient c = client.getAndSet(null); + if (c != null) { + try { + c.close(); + } catch (IOException e) { + throw new IllegalStateException(e); + } } } + private RaftClient getClient() { + return Objects.requireNonNull(client.get(), "client is null"); + } + @Override public ContainerCommandResponseProto sendCommand( ContainerCommandRequestProto request) throws IOException { LOG.debug("sendCommand {}", request); - final RaftClientReply reply = client.send( + final RaftClientReply reply = getClient().send( () -> ShadedProtoUtil.asShadedByteString(request.toByteArray())); LOG.debug("reply {}", reply); Preconditions.checkState(reply.isSuccess()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java new file mode 100644 index 0000000000..d56fad2bd5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.scm.XceiverClientRatis; +import org.apache.hadoop.scm.container.common.helpers.Pipeline; +import org.apache.ratis.rpc.RpcType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.stream.Collectors; + +/** + * Helpers for Ratis tests. + */ +public interface RatisTestHelper { + Logger LOG = LoggerFactory.getLogger(RatisTestHelper.class); + + static void initRatisConf( + RpcType rpc, Pipeline pipeline, Configuration conf) { + conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true); + conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY, rpc.name()); + LOG.info(OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY + + " = " + rpc.name()); + final String s = pipeline.getMachines().stream() + .map(dn -> dn.getXferAddr()) + .collect(Collectors.joining(",")); + conf.setStrings(OzoneConfigKeys.DFS_CONTAINER_RATIS_CONF, s); + LOG.info(OzoneConfigKeys.DFS_CONTAINER_RATIS_CONF + " = " + s); + } + + static XceiverClientRatis newXceiverClientRatis( + RpcType rpcType, Pipeline pipeline, OzoneConfiguration conf) + throws IOException { + initRatisConf(rpcType, pipeline, conf); + return XceiverClientRatis.newXceiverClientRatis(pipeline, conf); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 89ad9501f3..a1abfebc61 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -21,20 +21,20 @@ import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import org.apache.commons.codec.binary.Hex; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos; import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos .ContainerCommandRequestProto; import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos .ContainerCommandResponseProto; -import org.apache.hadoop.ozone.protocol.proto.OzoneProtos; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; -import org.apache.hadoop.scm.container.common.helpers.Pipeline; import org.apache.hadoop.ozone.container.common.helpers.KeyData; -import org.apache.ratis.rpc.RpcType; +import org.apache.hadoop.ozone.protocol.proto.OzoneProtos; +import org.apache.hadoop.scm.container.common.helpers.Pipeline; +import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,12 +43,7 @@ import java.net.ServerSocket; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; -import java.util.LinkedList; -import java.util.List; -import java.util.Random; -import java.util.UUID; -import java.util.Map; -import java.util.stream.Collectors; +import java.util.*; /** * Helpers for container tests. @@ -64,6 +59,15 @@ public final class ContainerTestHelper { private ContainerTestHelper() { } + public static void setOzoneLocalStorageRoot( + Class clazz, OzoneConfiguration conf) { + String path = GenericTestUtils.getTempPath(clazz.getSimpleName()); + path += conf.getTrimmed( + OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, + OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT); + conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path); + } + // TODO: mock multi-node pipeline /** * Create a pipeline with single node replica. @@ -107,16 +111,6 @@ public static Pipeline createPipeline(String containerName, int numNodes) return pipeline; } - public static void initRatisConf( - RpcType rpc, Pipeline pipeline, Configuration conf) { - conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true); - conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY, rpc.name()); - conf.setStrings(OzoneConfigKeys.DFS_CONTAINER_RATIS_CONF, - pipeline.getMachines().stream() - .map(dn -> dn.getXferAddr()) - .collect(Collectors.joining(","))); - } - /** * Creates a ChunkInfo for testing. * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java index 73829d3584..266aa7f892 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java @@ -26,12 +26,8 @@ import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.apache.hadoop.scm.XceiverClient; -import org.apache.hadoop.scm.XceiverClientRatis; import org.apache.hadoop.scm.XceiverClientSpi; import org.apache.hadoop.scm.container.common.helpers.Pipeline; -import org.apache.hadoop.test.GenericTestUtils; -import org.apache.ratis.rpc.RpcType; -import org.apache.ratis.rpc.SupportedRpcType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -40,6 +36,7 @@ import java.util.HashMap; import java.util.Map; + /** * Tests ozone containers. */ @@ -53,12 +50,7 @@ public class TestOzoneContainer { @Test public void testCreateOzoneContainer() throws Exception { String containerName = OzoneUtils.getRequestID(); - OzoneConfiguration conf = new OzoneConfiguration(); - String path = GenericTestUtils - .getTempPath(TestOzoneContainer.class.getSimpleName()); - path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, - OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT); - conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path); + OzoneConfiguration conf = newOzoneConfiguration(); OzoneContainer container = null; MiniOzoneCluster cluster = null; try { @@ -76,12 +68,7 @@ public void testCreateOzoneContainer() throws Exception { XceiverClient client = new XceiverClient(pipeline, conf); client.connect(); - ContainerProtos.ContainerCommandRequestProto request = - ContainerTestHelper.getCreateContainerRequest(containerName); - ContainerProtos.ContainerCommandResponseProto response = - client.sendCommand(request); - Assert.assertNotNull(response); - Assert.assertTrue(request.getTraceID().equals(response.getTraceID())); + createContainerForTesting(client, containerName); } finally { if (container != null) { container.stop(); @@ -92,69 +79,19 @@ public void testCreateOzoneContainer() throws Exception { } } - @Test - public void testOzoneContainerViaDataNodeRatisGrpc() throws Exception { - runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.GRPC, 1); - runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.GRPC, 3); - } - - @Test - public void testOzoneContainerViaDataNodeRatisNetty() throws Exception { - runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.NETTY, 1); - runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.NETTY, 3); - } - - private static void runTestOzoneContainerViaDataNodeRatis( - RpcType rpc, int numNodes) throws Exception { - ContainerTestHelper.LOG.info("runTestOzoneContainerViaDataNodeRatis(rpc=" - + rpc + ", numNodes=" + numNodes); - - final String containerName = OzoneUtils.getRequestID(); - final Pipeline pipeline = ContainerTestHelper.createPipeline( - containerName, numNodes); - final OzoneConfiguration conf = initOzoneConfiguration(pipeline); - ContainerTestHelper.initRatisConf(rpc, pipeline, conf); - - final MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf) - .setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL) - .numDataNodes(pipeline.getMachines().size()) - .build(); - cluster.waitOzoneReady(); - final XceiverClientSpi client = XceiverClientRatis.newXceiverClientRatis( - pipeline, conf); - - try { - runTestOzoneContainerViaDataNode(containerName, client); - } finally { - cluster.shutdown(); - } - } - - private static OzoneConfiguration initOzoneConfiguration(Pipeline pipeline) { + static OzoneConfiguration newOzoneConfiguration() { final OzoneConfiguration conf = new OzoneConfiguration(); - conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, - pipeline.getLeader().getContainerPort()); - - setOzoneLocalStorageRoot(conf); + ContainerTestHelper.setOzoneLocalStorageRoot( + TestOzoneContainer.class, conf); return conf; } - private static void setOzoneLocalStorageRoot(OzoneConfiguration conf) { - String path = GenericTestUtils - .getTempPath(TestOzoneContainer.class.getSimpleName()); - path += conf.getTrimmed( - OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, - OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT); - conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path); - } - @Test public void testOzoneContainerViaDataNode() throws Exception { MiniOzoneCluster cluster = null; try { String containerName = OzoneUtils.getRequestID(); - OzoneConfiguration conf = new OzoneConfiguration(); - setOzoneLocalStorageRoot(conf); + OzoneConfiguration conf = newOzoneConfiguration(); // Start ozone container Via Datanode create. @@ -178,8 +115,8 @@ public void testOzoneContainerViaDataNode() throws Exception { } } - static void runTestOzoneContainerViaDataNode(String containerName, - XceiverClientSpi client) throws Exception { + static void runTestOzoneContainerViaDataNode( + String containerName, XceiverClientSpi client) throws Exception { ContainerProtos.ContainerCommandRequestProto request, writeChunkRequest, putKeyRequest, updateRequest1, updateRequest2; @@ -261,33 +198,44 @@ static void runTestOzoneContainerViaDataNode(String containerName, public void testBothGetandPutSmallFile() throws Exception { MiniOzoneCluster cluster = null; XceiverClient client = null; - ContainerProtos.ContainerCommandResponseProto response; - ContainerProtos.ContainerCommandRequestProto - smallFileRequest, getSmallFileRequest; try { - String keyName = OzoneUtils.getRequestID(); - OzoneConfiguration conf = new OzoneConfiguration(); + OzoneConfiguration conf = newOzoneConfiguration(); client = createClientForTesting(conf); cluster = new MiniOzoneCluster.Builder(conf) .setRandomContainerPort(false) .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build(); + String containerName = client.getPipeline().getContainerName(); + + runTestBothGetandPutSmallFile(containerName, client); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + static void runTestBothGetandPutSmallFile( + String containerName, XceiverClientSpi client) throws Exception { + try { client.connect(); - String containerName = client.getPipeline().getContainerName(); createContainerForTesting(client, containerName); - smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest( - client.getPipeline(), containerName, keyName, 1024); + String keyName = OzoneUtils.getRequestID(); + final ContainerProtos.ContainerCommandRequestProto smallFileRequest + = ContainerTestHelper.getWriteSmallFileRequest( + client.getPipeline(), containerName, keyName, 1024); - response = client.sendCommand(smallFileRequest); + ContainerProtos.ContainerCommandResponseProto response + = client.sendCommand(smallFileRequest); Assert.assertNotNull(response); Assert.assertTrue(smallFileRequest.getTraceID() .equals(response.getTraceID())); - getSmallFileRequest = - ContainerTestHelper.getReadSmallFileRequest(smallFileRequest - .getPutSmallFile().getKey()); + final ContainerProtos.ContainerCommandRequestProto getSmallFileRequest + = ContainerTestHelper.getReadSmallFileRequest( + smallFileRequest.getPutSmallFile().getKey()); response = client.sendCommand(getSmallFileRequest); Assert.assertArrayEquals( smallFileRequest.getPutSmallFile().getData().toByteArray(), @@ -296,9 +244,6 @@ public void testBothGetandPutSmallFile() throws Exception { if (client != null) { client.close(); } - if (cluster != null) { - cluster.shutdown(); - } } } @@ -311,7 +256,7 @@ public void testCloseContainer() throws Exception { writeChunkRequest, putKeyRequest, request; try { - OzoneConfiguration conf = new OzoneConfiguration(); + OzoneConfiguration conf = newOzoneConfiguration(); client = createClientForTesting(conf); cluster = new MiniOzoneCluster.Builder(conf) @@ -399,7 +344,7 @@ public void testDeleteContainer() throws Exception { ContainerProtos.ContainerCommandRequestProto request, writeChunkRequest, putKeyRequest; try { - OzoneConfiguration conf = new OzoneConfiguration(); + OzoneConfiguration conf = newOzoneConfiguration(); client = createClientForTesting(conf); cluster = new MiniOzoneCluster.Builder(conf) @@ -470,18 +415,10 @@ public void testDeleteContainer() throws Exception { } } - private XceiverClient createClientForTesting(OzoneConfiguration conf) + private static XceiverClient createClientForTesting(OzoneConfiguration conf) throws Exception { - String containerName = OzoneUtils.getRequestID(); - String path = GenericTestUtils - .getTempPath(TestOzoneContainer.class.getSimpleName()); - path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, - OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT); - conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path); - // Start ozone container Via Datanode create. - Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(containerName); conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java new file mode 100644 index 0000000000..2662909eba --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.container.ozoneimpl; + +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConfiguration; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.RatisTestHelper; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.web.utils.OzoneUtils; +import org.apache.hadoop.scm.XceiverClientSpi; +import org.apache.hadoop.scm.container.common.helpers.Pipeline; +import org.apache.ratis.rpc.RpcType; +import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.util.CheckedBiConsumer; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tests ozone containers with Apache Ratis. + */ +public class TestOzoneContainerRatis { + private static final Logger LOG = LoggerFactory.getLogger( + TestOzoneContainerRatis.class); + + static OzoneConfiguration newOzoneConfiguration() { + final OzoneConfiguration conf = new OzoneConfiguration(); + ContainerTestHelper.setOzoneLocalStorageRoot( + TestOzoneContainerRatis.class, conf); + return conf; + } + + + /** Set the timeout for every test. */ + @Rule + public Timeout testTimeout = new Timeout(300000); + + @Test + public void testOzoneContainerViaDataNodeRatisGrpc() throws Exception { + runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.GRPC, 1); + runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.GRPC, 3); + } + + @Test + public void testOzoneContainerViaDataNodeRatisNetty() throws Exception { + runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.NETTY, 1); + runTestOzoneContainerViaDataNodeRatis(SupportedRpcType.NETTY, 3); + } + + private static void runTestOzoneContainerViaDataNodeRatis( + RpcType rpc, int numNodes) throws Exception { + runTest("runTestOzoneContainerViaDataNodeRatis", rpc, numNodes, + TestOzoneContainer::runTestOzoneContainerViaDataNode); + } + + private static void runTest( + String testName, RpcType rpc, int numNodes, + CheckedBiConsumer test) + throws Exception { + LOG.info(testName + "(rpc=" + rpc + ", numNodes=" + numNodes); + + final OzoneConfiguration conf = newOzoneConfiguration(); + final String containerName = OzoneUtils.getRequestID(); + final Pipeline pipeline = ContainerTestHelper.createPipeline( + containerName, numNodes); + final XceiverClientSpi client = RatisTestHelper.newXceiverClientRatis( + rpc, pipeline, conf); + + final MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf) + .setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL) + .numDataNodes(pipeline.getMachines().size()) + .build(); + cluster.waitOzoneReady(); + + try { + test.accept(containerName, client); + } finally { + cluster.shutdown(); + } + } + + private static void runTestBothGetandPutSmallFileRatis( + RpcType rpc, int numNodes) throws Exception { + runTest("runTestBothGetandPutSmallFileRatis", rpc, numNodes, + TestOzoneContainer::runTestBothGetandPutSmallFile); + } + + @Test + public void testBothGetandPutSmallFileRatisNetty() throws Exception { + runTestBothGetandPutSmallFileRatis(SupportedRpcType.NETTY, 1); + runTestBothGetandPutSmallFileRatis(SupportedRpcType.NETTY, 3); + } + + @Test + public void testBothGetandPutSmallFileRatisGrpc() throws Exception { + runTestBothGetandPutSmallFileRatis(SupportedRpcType.GRPC, 1); + runTestBothGetandPutSmallFileRatis(SupportedRpcType.GRPC, 3); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java index e7ae679729..308bc9006d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfiguration; +import org.apache.hadoop.ozone.RatisTestHelper; import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.common.impl.Dispatcher; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; @@ -120,7 +121,7 @@ static XceiverServerRatis newXceiverServerRatis( static void runTestClientServerRatis(RpcType rpc, int numNodes) throws Exception { runTestClientServer(numNodes, - (pipeline, conf) -> ContainerTestHelper.initRatisConf( + (pipeline, conf) -> RatisTestHelper.initRatisConf( rpc, pipeline, conf), XceiverClientRatis::newXceiverClientRatis, TestContainerServer::newXceiverServerRatis);