|
@@ -21,17 +21,22 @@ package org.apache.hadoop.ozone.container;
|
|
|
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.hdfs.protocol.DatanodeID;
|
|
|
+import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
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.junit.Assert;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.net.ServerSocket;
|
|
@@ -42,11 +47,14 @@ import java.util.List;
|
|
|
import java.util.Random;
|
|
|
import java.util.UUID;
|
|
|
import java.util.Map;
|
|
|
+import java.util.stream.Collectors;
|
|
|
|
|
|
/**
|
|
|
* Helpers for container tests.
|
|
|
*/
|
|
|
public final class ContainerTestHelper {
|
|
|
+ public static final Logger LOG = LoggerFactory.getLogger(
|
|
|
+ ContainerTestHelper.class);
|
|
|
private static Random r = new Random();
|
|
|
|
|
|
/**
|
|
@@ -64,19 +72,50 @@ public final class ContainerTestHelper {
|
|
|
*/
|
|
|
public static Pipeline createSingleNodePipeline(String containerName) throws
|
|
|
IOException {
|
|
|
+ return createPipeline(containerName, 1);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static DatanodeID createDatanodeID() throws IOException {
|
|
|
ServerSocket socket = new ServerSocket(0);
|
|
|
int port = socket.getLocalPort();
|
|
|
DatanodeID datanodeID = new DatanodeID(socket.getInetAddress()
|
|
|
.getHostAddress(), socket.getInetAddress().getHostName(),
|
|
|
UUID.randomUUID().toString(), port, port, port, port);
|
|
|
datanodeID.setContainerPort(port);
|
|
|
- Pipeline pipeline = new Pipeline(datanodeID.getDatanodeUuid());
|
|
|
- pipeline.addMember(datanodeID);
|
|
|
- pipeline.setContainerName(containerName);
|
|
|
socket.close();
|
|
|
+ return datanodeID;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create a pipeline with single node replica.
|
|
|
+ *
|
|
|
+ * @return Pipeline with single node in it.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ public static Pipeline createPipeline(String containerName, int numNodes)
|
|
|
+ throws IOException {
|
|
|
+ Preconditions.checkArgument(numNodes >= 1);
|
|
|
+ final DatanodeID leader = createDatanodeID();
|
|
|
+ Pipeline pipeline = new Pipeline(leader.getDatanodeUuid());
|
|
|
+ pipeline.setContainerName(containerName);
|
|
|
+ pipeline.addMember(leader);
|
|
|
+
|
|
|
+ for(int i = 1; i < numNodes; i++) {
|
|
|
+ pipeline.addMember(createDatanodeID());
|
|
|
+ }
|
|
|
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.
|
|
|
*
|
|
@@ -133,6 +172,8 @@ public final class ContainerTestHelper {
|
|
|
public static ContainerCommandRequestProto getWriteChunkRequest(
|
|
|
Pipeline pipeline, String containerName, String keyName, int datalen)
|
|
|
throws IOException, NoSuchAlgorithmException {
|
|
|
+ LOG.trace("writeChunk {} (key={}) to pipeline=",
|
|
|
+ datalen, keyName, pipeline);
|
|
|
ContainerProtos.WriteChunkRequestProto.Builder writeRequest =
|
|
|
ContainerProtos.WriteChunkRequestProto
|
|
|
.newBuilder();
|
|
@@ -225,6 +266,9 @@ public final class ContainerTestHelper {
|
|
|
public static ContainerCommandRequestProto getReadChunkRequest(
|
|
|
ContainerProtos.WriteChunkRequestProto request)
|
|
|
throws IOException, NoSuchAlgorithmException {
|
|
|
+ LOG.trace("readChunk key={} from pipeline={}",
|
|
|
+ request.getKeyName(), request.getPipeline());
|
|
|
+
|
|
|
ContainerProtos.ReadChunkRequestProto.Builder readRequest =
|
|
|
ContainerProtos.ReadChunkRequestProto.newBuilder();
|
|
|
|
|
@@ -252,6 +296,9 @@ public final class ContainerTestHelper {
|
|
|
ContainerProtos.WriteChunkRequestProto writeRequest)
|
|
|
throws
|
|
|
IOException, NoSuchAlgorithmException {
|
|
|
+ LOG.trace("deleteChunk key={} from pipeline={}",
|
|
|
+ writeRequest.getKeyName(), writeRequest.getPipeline());
|
|
|
+
|
|
|
ContainerProtos.DeleteChunkRequestProto.Builder deleteRequest =
|
|
|
ContainerProtos.DeleteChunkRequestProto
|
|
|
.newBuilder();
|
|
@@ -275,6 +322,8 @@ public final class ContainerTestHelper {
|
|
|
*/
|
|
|
public static ContainerCommandRequestProto getCreateContainerRequest(
|
|
|
String containerName) throws IOException {
|
|
|
+ LOG.trace("createContainer: {}", containerName);
|
|
|
+
|
|
|
ContainerProtos.CreateContainerRequestProto.Builder createRequest =
|
|
|
ContainerProtos.CreateContainerRequestProto
|
|
|
.newBuilder();
|
|
@@ -358,6 +407,9 @@ public final class ContainerTestHelper {
|
|
|
*/
|
|
|
public static ContainerCommandRequestProto getPutKeyRequest(
|
|
|
ContainerProtos.WriteChunkRequestProto writeRequest) {
|
|
|
+ LOG.trace("putKey: {} to pipeline={}",
|
|
|
+ writeRequest.getKeyName(), writeRequest.getPipeline());
|
|
|
+
|
|
|
ContainerProtos.PutKeyRequestProto.Builder putRequest =
|
|
|
ContainerProtos.PutKeyRequestProto.newBuilder();
|
|
|
|
|
@@ -384,6 +436,9 @@ public final class ContainerTestHelper {
|
|
|
*/
|
|
|
public static ContainerCommandRequestProto getKeyRequest(
|
|
|
ContainerProtos.PutKeyRequestProto putKeyRequest) {
|
|
|
+ LOG.trace("getKey: name={} from pipeline={}",
|
|
|
+ putKeyRequest.getKeyData().getName(), putKeyRequest.getPipeline());
|
|
|
+
|
|
|
ContainerProtos.GetKeyRequestProto.Builder getRequest =
|
|
|
ContainerProtos.GetKeyRequestProto.newBuilder();
|
|
|
ContainerProtos.KeyData.Builder keyData = ContainerProtos.KeyData
|
|
@@ -422,6 +477,9 @@ public final class ContainerTestHelper {
|
|
|
*/
|
|
|
public static ContainerCommandRequestProto getDeleteKeyRequest(
|
|
|
ContainerProtos.PutKeyRequestProto putKeyRequest) {
|
|
|
+ LOG.trace("deleteKey: name={} from pipeline={}",
|
|
|
+ putKeyRequest.getKeyData().getName(), putKeyRequest.getPipeline());
|
|
|
+
|
|
|
ContainerProtos.DeleteKeyRequestProto.Builder delRequest =
|
|
|
ContainerProtos.DeleteKeyRequestProto.newBuilder();
|
|
|
delRequest.setPipeline(putKeyRequest.getPipeline());
|