|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.ozone.container.common.impl;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+import com.google.protobuf.ByteString;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
|
.ContainerCommandRequestProto;
|
|
@@ -29,15 +30,18 @@ import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
|
+import org.apache.hadoop.ozone.container.common.helpers.FileUtils;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
|
|
-import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
|
|
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.LinkedList;
|
|
|
+import java.util.List;
|
|
|
|
|
|
/**
|
|
|
* Ozone Container dispatcher takes a call from the netty server and routes it
|
|
@@ -69,7 +73,6 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
(cmdType == Type.ReadContainer) ||
|
|
|
(cmdType == Type.ListContainer) ||
|
|
|
(cmdType == Type.UpdateContainer)) {
|
|
|
-
|
|
|
return containerProcessHandler(msg);
|
|
|
}
|
|
|
|
|
@@ -86,6 +89,11 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
return chunkProcessHandler(msg);
|
|
|
}
|
|
|
|
|
|
+ if ((cmdType == Type.PutSmallFile) ||
|
|
|
+ (cmdType == Type.GetSmallFile)) {
|
|
|
+ return smallFileHandler(msg);
|
|
|
+ }
|
|
|
+
|
|
|
return ContainerUtils.unsupportedRequest(msg);
|
|
|
}
|
|
|
|
|
@@ -219,6 +227,18 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private ContainerCommandResponseProto smallFileHandler(
|
|
|
+ ContainerCommandRequestProto msg) throws IOException {
|
|
|
+ switch (msg.getCmdType()) {
|
|
|
+ case PutSmallFile:
|
|
|
+ return handlePutSmallFile(msg);
|
|
|
+ case GetSmallFile:
|
|
|
+ return handleGetSmallFile(msg);
|
|
|
+ default:
|
|
|
+ return ContainerUtils.unsupportedRequest(msg);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Calls into container logic and returns appropriate response.
|
|
|
*
|
|
@@ -387,7 +407,7 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
*/
|
|
|
private ContainerCommandResponseProto handlePutKey(
|
|
|
ContainerCommandRequestProto msg) throws IOException {
|
|
|
- if(!msg.hasPutKey()){
|
|
|
+ if (!msg.hasPutKey()) {
|
|
|
LOG.debug("Malformed put key request. trace ID: {}",
|
|
|
msg.getTraceID());
|
|
|
return ContainerUtils.malformedRequest(msg);
|
|
@@ -409,7 +429,7 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
*/
|
|
|
private ContainerCommandResponseProto handleGetKey(
|
|
|
ContainerCommandRequestProto msg) throws IOException {
|
|
|
- if(!msg.hasGetKey()){
|
|
|
+ if (!msg.hasGetKey()) {
|
|
|
LOG.debug("Malformed get key request. trace ID: {}",
|
|
|
msg.getTraceID());
|
|
|
return ContainerUtils.malformedRequest(msg);
|
|
@@ -430,7 +450,7 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
*/
|
|
|
private ContainerCommandResponseProto handleDeleteKey(
|
|
|
ContainerCommandRequestProto msg) throws IOException {
|
|
|
- if(!msg.hasDeleteKey()){
|
|
|
+ if (!msg.hasDeleteKey()) {
|
|
|
LOG.debug("Malformed delete key request. trace ID: {}",
|
|
|
msg.getTraceID());
|
|
|
return ContainerUtils.malformedRequest(msg);
|
|
@@ -447,4 +467,73 @@ public class Dispatcher implements ContainerDispatcher {
|
|
|
return KeyUtils.getKeyResponse(msg);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Handles writing a chunk and associated key using single RPC.
|
|
|
+ *
|
|
|
+ * @param msg - Message.
|
|
|
+ * @return ContainerCommandResponseProto
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private ContainerCommandResponseProto handlePutSmallFile(
|
|
|
+ ContainerCommandRequestProto msg) throws IOException {
|
|
|
+
|
|
|
+ if (!msg.hasPutSmallFile()) {
|
|
|
+ LOG.debug("Malformed put small file request. trace ID: {}",
|
|
|
+ msg.getTraceID());
|
|
|
+ return ContainerUtils.malformedRequest(msg);
|
|
|
+ }
|
|
|
+
|
|
|
+ Pipeline pipeline =
|
|
|
+ Pipeline.getFromProtoBuf(msg.getPutSmallFile().getKey().getPipeline());
|
|
|
+ Preconditions.checkNotNull(pipeline);
|
|
|
+ KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
|
|
|
+ .getKeyData());
|
|
|
+ ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getPutSmallFile()
|
|
|
+ .getChunkInfo());
|
|
|
+ byte[] data = msg.getPutSmallFile().getData().toByteArray();
|
|
|
+
|
|
|
+ this.containerManager.getChunkManager().writeChunk(pipeline, keyData
|
|
|
+ .getKeyName(), chunkInfo, data);
|
|
|
+ List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
|
|
|
+ chunks.add(chunkInfo.getProtoBufMessage());
|
|
|
+ keyData.setChunks(chunks);
|
|
|
+ this.containerManager.getKeyManager().putKey(pipeline, keyData);
|
|
|
+ return FileUtils.getPutFileResponse(msg);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Handles getting a data stream using a key. This helps in reducing the RPC
|
|
|
+ * overhead for small files.
|
|
|
+ *
|
|
|
+ * @param msg - ContainerCommandRequestProto
|
|
|
+ * @return ContainerCommandResponseProto
|
|
|
+ */
|
|
|
+ private ContainerCommandResponseProto handleGetSmallFile(
|
|
|
+ ContainerCommandRequestProto msg) throws IOException {
|
|
|
+ ByteString dataBuf = ByteString.EMPTY;
|
|
|
+ if (!msg.hasGetSmallFile()) {
|
|
|
+ LOG.debug("Malformed get small file request. trace ID: {}",
|
|
|
+ msg.getTraceID());
|
|
|
+ return ContainerUtils.malformedRequest(msg);
|
|
|
+ }
|
|
|
+ Pipeline pipeline =
|
|
|
+ Pipeline.getFromProtoBuf(msg.getGetSmallFile().getKey().getPipeline());
|
|
|
+ Preconditions.checkNotNull(pipeline);
|
|
|
+ KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
|
|
|
+ .getKey().getKeyData());
|
|
|
+ KeyData data = this.containerManager.getKeyManager().getKey(keyData);
|
|
|
+ ContainerProtos.ChunkInfo c = null;
|
|
|
+ for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
|
|
|
+ ByteString current =
|
|
|
+ ByteString.copyFrom(this.containerManager.getChunkManager().readChunk(
|
|
|
+ pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
|
|
|
+ chunk)));
|
|
|
+ dataBuf = dataBuf.concat(current);
|
|
|
+ c = chunk;
|
|
|
+ }
|
|
|
+
|
|
|
+ return FileUtils.getGetSmallFileResponse(msg, dataBuf.toByteArray(),
|
|
|
+ ChunkInfo.getFromProtoBuf(c));
|
|
|
+ }
|
|
|
+
|
|
|
}
|