byteBufferToByteString) {
- boolean isReadChunkV0 = getReadChunkVersion(request.getReadChunk())
- .equals(ContainerProtos.ReadChunkVersion.V0);
+ boolean isReadChunkV0 = getReadChunkVersion(request.getReadChunk()).equals(ContainerProtos.ReadChunkVersion.V0);
ReadChunkResponseProto.Builder response;
@@ -333,8 +349,16 @@ public static ContainerCommandResponseProto getReadChunkResponse(
.build();
}
- public static ContainerCommandResponseProto getFinalizeBlockResponse(
- ContainerCommandRequestProto msg, BlockData data) {
+ /**
+ * Generates a successful response for the FinalizeBlock operation.
+ *
+ * @param msg The request message containing the FinalizeBlock command.
+ * @param data The block data associated with the FinalizeBlock operation.
+ * @return A ContainerCommandResponseProto object indicating the success of the FinalizeBlock operation
+ * and containing relevant response data.
+ */
+ public static ContainerCommandResponseProto getFinalizeBlockResponse(ContainerCommandRequestProto msg,
+ BlockData data) {
ContainerProtos.FinalizeBlockResponseProto.Builder blockData =
ContainerProtos.FinalizeBlockResponseProto.newBuilder()
@@ -345,9 +369,15 @@ public static ContainerCommandResponseProto getFinalizeBlockResponse(
.build();
}
- public static ContainerCommandResponseProto getEchoResponse(
- ContainerCommandRequestProto msg) {
-
+ /**
+ * Generates an echo response based on the provided request message.
+ * The response contains a random payload of the specified size and optionally simulates a delay before responding.
+ *
+ * @param msg The request message of type ContainerCommandRequestProto,
+ * containing the EchoRequest with payload size and optional sleep time.
+ * @return A ContainerCommandResponseProto object containing the echo response with a random payload.
+ */
+ public static ContainerCommandResponseProto getEchoResponse(ContainerCommandRequestProto msg) {
ContainerProtos.EchoRequestProto echoRequest = msg.getEcho();
int responsePayload = echoRequest.getPayloadSizeResp();
@@ -360,10 +390,9 @@ public static ContainerCommandResponseProto getEchoResponse(
throw new RuntimeException(e);
}
- ContainerProtos.EchoResponseProto.Builder echo =
- ContainerProtos.EchoResponseProto
- .newBuilder()
- .setPayload(UnsafeByteOperations.unsafeWrap(RandomUtils.nextBytes(responsePayload)));
+ ContainerProtos.EchoResponseProto.Builder echo = ContainerProtos.EchoResponseProto
+ .newBuilder()
+ .setPayload(UnsafeByteOperations.unsafeWrap(RandomUtils.nextBytes(responsePayload)));
return getSuccessResponseBuilder(msg)
.setEcho(echo)
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerApi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerApi.java
new file mode 100644
index 00000000000..a2e886a5897
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerApi.java
@@ -0,0 +1,35 @@
+/*
+
+ * 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.hdds.scm.storage;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.VerifyBlockResponseProto;
+
+/**
+ * Interface for communication with a datanode.
+ * Provides methods to perform any protocol calls by Container clients on a single datanode.
+ */
+public interface ContainerApi extends AutoCloseable {
+ /**
+ * Verifies the integrity and validity of a block within the container.
+ *
+ * @return a VerifyBlockResponseProto object containing the result of the block verification operation.
+ */
+ VerifyBlockResponseProto verifyBlock();
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerApiHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerApiHelper.java
new file mode 100644
index 00000000000..ad45a7ccb81
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerApiHelper.java
@@ -0,0 +1,71 @@
+/*
+ * 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.hdds.scm.storage;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.VerifyBlockRequestProto;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.security.token.Token;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.VerifyBlock;
+
+/**
+ * Class designed working with Datanode Proto requests and responses.
+ */
+class ContainerApiHelper {
+ /**
+ * Creates a request to verify a block on the datanode.
+ *
+ * @param datanodeBlockID The identifier for the block on the datanode.
+ * @param token The security token used for authentication.
+ * @param datanodeUuid The unique identifier of the datanode.
+ * @return A {@link ContainerCommandRequestProto} object representing the verify block request.
+ * @throws IOException If an I/O error occurs during the request creation.
+ */
+ ContainerCommandRequestProto createVerifyBlockRequest(DatanodeBlockID datanodeBlockID,
+ Token token, String datanodeUuid) throws IOException {
+
+ VerifyBlockRequestProto.Builder verifyBlockRequestBuilder = ContainerProtos.VerifyBlockRequestProto
+ .newBuilder()
+ .setBlockID(datanodeBlockID);
+
+ ContainerCommandRequestProto.Builder commandRequestBuilder = ContainerCommandRequestProto
+ .newBuilder()
+ .setCmdType(VerifyBlock)
+ .setContainerID(datanodeBlockID.getContainerID())
+ .setDatanodeUuid(datanodeUuid)
+ .setVerifyBlock(verifyBlockRequestBuilder);
+
+ if (token != null) {
+ commandRequestBuilder.setEncodedToken(token.encodeToUrlString());
+ }
+
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ commandRequestBuilder.setTraceID(traceId);
+ }
+
+ return commandRequestBuilder.build();
+ }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerMultinodeApi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerMultinodeApi.java
new file mode 100644
index 00000000000..6dc9b13801a
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerMultinodeApi.java
@@ -0,0 +1,47 @@
+/*
+ * 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.hdds.scm.storage;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.VerifyBlockResponseProto;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Interface for communication with multiple datanodes.
+ * Provides methods to perform any protocol calls by Container clients on multiple datanodes.
+ */
+public interface ContainerMultinodeApi extends AutoCloseable {
+ /**
+ * Verifies the specified block on multiple datanodes.
+ *
+ * @param datanodeBlockID the ID of the block to be verified
+ * @param token the security token required for block verification
+ * @return a map containing the datanode details and their respective verification response
+ * @throws IOException if an I/O error occurs during verification
+ * @throws InterruptedException if the verification process is interrupted
+ */
+ Map verifyBlock(DatanodeBlockID datanodeBlockID,
+ Token token) throws IOException, InterruptedException;
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerMultinodeApiImpl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerMultinodeApiImpl.java
new file mode 100644
index 00000000000..50695a4ed9f
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerMultinodeApiImpl.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hdds.scm.storage;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.VerifyBlockResponseProto;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Implementation of the {@link ContainerMultinodeApi} interface.
+ * This class provides methods to perform protocol calls on multiple datanodes.
+ */
+public class ContainerMultinodeApiImpl implements ContainerMultinodeApi {
+
+ private final XceiverClientSpi client;
+
+ private final ContainerApiHelper requestHelper = new ContainerApiHelper();
+
+ public ContainerMultinodeApiImpl(XceiverClientSpi client) {
+ this.client = client;
+ }
+
+ @Override
+ public Map verifyBlock(DatanodeBlockID datanodeBlockID,
+ Token token) throws IOException, InterruptedException {
+
+ String datanodeUuid = client.getPipeline().getFirstNode().getUuidString();
+
+ Map datanodeToResponseMap = new HashMap<>();
+
+ ContainerCommandRequestProto request = requestHelper.createVerifyBlockRequest(datanodeBlockID, token, datanodeUuid);
+ Map responses = client.sendCommandOnAllNodes(request);
+
+ responses.forEach((key, value) -> datanodeToResponseMap.put(key, value.getVerifyBlock()));
+
+ return datanodeToResponseMap;
+ }
+
+ @Override
+ public void close() throws IOException {
+ client.close();
+ }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index cb41479b5f3..2108c5b7fa4 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -85,12 +85,10 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED;
/**
- * Implementation of all container protocol calls performed by Container
- * clients.
+ * Implementation of all container protocol calls performed by Container clients.
*/
public final class ContainerProtocolCalls {
- private static final Logger LOG =
- LoggerFactory.getLogger(ContainerProtocolCalls.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ContainerProtocolCalls.class);
/**
* There is no need to instantiate this class.
@@ -105,25 +103,21 @@ private ContainerProtocolCalls() {
* @param containerID the ID of the container to list block
* @param startLocalID the localID of the first block to get
* @param count max number of blocks to get
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return container protocol list block response
* @throws IOException if there is an I/O error while performing the call
*/
- public static ListBlockResponseProto listBlock(XceiverClientSpi xceiverClient,
- long containerID, Long startLocalID, int count,
- Token extends TokenIdentifier> token) throws IOException {
+ public static ListBlockResponseProto listBlock(XceiverClientSpi xceiverClient, long containerID, Long startLocalID,
+ int count, Token extends TokenIdentifier> token) throws IOException {
- ListBlockRequestProto.Builder listBlockBuilder =
- ListBlockRequestProto.newBuilder()
- .setCount(count);
+ ListBlockRequestProto.Builder listBlockBuilder = ListBlockRequestProto.newBuilder().setCount(count);
if (startLocalID != null) {
listBlockBuilder.setStartLocalID(startLocalID);
}
// datanodeID doesn't matter for read only requests
- String datanodeID =
- xceiverClient.getPipeline().getFirstNode().getUuidString();
+ String datanodeID = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder builder =
ContainerCommandRequestProto.newBuilder()
@@ -132,24 +126,11 @@ public static ListBlockResponseProto listBlock(XceiverClientSpi xceiverClient,
.setDatanodeUuid(datanodeID)
.setListBlock(listBlockBuilder.build());
- if (token != null) {
- builder.setEncodedToken(token.encodeToUrlString());
- }
- String traceId = TracingUtil.exportCurrentSpan();
- if (traceId != null) {
- builder.setTraceID(traceId);
- }
-
- ContainerCommandRequestProto request = builder.build();
- ContainerCommandResponseProto response =
- xceiverClient.sendCommand(request, getValidatorList());
- return response.getListBlock();
+ return sendContainerCommand(xceiverClient, builder, token).getListBlock();
}
- static T tryEachDatanode(Pipeline pipeline,
- CheckedFunction op,
- Function toErrorMessage)
- throws IOException {
+ static T tryEachDatanode(Pipeline pipeline, CheckedFunction op,
+ Function toErrorMessage) throws IOException {
final Set excluded = new HashSet<>();
for (; ;) {
final DatanodeDetails d = pipeline.getClosestNode(excluded);
@@ -170,8 +151,7 @@ static T tryEachDatanode(Pipeline pipeline,
span.log("failed to connect to DN " + d);
excluded.add(d);
if (excluded.size() < pipeline.size()) {
- LOG.warn(toErrorMessage.apply(d)
- + "; will try another datanode.", e);
+ LOG.warn("{}; will try another datanode.", toErrorMessage.apply(d), e);
} else {
throw e;
}
@@ -185,7 +165,7 @@ static T tryEachDatanode(Pipeline pipeline,
* @param xceiverClient client to perform call
* @param validators functions to validate the response
* @param blockID blockID to identify container
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return container protocol get block response
* @throws IOException if there is an I/O error while performing the call
*/
@@ -210,32 +190,21 @@ static String toErrorMessage(BlockID blockId, DatanodeDetails d) {
blockId.getLocalID(), blockId.getContainerID(), d);
}
- public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
- BlockID datanodeBlockID,
+ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, BlockID datanodeBlockID,
Token extends TokenIdentifier> token, Map replicaIndexes) throws IOException {
return getBlock(xceiverClient, getValidatorList(), datanodeBlockID, token, replicaIndexes);
}
- private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
- List validators,
- ContainerCommandRequestProto.Builder builder, BlockID blockID,
- DatanodeDetails datanode, Map replicaIndexes) throws IOException {
- String traceId = TracingUtil.exportCurrentSpan();
- if (traceId != null) {
- builder.setTraceID(traceId);
- }
- final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder();
- int replicaIndex = replicaIndexes.getOrDefault(datanode, 0);
- if (replicaIndex > 0) {
- datanodeBlockID.setReplicaIndex(replicaIndex);
- }
+ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, List validators,
+ ContainerCommandRequestProto.Builder builder, BlockID blockID, DatanodeDetails datanode,
+ Map replicaIndexes) throws IOException {
+ DatanodeBlockID datanodeBlockID = buildDatanodeId(builder, blockID, datanode, replicaIndexes);
final GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto.newBuilder()
- .setBlockID(datanodeBlockID.build());
+ .setBlockID(datanodeBlockID);
final ContainerCommandRequestProto request = builder
.setDatanodeUuid(datanode.getUuidString())
.setGetBlock(readBlockRequest).build();
- ContainerCommandResponseProto response =
- xceiverClient.sendCommand(request, validators);
+ ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators);
return response.getGetBlock();
}
@@ -244,17 +213,13 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
*
* @param xceiverClient client to perform call
* @param blockID blockId for the Block
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return container protocol getLastCommittedBlockLength response
* @throws IOException if there is an I/O error while performing the call
*/
- public static ContainerProtos.GetCommittedBlockLengthResponseProto
- getCommittedBlockLength(
- XceiverClientSpi xceiverClient, BlockID blockID,
- Token token)
- throws IOException {
- ContainerProtos.GetCommittedBlockLengthRequestProto.Builder
- getBlockLengthRequestBuilder =
+ public static ContainerProtos.GetCommittedBlockLengthResponseProto getCommittedBlockLength(
+ XceiverClientSpi xceiverClient, BlockID blockID, Token token) throws IOException {
+ ContainerProtos.GetCommittedBlockLengthRequestProto.Builder getBlockLengthRequestBuilder =
ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder().
setBlockID(blockID.getDatanodeBlockIDProtobuf());
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
@@ -272,8 +237,7 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
builder.setTraceID(traceId);
}
ContainerCommandRequestProto request = builder.build();
- ContainerCommandResponseProto response =
- xceiverClient.sendCommand(request, getValidatorList());
+ ContainerCommandResponseProto response = xceiverClient.sendCommand(request, getValidatorList());
return response.getGetCommittedBlockLength();
}
@@ -283,17 +247,14 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
* @param xceiverClient client to perform call
* @param containerBlockData block data to identify container
* @param eof whether this is the last putBlock for the same block
- * @param tokenString a serialized token for this block (may be null)
+ * @param tokenString a serialized token for this block (maybe null)
* @return putBlockResponse
* @throws IOException if there is an error while performing the call
*/
- public static XceiverClientReply putBlockAsync(XceiverClientSpi xceiverClient,
- BlockData containerBlockData,
- boolean eof,
- String tokenString)
- throws IOException, InterruptedException, ExecutionException {
- final ContainerCommandRequestProto request = getPutBlockRequest(
- xceiverClient.getPipeline(), containerBlockData, eof, tokenString);
+ public static XceiverClientReply putBlockAsync(XceiverClientSpi xceiverClient, BlockData containerBlockData,
+ boolean eof, String tokenString) throws IOException, InterruptedException, ExecutionException {
+ final ContainerCommandRequestProto request =
+ getPutBlockRequest(xceiverClient.getPipeline(), containerBlockData, eof, tokenString);
return xceiverClient.sendCommandAsync(request);
}
@@ -302,16 +263,13 @@ public static XceiverClientReply putBlockAsync(XceiverClientSpi xceiverClient,
*
* @param xceiverClient client to perform call
* @param blockID block ID to identify block
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return FinalizeBlockResponseProto
* @throws IOException if there is an I/O error while performing the call
*/
- public static ContainerProtos.FinalizeBlockResponseProto finalizeBlock(
- XceiverClientSpi xceiverClient, DatanodeBlockID blockID,
- Token token)
- throws IOException {
- FinalizeBlockRequestProto.Builder finalizeBlockRequest =
- FinalizeBlockRequestProto.newBuilder().setBlockID(blockID);
+ public static ContainerProtos.FinalizeBlockResponseProto finalizeBlock(XceiverClientSpi xceiverClient,
+ DatanodeBlockID blockID, Token token) throws IOException {
+ FinalizeBlockRequestProto.Builder finalizeBlockRequest = FinalizeBlockRequestProto.newBuilder().setBlockID(blockID);
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder builder =
ContainerCommandRequestProto.newBuilder().setCmdType(Type.FinalizeBlock)
@@ -322,14 +280,22 @@ public static ContainerProtos.FinalizeBlockResponseProto finalizeBlock(
builder.setEncodedToken(token.encodeToUrlString());
}
ContainerCommandRequestProto request = builder.build();
- ContainerCommandResponseProto response =
- xceiverClient.sendCommand(request, getValidatorList());
+ ContainerCommandResponseProto response = xceiverClient.sendCommand(request, getValidatorList());
return response.getFinalizeBlock();
}
- public static ContainerCommandRequestProto getPutBlockRequest(
- Pipeline pipeline, BlockData containerBlockData, boolean eof,
- String tokenString) throws IOException {
+ /**
+ * Constructs and returns a ContainerCommandRequestProto for a PutBlock operation.
+ *
+ * @param pipeline the pipeline that specifies the nodes involved in the operation
+ * @param containerBlockData the block data to be put
+ * @param eof flag indicating if this is the end of the file
+ * @param tokenString an optional security token for the operation may be null
+ * @return a ContainerCommandRequestProto representing the PutBlock request
+ * @throws IOException if an I/O error occurs during request creation
+ */
+ public static ContainerCommandRequestProto getPutBlockRequest(Pipeline pipeline, BlockData containerBlockData,
+ boolean eof, String tokenString) throws IOException {
PutBlockRequestProto.Builder createBlockRequest =
PutBlockRequestProto.newBuilder()
.setBlockData(containerBlockData)
@@ -353,14 +319,12 @@ public static ContainerCommandRequestProto getPutBlockRequest(
* @param chunk information about chunk to read
* @param blockID ID of the block
* @param validators functions to validate the response
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return container protocol read chunk response
* @throws IOException if there is an I/O error while performing the call
*/
- public static ContainerProtos.ReadChunkResponseProto readChunk(
- XceiverClientSpi xceiverClient, ChunkInfo chunk, DatanodeBlockID blockID,
- List validators,
- Token extends TokenIdentifier> token) throws IOException {
+ public static ContainerProtos.ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
+ DatanodeBlockID blockID, List validators, Token extends TokenIdentifier> token) throws IOException {
ReadChunkRequestProto.Builder readChunkRequest =
ReadChunkRequestProto.newBuilder()
.setBlockID(blockID)
@@ -374,25 +338,21 @@ public static ContainerProtos.ReadChunkResponseProto readChunk(
builder.setEncodedToken(token.encodeToUrlString());
}
- Span span = GlobalTracer.get()
- .buildSpan("readChunk").start();
+ Span span = GlobalTracer.get().buildSpan("readChunk").start();
try (Scope ignored = GlobalTracer.get().activateSpan(span)) {
span.setTag("offset", chunk.getOffset())
.setTag("length", chunk.getLen())
.setTag("block", blockID.toString());
return tryEachDatanode(xceiverClient.getPipeline(),
- d -> readChunk(xceiverClient, chunk, blockID,
- validators, builder, d),
+ d -> readChunk(xceiverClient, chunk, blockID, validators, builder, d),
d -> toErrorMessage(chunk, blockID, d));
} finally {
span.finish();
}
}
- private static ContainerProtos.ReadChunkResponseProto readChunk(
- XceiverClientSpi xceiverClient, ChunkInfo chunk, DatanodeBlockID blockID,
- List validators,
- ContainerCommandRequestProto.Builder builder,
+ private static ContainerProtos.ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
+ DatanodeBlockID blockID, List validators, ContainerCommandRequestProto.Builder builder,
DatanodeDetails d) throws IOException {
ContainerCommandRequestProto.Builder requestBuilder = builder
.setDatanodeUuid(d.getUuidString());
@@ -401,19 +361,16 @@ private static ContainerProtos.ReadChunkResponseProto readChunk(
if (traceId != null) {
requestBuilder = requestBuilder.setTraceID(traceId);
}
- ContainerCommandResponseProto reply =
- xceiverClient.sendCommand(requestBuilder.build(), validators);
+ ContainerCommandResponseProto reply = xceiverClient.sendCommand(requestBuilder.build(), validators);
final ReadChunkResponseProto response = reply.getReadChunk();
final long readLen = getLen(response);
if (readLen != chunk.getLen()) {
- throw new IOException(toErrorMessage(chunk, blockID, d)
- + ": readLen=" + readLen);
+ throw new IOException(toErrorMessage(chunk, blockID, d) + ": readLen=" + readLen);
}
return response;
}
- static String toErrorMessage(ChunkInfo chunk, DatanodeBlockID blockId,
- DatanodeDetails d) {
+ static String toErrorMessage(ChunkInfo chunk, DatanodeBlockID blockId, DatanodeDetails d) {
return String.format("Failed to read chunk %s (len=%s) %s from %s",
chunk.getChunkName(), chunk.getLen(), blockId, d);
}
@@ -440,10 +397,8 @@ static long getLen(ReadChunkResponseProto response) {
* @throws IOException if there is an I/O error while performing the call
*/
@SuppressWarnings("parameternumber")
- public static XceiverClientReply writeChunkAsync(
- XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID,
- ByteString data, String tokenString,
- int replicationIndex, BlockData blockData, boolean close)
+ public static XceiverClientReply writeChunkAsync(XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID,
+ ByteString data, String tokenString, int replicationIndex, BlockData blockData, boolean close)
throws IOException, ExecutionException, InterruptedException {
WriteChunkRequestProto.Builder writeChunkRequest =
@@ -487,11 +442,10 @@ public static XceiverClientReply writeChunkAsync(
* @param client - client that communicates with the container.
* @param blockID - ID of the block
* @param data - Data to be written into the container.
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return container protocol writeSmallFile response
*/
- public static PutSmallFileResponseProto writeSmallFile(
- XceiverClientSpi client, BlockID blockID, byte[] data,
+ public static PutSmallFileResponseProto writeSmallFile(XceiverClientSpi client, BlockID blockID, byte[] data,
Token token) throws IOException {
BlockData containerBlockData =
@@ -532,15 +486,14 @@ public static PutSmallFileResponseProto writeSmallFile(
builder.setEncodedToken(token.encodeToUrlString());
}
ContainerCommandRequestProto request = builder.build();
- ContainerCommandResponseProto response =
- client.sendCommand(request, getValidatorList());
+ ContainerCommandResponseProto response = client.sendCommand(request, getValidatorList());
return response.getPutSmallFile();
}
/**
* createRecoveringContainer call that creates a container on the datanode.
- * Currently this is used for EC reconstruction containers. When EC
- * reconstruction coordinator reconstructing the containers, the in progress
+ * Currently, this is used for EC reconstruction containers.
+ * When EC reconstruction coordinator reconstructed the containers, the in progress
* containers would be created as "RECOVERING" state containers.
* @param client - client
* @param containerID - ID of container
@@ -548,39 +501,37 @@ public static PutSmallFileResponseProto writeSmallFile(
* @param replicaIndex - index position of the container replica
*/
@InterfaceStability.Evolving
- public static void createRecoveringContainer(XceiverClientSpi client,
- long containerID, String encodedToken, int replicaIndex)
- throws IOException {
- createContainer(client, containerID, encodedToken,
- ContainerProtos.ContainerDataProto.State.RECOVERING, replicaIndex);
+ public static void createRecoveringContainer(XceiverClientSpi client, long containerID, String encodedToken,
+ int replicaIndex) throws IOException {
+ createContainer(client, containerID, encodedToken, ContainerProtos.ContainerDataProto.State.RECOVERING,
+ replicaIndex);
}
/**
- * createContainer call that creates a container on the datanode.
+ * Creates a container on the datanode.
+ *
* @param client - client
* @param containerID - ID of container
* @param encodedToken - encodedToken if security is enabled
*/
- public static void createContainer(XceiverClientSpi client, long containerID,
- String encodedToken) throws IOException {
+ public static void createContainer(XceiverClientSpi client, long containerID, String encodedToken)
+ throws IOException {
createContainer(client, containerID, encodedToken, null, 0);
}
/**
- * createContainer call that creates a container on the datanode.
+ * Creates a container on the datanode.
+ *
* @param client - client
* @param containerID - ID of container
* @param encodedToken - encodedToken if security is enabled
* @param state - state of the container
* @param replicaIndex - index position of the container replica
*/
- public static void createContainer(XceiverClientSpi client,
- long containerID, String encodedToken,
- ContainerProtos.ContainerDataProto.State state, int replicaIndex)
- throws IOException {
+ public static void createContainer(XceiverClientSpi client, long containerID, String encodedToken,
+ ContainerProtos.ContainerDataProto.State state, int replicaIndex) throws IOException {
ContainerProtos.CreateContainerRequestProto.Builder createRequest =
ContainerProtos.CreateContainerRequestProto.newBuilder();
- createRequest
- .setContainerType(ContainerProtos.ContainerType.KeyValueContainer);
+ createRequest.setContainerType(ContainerProtos.ContainerType.KeyValueContainer);
if (state != null) {
createRequest.setState(state);
}
@@ -589,8 +540,7 @@ public static void createContainer(XceiverClientSpi client,
}
String id = client.getPipeline().getFirstNode().getUuidString();
- ContainerCommandRequestProto.Builder request =
- ContainerCommandRequestProto.newBuilder();
+ ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder();
if (encodedToken != null) {
request.setEncodedToken(encodedToken);
}
@@ -609,21 +559,25 @@ public static void createContainer(XceiverClientSpi client,
/**
* Deletes a container from a pipeline.
*
- * @param force whether or not to forcibly delete the container.
+ * @param force whether to forcibly delete the container.
* @param encodedToken - encodedToken if security is enabled
*/
- public static void deleteContainer(XceiverClientSpi client, long containerID,
- boolean force, String encodedToken) throws IOException {
+ public static void deleteContainer(XceiverClientSpi client, long containerID, boolean force, String encodedToken)
+ throws IOException {
ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest =
ContainerProtos.DeleteContainerRequestProto.newBuilder();
deleteRequest.setForceDelete(force);
String id = client.getPipeline().getFirstNode().getUuidString();
- ContainerCommandRequestProto.Builder request =
- ContainerCommandRequestProto.newBuilder();
+ ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.DeleteContainer);
request.setContainerID(containerID);
request.setDeleteContainer(deleteRequest);
+ sendContainerCommand(client, encodedToken, id, request);
+ }
+
+ private static void sendContainerCommand(XceiverClientSpi client, String encodedToken, String id,
+ ContainerCommandRequestProto.Builder request) throws IOException {
request.setDatanodeUuid(id);
if (encodedToken != null) {
request.setEncodedToken(encodedToken);
@@ -636,57 +590,38 @@ public static void deleteContainer(XceiverClientSpi client, long containerID,
}
/**
- * Close a container.
+ * Closes an open container identified by the specified container ID through the given client.
*
- * @param encodedToken - encodedToken if security is enabled
+ * @param client The xceiver client instance used to communicate with the container.
+ * @param containerID The identifier of the container to be closed.
+ * @param encodedToken An encoded authentication token required to authorize the command.
+ * @throws IOException If an I/O error occurs during the operation.
*/
- public static void closeContainer(XceiverClientSpi client,
- long containerID, String encodedToken)
- throws IOException {
+ public static void closeContainer(XceiverClientSpi client, long containerID, String encodedToken) throws IOException {
String id = client.getPipeline().getFirstNode().getUuidString();
- ContainerCommandRequestProto.Builder request =
- ContainerCommandRequestProto.newBuilder();
+ ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder();
request.setCmdType(Type.CloseContainer);
request.setContainerID(containerID);
request.setCloseContainer(CloseContainerRequestProto.getDefaultInstance());
- request.setDatanodeUuid(id);
- if (encodedToken != null) {
- request.setEncodedToken(encodedToken);
- }
- String traceId = TracingUtil.exportCurrentSpan();
- if (traceId != null) {
- request.setTraceID(traceId);
- }
- client.sendCommand(request.build(), getValidatorList());
+ sendContainerCommand(client, encodedToken, id, request);
}
/**
- * readContainer call that gets meta data from an existing container.
+ * Reads the content of a specified container using the given client and encoded token.
*
- * @param client - client
- * @param encodedToken - encodedToken if security is enabled
+ * @param client An instance of XceiverClientSpi used to communicate with the server.
+ * @param containerID The ID of the container to be read.
+ * @param encodedToken A token used for authorization to read the container.
+ * @return A ReadContainerResponseProto object containing the response from the server.
+ * @throws IOException If an I/O error occurs while communicating with the server.
*/
- public static ReadContainerResponseProto readContainer(
- XceiverClientSpi client, long containerID, String encodedToken)
+ public static ReadContainerResponseProto readContainer(XceiverClientSpi client, long containerID, String encodedToken)
throws IOException {
String id = client.getPipeline().getFirstNode().getUuidString();
- ContainerCommandRequestProto.Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(Type.ReadContainer);
- request.setContainerID(containerID);
- request.setReadContainer(ReadContainerRequestProto.getDefaultInstance());
- request.setDatanodeUuid(id);
- if (encodedToken != null) {
- request.setEncodedToken(encodedToken);
- }
- String traceId = TracingUtil.exportCurrentSpan();
- if (traceId != null) {
- request.setTraceID(traceId);
- }
- ContainerCommandResponseProto response =
- client.sendCommand(request.build(), getValidatorList());
+ ContainerCommandRequestProto request = buildReadContainerRequest(containerID, encodedToken, id);
+ ContainerCommandResponseProto response = client.sendCommand(request, getValidatorList());
return response.getReadContainer();
}
@@ -695,11 +630,10 @@ public static ReadContainerResponseProto readContainer(
* Reads the data given the blockID.
*
* @param blockID - ID of the block
- * @param token a token for this block (may be null)
+ * @param token a token for this block (maybe null)
* @return GetSmallFileResponseProto
*/
- public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
- BlockID blockID,
+ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, BlockID blockID,
Token token) throws IOException {
GetBlockRequestProto.Builder getBlock = GetBlockRequestProto
.newBuilder()
@@ -717,6 +651,13 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
.setContainerID(blockID.getContainerID())
.setDatanodeUuid(id)
.setGetSmallFile(getSmallFileRequest);
+
+ ContainerCommandResponseProto response = sendContainerCommand(client, builder, token);
+ return response.getGetSmallFile();
+ }
+
+ private static ContainerCommandResponseProto sendContainerCommand(XceiverClientSpi client,
+ ContainerCommandRequestProto.Builder builder, Token extends TokenIdentifier> token) throws IOException {
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
@@ -725,15 +666,23 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
builder.setTraceID(traceId);
}
ContainerCommandRequestProto request = builder.build();
- ContainerCommandResponseProto response =
- client.sendCommand(request, getValidatorList());
- return response.getGetSmallFile();
+ return client.sendCommand(request, getValidatorList());
}
/**
- * Send an echo to DataNode.
+ * Executes an echo command on the specified container using the given xceiver client.
+ *
+ * @param client The xceiver client used to send the command.
+ * @param encodedContainerID The encoded token of the container ID.
+ * @param containerID The numeric ID of the container.
+ * @param payloadReqBytes The payload in bytes for the echo request.
+ * @param payloadRespSizeKB The expected size of the payload in the response, in kilobytes.
+ * @param sleepTimeMs The amount of time in milliseconds to sleep before responding.
+ * @param readOnly A boolean indicating if the echo operation should be read-only.
*
- * @return EchoResponseProto
+ * @return An EchoResponseProto containing the response payload.
+ *
+ * @throws IOException If an I/O error occurs while sending the command.
*/
public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID,
long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs, boolean readOnly)
@@ -762,32 +711,32 @@ public static EchoResponseProto echo(XceiverClientSpi client, String encodedCont
builder.setTraceID(traceId);
}
ContainerCommandRequestProto request = builder.build();
- ContainerCommandResponseProto response =
- client.sendCommand(request, getValidatorList());
+ ContainerCommandResponseProto response = client.sendCommand(request, getValidatorList());
return response.getEcho();
}
/**
- * Validates a response from a container protocol call. Any non-successful
- * return code is mapped to a corresponding exception and thrown.
+ * Validates a response from a container protocol call.
+ * Any non-successful return code is mapped to a corresponding exception and thrown.
*
* @param response container protocol call response
* @throws StorageContainerException if the container protocol call failed
*/
- public static void validateContainerResponse(
- ContainerCommandResponseProto response
- ) throws StorageContainerException {
- if (response.getResult() == ContainerProtos.Result.SUCCESS) {
+ public static void validateContainerResponse(ContainerCommandResponseProto response)
+ throws StorageContainerException {
+ switch (response.getResult()) {
+ case SUCCESS:
return;
- } else if (response.getResult()
- == ContainerProtos.Result.BLOCK_NOT_COMMITTED) {
+
+ case BLOCK_NOT_COMMITTED:
throw new BlockNotCommittedException(response.getMessage());
- } else if (response.getResult()
- == ContainerProtos.Result.CLOSED_CONTAINER_IO) {
+
+ case CLOSED_CONTAINER_IO:
throw new ContainerNotOpenException(response.getMessage());
+
+ default:
+ throw new StorageContainerException(response.getMessage(), response.getResult());
}
- throw new StorageContainerException(
- response.getMessage(), response.getResult());
}
private static List getValidatorList() {
@@ -797,30 +746,40 @@ private static List getValidatorList() {
private static final List VALIDATORS = createValidators();
private static List createValidators() {
- return singletonList(
- (request, response) -> validateContainerResponse(response));
+ return singletonList((request, response) -> validateContainerResponse(response));
}
+ /**
+ * Converts the given Validator into a list of Validators, combined with default validators.
+ *
+ * @param validator The Validator to be added to the list.
+ * @return An unmodifiable list of Validators, including the given Validator and default validators.
+ */
public static List toValidatorList(Validator validator) {
final List defaults = getValidatorList();
- final List validators
- = new ArrayList<>(defaults.size() + 1);
+ final List validators = new ArrayList<>(defaults.size() + 1);
validators.addAll(defaults);
validators.add(validator);
return Collections.unmodifiableList(validators);
}
- public static HashMap
- getBlockFromAllNodes(
- XceiverClientSpi xceiverClient,
- DatanodeBlockID datanodeBlockID,
- Token token)
+ /**
+ * Retrieves block information from all nodes in the pipeline.
+ *
+ * @param xceiverClient the client interface for communication with the data nodes.
+ * @param datanodeBlockID the identifier of the data block.
+ * @param token the security token used for authentication and authorization.
+ * @return a map containing data node details mapped to their respective block response protocol objects.
+ * @throws IOException if there is an I/O error during communication with the data nodes.
+ * @throws InterruptedException if the operation is interrupted.
+ */
+ public static Map getBlockFromAllNodes(XceiverClientSpi xceiverClient,
+ DatanodeBlockID datanodeBlockID, Token token)
throws IOException, InterruptedException {
GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto
.newBuilder()
.setBlockID(datanodeBlockID);
- HashMap datanodeToResponseMap
- = new HashMap<>();
+ Map datanodeToResponseMap = new HashMap<>();
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto
.newBuilder()
@@ -836,23 +795,41 @@ public static List toValidatorList(Validator validator) {
builder.setTraceID(traceId);
}
ContainerCommandRequestProto request = builder.build();
- Map responses =
- xceiverClient.sendCommandOnAllNodes(request);
- for (Map.Entry entry:
- responses.entrySet()) {
+ Map responses = xceiverClient.sendCommandOnAllNodes(request);
+ for (Map.Entry entry: responses.entrySet()) {
datanodeToResponseMap.put(entry.getKey(), entry.getValue().getGetBlock());
}
return datanodeToResponseMap;
}
- public static HashMap
- readContainerFromAllNodes(XceiverClientSpi client, long containerID,
- String encodedToken) throws IOException, InterruptedException {
+ /**
+ * Reads a container from all nodes in the pipeline associated with the given client.
+ *
+ * @param client The XceiverClientSpi used to send the read container request to all nodes.
+ * @param containerID The ID of the container to be read.
+ * @param encodedToken The security token used for authorization.
+ * @return A Map mapping each DatanodeDetails to its corresponding ReadContainerResponseProto,
+ * representing the response from each node for the read container request.
+ * @throws IOException If an I/O error occurs while sending the request or processing the response.
+ * @throws InterruptedException If the thread is interrupted while waiting for the response.
+ */
+ public static Map readContainerFromAllNodes(XceiverClientSpi client,
+ long containerID, String encodedToken) throws IOException, InterruptedException {
+
String id = client.getPipeline().getFirstNode().getUuidString();
- HashMap datanodeToResponseMap
- = new HashMap<>();
- ContainerCommandRequestProto.Builder request =
- ContainerCommandRequestProto.newBuilder();
+ Map datanodeToResponseMap = new HashMap<>();
+ ContainerCommandRequestProto request = buildReadContainerRequest(containerID, encodedToken, id);
+ Map responses = client.sendCommandOnAllNodes(request);
+ for (Map.Entry entry : responses.entrySet()) {
+ datanodeToResponseMap.put(entry.getKey(), entry.getValue().getReadContainer());
+ }
+ return datanodeToResponseMap;
+ }
+
+ private static ContainerCommandRequestProto buildReadContainerRequest(long containerID, String encodedToken,
+ String id) {
+
+ ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder();
request.setCmdType(Type.ReadContainer);
request.setContainerID(containerID);
request.setReadContainer(ReadContainerRequestProto.getDefaultInstance());
@@ -864,14 +841,22 @@ public static List toValidatorList(Validator validator) {
if (traceId != null) {
request.setTraceID(traceId);
}
- Map responses =
- client.sendCommandOnAllNodes(request.build());
- for (Map.Entry entry :
- responses.entrySet()) {
- datanodeToResponseMap.put(entry.getKey(),
- entry.getValue().getReadContainer());
- }
- return datanodeToResponseMap;
+
+ return request.build();
}
+ private static DatanodeBlockID buildDatanodeId(ContainerCommandRequestProto.Builder builder, BlockID blockID,
+ DatanodeDetails datanode, Map replicaIndexes) {
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
+ final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder();
+ int replicaIndex = replicaIndexes.getOrDefault(datanode, 0);
+ if (replicaIndex > 0) {
+ datanodeBlockID.setReplicaIndex(replicaIndex);
+ }
+
+ return datanodeBlockID.build();
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index b34a5d8387b..c6023ccd071 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -40,7 +40,6 @@ public final class OzoneConsts {
public static final String SCM_CERT_SERIAL_ID = "scmCertSerialId";
public static final String PRIMARY_SCM_NODE_ID = "primaryScmNodeId";
- public static final String OZONE_SIMPLE_ROOT_USER = "root";
public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";
public static final String STORAGE_ID = "storageID";
@@ -76,12 +75,6 @@ public final class OzoneConsts {
"EEE, dd MMM yyyy HH:mm:ss zzz";
public static final String OZONE_TIME_ZONE = "GMT";
- public static final String OZONE_COMPONENT = "component";
- public static final String OZONE_FUNCTION = "function";
- public static final String OZONE_RESOURCE = "resource";
- public static final String OZONE_USER = "user";
- public static final String OZONE_REQUEST = "request";
-
// OM Http server endpoints
public static final String OZONE_OM_SERVICE_LIST_HTTP_ENDPOINT =
"/serviceList";
@@ -101,14 +94,9 @@ public final class OzoneConsts {
public static final String CONTAINER_EXTENSION = ".container";
- public static final String CONTAINER_META = ".meta";
-
- // Refer to {@link ContainerReader} for container storage layout on disk.
- public static final String CONTAINER_PREFIX = "containers";
public static final String CONTAINER_META_PATH = "metadata";
public static final String CONTAINER_TEMPORARY_CHUNK_PREFIX = "tmp";
public static final String CONTAINER_CHUNK_NAME_DELIMITER = ".";
- public static final String CONTAINER_ROOT_PREFIX = "repository";
public static final String FILE_HASH = "SHA-256";
public static final String MD5_HASH = "MD5";
@@ -128,7 +116,6 @@ public final class OzoneConsts {
* level DB names used by SCM and data nodes.
*/
public static final String CONTAINER_DB_SUFFIX = "container.db";
- public static final String PIPELINE_DB_SUFFIX = "pipeline.db";
public static final String DN_CONTAINER_DB = "-dn-" + CONTAINER_DB_SUFFIX;
public static final String OM_DB_NAME = "om.db";
public static final String SCM_DB_NAME = "scm.db";
@@ -187,10 +174,8 @@ public final class OzoneConsts {
public static final String OM_USER_PREFIX = "$";
public static final String OM_S3_PREFIX = "S3:";
public static final String OM_S3_CALLER_CONTEXT_PREFIX = "S3Auth:S3G|";
- public static final String OM_S3_VOLUME_PREFIX = "s3";
public static final String OM_S3_SECRET = "S3Secret:";
public static final String OM_PREFIX = "Prefix:";
- public static final String OM_TENANT = "Tenant:";
/**
* Max chunk size limit.
@@ -198,11 +183,6 @@ public final class OzoneConsts {
public static final int OZONE_SCM_CHUNK_MAX_SIZE = 32 * 1024 * 1024;
- /**
- * Max OM Quota size of Long.MAX_VALUE.
- */
- public static final long MAX_QUOTA_IN_BYTES = Long.MAX_VALUE;
-
/**
* Quota RESET default is -1, which means quota is not set.
*/
@@ -214,36 +194,20 @@ public final class OzoneConsts {
*/
public enum Units { TB, GB, MB, KB, B }
- /**
- * Max number of keys returned per list buckets operation.
- */
- public static final int MAX_LISTBUCKETS_SIZE = 1024;
-
- /**
- * Max number of keys returned per list keys operation.
- */
- public static final int MAX_LISTKEYS_SIZE = 1024;
-
- /**
- * Max number of volumes returned per list volumes operation.
- */
- public static final int MAX_LISTVOLUMES_SIZE = 1024;
-
- public static final int INVALID_PORT = -1;
-
/**
* Object ID to identify reclaimable uncommitted blocks.
*/
public static final long OBJECT_ID_RECLAIM_BLOCKS = 0L;
-
/**
* Default SCM Datanode ID file name.
*/
public static final String OZONE_SCM_DATANODE_ID_FILE_DEFAULT = "datanode.id";
- // The ServiceListJSONServlet context attribute where OzoneManager
- // instance gets stored.
+ /**
+ * The ServiceListJSONServlet context attribute where OzoneManager
+ * instance gets stored.
+ */
public static final String OM_CONTEXT_ATTRIBUTE = "ozone.om";
public static final String SCM_CONTEXT_ATTRIBUTE = "ozone.scm";
@@ -308,12 +272,8 @@ private OzoneConsts() {
public static final String KEY_PREFIX = "keyPrefix";
public static final String ACL = "acl";
public static final String ACLS = "acls";
- public static final String USER_ACL = "userAcl";
- public static final String ADD_ACLS = "addAcls";
- public static final String REMOVE_ACLS = "removeAcls";
public static final String MAX_NUM_OF_BUCKETS = "maxNumOfBuckets";
public static final String HAS_SNAPSHOT = "hasSnapshot";
- public static final String TO_KEY_NAME = "toKeyName";
public static final String STORAGE_TYPE = "storageType";
public static final String RESOURCE_TYPE = "resourceType";
public static final String IS_VERSION_ENABLED = "isVersionEnabled";
@@ -323,7 +283,6 @@ private OzoneConsts() {
public static final String REPLICATION_TYPE = "replicationType";
public static final String REPLICATION_FACTOR = "replicationFactor";
public static final String REPLICATION_CONFIG = "replicationConfig";
- public static final String KEY_LOCATION_INFO = "keyLocationInfo";
public static final String MULTIPART_LIST = "multipartList";
public static final String UPLOAD_ID = "uploadID";
public static final String PART_NUMBER_MARKER = "partNumberMarker";
@@ -378,10 +337,6 @@ private OzoneConsts() {
public static final String JAVA_TMP_DIR = "java.io.tmpdir";
public static final String LOCALHOST = "localhost";
-
- public static final int S3_BUCKET_MIN_LENGTH = 3;
- public static final int S3_BUCKET_MAX_LENGTH = 64;
-
public static final int S3_SECRET_KEY_MIN_LENGTH = 8;
public static final int S3_REQUEST_HEADER_METADATA_SIZE_LIMIT_KB = 2;
@@ -398,7 +353,6 @@ private OzoneConsts() {
public static final String GDPR_ALGORITHM_NAME = "AES";
public static final int GDPR_DEFAULT_RANDOM_SECRET_LENGTH = 16;
public static final Charset GDPR_CHARSET = StandardCharsets.UTF_8;
- public static final String GDPR_LENGTH = "length";
public static final String GDPR_SECRET = "secret";
public static final String GDPR_ALGORITHM = "algorithm";
@@ -426,13 +380,6 @@ private OzoneConsts() {
public static final String CONTAINER_DB_TYPE_ROCKSDB = "RocksDB";
- // SCM HA
- public static final String SCM_SERVICE_ID_DEFAULT = "scmServiceIdDefault";
-
- // SCM Ratis snapshot file to store the last applied index
- public static final String SCM_RATIS_SNAPSHOT_INDEX = "scmRatisSnapshotIndex";
-
- public static final String SCM_RATIS_SNAPSHOT_TERM = "scmRatisSnapshotTerm";
// An on-disk transient marker file used when replacing DB with checkpoint
public static final String DB_TRANSIENT_MARKER = "dbInconsistentMarker";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
index eec2ceeb5e8..982b559c7a5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
@@ -44,6 +44,8 @@ public enum OzoneManagerVersion implements ComponentVersion {
ATOMIC_REWRITE_KEY(6, "OzoneManager version that supports rewriting key as atomic operation"),
HBASE_SUPPORT(7, "OzoneManager version that supports HBase integration"),
+ LIGHTWEIGHT_LIST_STATUS(8, "OzoneManager version that supports lightweight"
+ + " listStatus API."),
FUTURE_VERSION(-1, "Used internally in the client when the server side is "
+ " newer and an unknown server version has arrived to the client.");
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
index 4bd170df8e8..ea5c5453f3f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.hdds.utils.db.Codec;
import org.apache.hadoop.hdds.utils.db.DelegatedCodec;
import org.apache.hadoop.hdds.utils.db.Proto3Codec;
+import org.apache.hadoop.ozone.OzoneConsts;
import java.io.IOException;
import java.util.Collections;
@@ -280,4 +281,14 @@ public void appendTo(StringBuilder sb) {
sb.append(", size=").append(size);
sb.append("]");
}
+
+ public long getBlockGroupLength() {
+ String lenStr = getMetadata()
+ .get(OzoneConsts.BLOCK_GROUP_LEN_KEY_IN_PUT_BLOCK);
+ // If we don't have the length, then it indicates a problem with the stripe.
+ // All replica should carry the length, so if it is not there, we return 0,
+ // which will cause us to set the length of the block to zero and not
+ // attempt to reconstruct it.
+ return (lenStr == null) ? 0 : Long.parseLong(lenStr);
+ }
}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index 2b7592e1c35..4c1d9d9dbc5 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -60,11 +60,9 @@
* Helpers for container tests.
*/
public final class ContainerTestHelper {
- private static final Logger LOG = LoggerFactory.getLogger(
- ContainerTestHelper.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ContainerTestHelper.class);
- public static final long CONTAINER_MAX_SIZE =
- (long) StorageUnit.GB.toBytes(1);
+ public static final long CONTAINER_MAX_SIZE = (long) StorageUnit.GB.toBytes(1);
public static final String DATANODE_UUID = UUID.randomUUID().toString();
private static final long DUMMY_CONTAINER_ID = 9999;
@@ -83,10 +81,8 @@ private ContainerTestHelper() {
* @param seqNo - Chunk number.
* @return ChunkInfo
*/
- public static ChunkInfo getChunk(long keyID, int seqNo, long offset,
- long len) {
- return new ChunkInfo(String.format("%d.data.%d", keyID,
- seqNo), offset, len);
+ public static ChunkInfo getChunk(long keyID, int seqNo, long offset, long len) {
+ return new ChunkInfo(String.format("%d.data.%d", keyID, seqNo), offset, len);
}
/**
@@ -107,10 +103,8 @@ public static ChunkBuffer getData(int len) {
* @param info - chunk info.
* @param data - data array
*/
- public static void setDataChecksum(ChunkInfo info, ChunkBuffer data)
- throws OzoneChecksumException {
- Checksum checksum = new Checksum(ChecksumType.CRC32,
- 1024 * 1024);
+ public static void setDataChecksum(ChunkInfo info, ChunkBuffer data) throws OzoneChecksumException {
+ Checksum checksum = new Checksum(ChecksumType.CRC32, 1024 * 1024);
info.setChecksumData(checksum.computeChecksum(data));
data.rewind();
}
@@ -123,17 +117,13 @@ public static void setDataChecksum(ChunkInfo info, ChunkBuffer data)
* @param datalen - Length of data.
* @return ContainerCommandRequestProto
*/
- public static ContainerCommandRequestProto getWriteChunkRequest(
- Pipeline pipeline, BlockID blockID, int datalen)
+ public static ContainerCommandRequestProto getWriteChunkRequest(Pipeline pipeline, BlockID blockID, int datalen)
throws IOException {
- LOG.trace("writeChunk {} (blockID={}) to pipeline={}",
- datalen, blockID, pipeline);
- return newWriteChunkRequestBuilder(pipeline, blockID, datalen)
- .build();
+ LOG.trace("writeChunk {} (blockID={}) to pipeline={}", datalen, blockID, pipeline);
+ return newWriteChunkRequestBuilder(pipeline, blockID, datalen).build();
}
- public static ContainerCommandRequestProto getListBlockRequest(
- ContainerCommandRequestProto writeChunkRequest) {
+ public static ContainerCommandRequestProto getListBlockRequest(ContainerCommandRequestProto writeChunkRequest) {
return ContainerCommandRequestProto.newBuilder()
.setContainerID(writeChunkRequest.getContainerID())
.setCmdType(ContainerProtos.Type.ListBlock)
@@ -143,8 +133,7 @@ public static ContainerCommandRequestProto getListBlockRequest(
.build();
}
- public static ContainerCommandRequestProto getPutBlockRequest(
- ContainerCommandRequestProto writeChunkRequest) {
+ public static ContainerCommandRequestProto getPutBlockRequest(ContainerCommandRequestProto writeChunkRequest) {
ContainerProtos.BlockData.Builder block =
ContainerProtos.BlockData.newBuilder()
.setSize(writeChunkRequest.getWriteChunk().getChunkData().getLen())
@@ -160,17 +149,15 @@ public static ContainerCommandRequestProto getPutBlockRequest(
.build();
}
- public static Builder newWriteChunkRequestBuilder(Pipeline pipeline,
- BlockID blockID, int datalen) throws IOException {
+ public static Builder newWriteChunkRequestBuilder(Pipeline pipeline, BlockID blockID, int datalen)
+ throws IOException {
ChunkBuffer data = getData(datalen);
return newWriteChunkRequestBuilder(pipeline, blockID, data, 0);
}
- public static Builder newWriteChunkRequestBuilder(
- Pipeline pipeline, BlockID blockID, ChunkBuffer data, int seq)
+ public static Builder newWriteChunkRequestBuilder(Pipeline pipeline, BlockID blockID, ChunkBuffer data, int seq)
throws IOException {
- LOG.trace("writeChunk {} (blockID={}) to pipeline={}",
- data.limit(), blockID, pipeline);
+ LOG.trace("writeChunk {} (blockID={}) to pipeline={}", data.limit(), blockID, pipeline);
ContainerProtos.WriteChunkRequestProto.Builder writeRequest =
ContainerProtos.WriteChunkRequestProto
.newBuilder();
@@ -183,14 +170,11 @@ public static Builder newWriteChunkRequestBuilder(
writeRequest.setChunkData(info.getProtoBufMessage());
writeRequest.setData(data.toByteString());
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.WriteChunk);
- request.setContainerID(blockID.getContainerID());
- request.setWriteChunk(writeRequest);
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
-
- return request;
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.WriteChunk)
+ .setContainerID(blockID.getContainerID())
+ .setWriteChunk(writeRequest)
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString());
}
/**
@@ -201,8 +185,7 @@ public static Builder newWriteChunkRequestBuilder(
* @param dataLen - Number of bytes in the data
* @return ContainerCommandRequestProto
*/
- public static ContainerCommandRequestProto getWriteSmallFileRequest(
- Pipeline pipeline, BlockID blockID, int dataLen)
+ public static ContainerCommandRequestProto getWriteSmallFileRequest(Pipeline pipeline, BlockID blockID, int dataLen)
throws Exception {
ContainerProtos.PutSmallFileRequestProto.Builder smallFileRequest =
ContainerProtos.PutSmallFileRequestProto.newBuilder();
@@ -210,9 +193,7 @@ public static ContainerCommandRequestProto getWriteSmallFileRequest(
ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, dataLen);
setDataChecksum(info, data);
-
- ContainerProtos.PutBlockRequestProto.Builder putRequest =
- ContainerProtos.PutBlockRequestProto.newBuilder();
+ ContainerProtos.PutBlockRequestProto.Builder putRequest = ContainerProtos.PutBlockRequestProto.newBuilder();
BlockData blockData = new BlockData(blockID);
List newList = new LinkedList<>();
@@ -224,13 +205,12 @@ public static ContainerCommandRequestProto getWriteSmallFileRequest(
smallFileRequest.setData(data.toByteString());
smallFileRequest.setBlock(putRequest);
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.PutSmallFile);
- request.setContainerID(blockID.getContainerID());
- request.setPutSmallFile(smallFileRequest);
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
- return request.build();
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.PutSmallFile)
+ .setContainerID(blockID.getContainerID())
+ .setPutSmallFile(smallFileRequest)
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString())
+ .build();
}
@@ -242,13 +222,12 @@ public static ContainerCommandRequestProto getReadSmallFileRequest(
ContainerCommandRequestProto getKey = getBlockRequest(pipeline, putKey);
smallFileRequest.setBlock(getKey.getGetBlock());
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.GetSmallFile);
- request.setContainerID(getKey.getGetBlock().getBlockID().getContainerID());
- request.setGetSmallFile(smallFileRequest);
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
- return request.build();
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.GetSmallFile)
+ .setContainerID(getKey.getGetBlock().getBlockID().getContainerID())
+ .setGetSmallFile(smallFileRequest)
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString())
+ .build();
}
/**
@@ -267,8 +246,7 @@ public static ContainerCommandRequestProto getReadChunkRequest(
public static Builder newReadChunkRequestBuilder(Pipeline pipeline,
ContainerProtos.WriteChunkRequestProtoOrBuilder writeChunk)
throws IOException {
- LOG.trace("readChunk blockID={} from pipeline={}",
- writeChunk.getBlockID(), pipeline);
+ LOG.trace("readChunk blockID={} from pipeline={}", writeChunk.getBlockID(), pipeline);
ContainerProtos.ReadChunkRequestProto.Builder readRequest =
ContainerProtos.ReadChunkRequestProto.newBuilder();
@@ -276,13 +254,11 @@ public static Builder newReadChunkRequestBuilder(Pipeline pipeline,
readRequest.setChunkData(writeChunk.getChunkData());
readRequest.setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1);
- Builder newRequest =
- ContainerCommandRequestProto.newBuilder();
- newRequest.setCmdType(ContainerProtos.Type.ReadChunk);
- newRequest.setContainerID(readRequest.getBlockID().getContainerID());
- newRequest.setReadChunk(readRequest);
- newRequest.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
- return newRequest;
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.ReadChunk)
+ .setContainerID(readRequest.getBlockID().getContainerID())
+ .setReadChunk(readRequest)
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString());
}
/**
@@ -297,17 +273,12 @@ public static ContainerCommandRequestProto getCreateContainerRequest(
return getContainerCommandRequestBuilder(containerID, pipeline).build();
}
- private static Builder getContainerCommandRequestBuilder(long containerID,
- Pipeline pipeline) throws IOException {
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.CreateContainer);
- request.setContainerID(containerID);
- request.setCreateContainer(
- ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
-
- return request;
+ private static Builder getContainerCommandRequestBuilder(long containerID, Pipeline pipeline) throws IOException {
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.CreateContainer)
+ .setContainerID(containerID)
+ .setCreateContainer(ContainerProtos.CreateContainerRequestProto.getDefaultInstance())
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString());
}
/**
@@ -343,16 +314,14 @@ public static ContainerCommandRequestProto getUpdateContainerRequest(
kvBuilder.setValue(metaData.get(key));
updateRequestBuilder.addMetadata(kvBuilder.build());
}
- Pipeline pipeline =
- MockPipeline.createSingleNodePipeline();
-
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.UpdateContainer);
- request.setContainerID(containerID);
- request.setUpdateContainer(updateRequestBuilder.build());
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
- return request.build();
+ Pipeline pipeline = MockPipeline.createSingleNodePipeline();
+
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.UpdateContainer)
+ .setContainerID(containerID)
+ .setUpdateContainer(updateRequestBuilder.build())
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString())
+ .build();
}
/**
@@ -361,17 +330,14 @@ public static ContainerCommandRequestProto getUpdateContainerRequest(
*
* @return ContainerCommandRequestProto.
*/
- public static ContainerCommandResponseProto
- getCreateContainerResponse(ContainerCommandRequestProto request) {
+ public static ContainerCommandResponseProto getCreateContainerResponse(ContainerCommandRequestProto request) {
- ContainerCommandResponseProto.Builder response =
- ContainerCommandResponseProto.newBuilder();
- response.setCmdType(ContainerProtos.Type.CreateContainer);
- response.setTraceID(request.getTraceID());
- response.setCreateContainer(
- ContainerProtos.CreateContainerResponseProto.getDefaultInstance());
- response.setResult(ContainerProtos.Result.SUCCESS);
- return response.build();
+ return ContainerCommandResponseProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.CreateContainer)
+ .setTraceID(request.getTraceID())
+ .setCreateContainer(ContainerProtos.CreateContainerResponseProto.getDefaultInstance())
+ .setResult(ContainerProtos.Result.SUCCESS)
+ .build();
}
/**
@@ -401,14 +367,11 @@ public static Builder newPutBlockRequestBuilder(Pipeline pipeline,
public static Builder newPutBlockRequestBuilder(Pipeline pipeline,
ContainerProtos.WriteChunkRequestProtoOrBuilder writeRequest, boolean incremental)
throws IOException {
- LOG.trace("putBlock: {} to pipeline={}",
- writeRequest.getBlockID(), pipeline);
+ LOG.trace("putBlock: {} to pipeline={}", writeRequest.getBlockID(), pipeline);
- ContainerProtos.PutBlockRequestProto.Builder putRequest =
- ContainerProtos.PutBlockRequestProto.newBuilder();
+ ContainerProtos.PutBlockRequestProto.Builder putRequest = ContainerProtos.PutBlockRequestProto.newBuilder();
- BlockData blockData = new BlockData(
- BlockID.getFromProtobuf(writeRequest.getBlockID()));
+ BlockData blockData = new BlockData(BlockID.getFromProtobuf(writeRequest.getBlockID()));
List newList = new LinkedList<>();
newList.add(writeRequest.getChunkData());
blockData.setChunks(newList);
@@ -418,13 +381,11 @@ public static Builder newPutBlockRequestBuilder(Pipeline pipeline,
}
putRequest.setBlockData(blockData.getProtoBufMessage());
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.PutBlock);
- request.setContainerID(blockData.getContainerID());
- request.setPutBlock(putRequest);
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
- return request;
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.PutBlock)
+ .setContainerID(blockData.getContainerID())
+ .setPutBlock(putRequest)
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString());
}
/**
@@ -445,17 +406,14 @@ public static Builder newGetBlockRequestBuilder(
throws IOException {
DatanodeBlockID blockID = putBlock.getBlockData().getBlockID();
- ContainerProtos.GetBlockRequestProto.Builder getRequest =
- ContainerProtos.GetBlockRequestProto.newBuilder();
+ ContainerProtos.GetBlockRequestProto.Builder getRequest = ContainerProtos.GetBlockRequestProto.newBuilder();
getRequest.setBlockID(blockID);
- Builder request =
- ContainerCommandRequestProto.newBuilder();
- request.setCmdType(ContainerProtos.Type.GetBlock);
- request.setContainerID(blockID.getContainerID());
- request.setGetBlock(getRequest);
- request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
- return request;
+ return ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.GetBlock)
+ .setContainerID(blockID.getContainerID())
+ .setGetBlock(getRequest)
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString());
}
/**
@@ -465,8 +423,7 @@ public static Builder newGetBlockRequestBuilder(
*/
public static void verifyGetBlock(ContainerCommandResponseProto response, int expectedChunksCount) {
assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
- assertEquals(expectedChunksCount,
- response.getGetBlock().getBlockData().getChunksCount());
+ assertEquals(expectedChunksCount, response.getGetBlock().getBlockData().getChunksCount());
}
public static Builder newGetCommittedBlockLengthBuilder(Pipeline pipeline,
@@ -549,9 +506,9 @@ public static ContainerProtos.ContainerCommandRequestProto getFinalizeBlockReque
.setContainerID(container.getContainerID()).setLocalID(localID)
.setBlockCommitSequenceId(0).build();
- builder.setFinalizeBlock(ContainerProtos.FinalizeBlockRequestProto
- .newBuilder().setBlockID(blockId).build());
- return builder.build();
+ return builder.setFinalizeBlock(ContainerProtos.FinalizeBlockRequestProto
+ .newBuilder().setBlockID(blockId).build())
+ .build();
}
public static BlockID getTestBlockID(long containerID) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java
index f7a38e3dec8..bd1e031ad71 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java
@@ -41,7 +41,8 @@ public enum DNAction implements AuditAction {
GET_COMMITTED_BLOCK_LENGTH,
STREAM_INIT,
FINALIZE_BLOCK,
- ECHO;
+ ECHO,
+ VERIFY_BLOCK;
@Override
public String getAction() {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index c5855b38b74..6a79bf8417a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -59,7 +59,6 @@
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.ozoneimpl.OnDemandContainerDataScanner;
import org.apache.hadoop.ozone.container.common.volume.VolumeUsage;
import org.apache.hadoop.util.Time;
@@ -88,8 +87,7 @@
public class HddsDispatcher implements ContainerDispatcher, Auditor {
static final Logger LOG = LoggerFactory.getLogger(HddsDispatcher.class);
- private static final AuditLogger AUDIT =
- new AuditLogger(AuditLoggerType.DNLOGGER);
+ private static final AuditLogger AUDIT = new AuditLogger(AuditLoggerType.DNLOGGER);
private static final String AUDIT_PARAM_CONTAINER_ID = "containerID";
private static final String AUDIT_PARAM_CONTAINER_TYPE = "containerType";
private static final String AUDIT_PARAM_FORCE_UPDATE = "forceUpdate";
@@ -101,48 +99,40 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
private static final String AUDIT_PARAM_BLOCK_DATA_STAGE = "stage";
private static final String AUDIT_PARAM_COUNT = "count";
private static final String AUDIT_PARAM_START_LOCAL_ID = "startLocalID";
- private static final String AUDIT_PARAM_PREV_CHUNKNAME = "prevChunkName";
+ private static final String AUDIT_PARAM_PREV_CHUNK_NAME = "prevChunkName";
private final Map handlers;
- private final ConfigurationSource conf;
private final ContainerSet containerSet;
- private final VolumeSet volumeSet;
private final StateContext context;
private final float containerCloseThreshold;
private final ProtocolMessageMetrics protocolMetrics;
- private OzoneProtocolMessageDispatcher dispatcher;
private String clusterId;
private ContainerMetrics metrics;
private final TokenVerifier tokenVerifier;
- private long slowOpThresholdNs;
- private VolumeUsage.MinFreeSpaceCalculator freeSpaceCalculator;
+ private final long slowOpThresholdNs;
+ private final VolumeUsage.MinFreeSpaceCalculator freeSpaceCalculator;
/**
* Constructs an OzoneContainer that receives calls from
* XceiverServerHandler.
*/
- public HddsDispatcher(ConfigurationSource config, ContainerSet contSet,
- VolumeSet volumes, Map handlers,
- StateContext context, ContainerMetrics metrics,
- TokenVerifier tokenVerifier) {
- this.conf = config;
+ public HddsDispatcher(ConfigurationSource config, ContainerSet contSet, Map handlers,
+ StateContext context, ContainerMetrics metrics, TokenVerifier tokenVerifier) {
this.containerSet = contSet;
- this.volumeSet = volumes;
this.context = context;
this.handlers = handlers;
this.metrics = metrics;
- this.containerCloseThreshold = conf.getFloat(
+ this.containerCloseThreshold = config.getFloat(
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD,
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
- this.tokenVerifier = tokenVerifier != null ? tokenVerifier
- : new NoopTokenVerifier();
- this.slowOpThresholdNs = getSlowOpThresholdMs(conf) * 1000000;
+ this.tokenVerifier = tokenVerifier != null ? tokenVerifier : new NoopTokenVerifier();
+ this.slowOpThresholdNs = getSlowOpThresholdMs(config) * 1000000;
- protocolMetrics =
- new ProtocolMessageMetrics<>(
- "HddsDispatcher",
- "HDDS dispatcher metrics",
- Type.values());
+ protocolMetrics = new ProtocolMessageMetrics<>(
+ "HddsDispatcher",
+ "HDDS dispatcher metrics",
+ Type.values());
this.dispatcher =
new OzoneProtocolMessageDispatcher<>("DatanodeClient",
@@ -150,7 +140,7 @@ public HddsDispatcher(ConfigurationSource config, ContainerSet contSet,
LOG,
HddsUtils::processForDebug,
HddsUtils::processForDebug);
- this.freeSpaceCalculator = new VolumeUsage.MinFreeSpaceCalculator(conf);
+ this.freeSpaceCalculator = new VolumeUsage.MinFreeSpaceCalculator(config);
}
@Override
@@ -164,10 +154,10 @@ public void shutdown() {
}
/**
- * Returns true for exceptions which can be ignored for marking the container
- * unhealthy.
+ * Returns true for exceptions which can be ignored for marking the container unhealthy.
+ *
* @param result ContainerCommandResponse error code.
- * @return true if exception can be ignored, false otherwise.
+ * @return {@code true} if exception can be ignored, {@code false} otherwise.
*/
private boolean canIgnoreException(Result result) {
switch (result) {
@@ -183,15 +173,12 @@ private boolean canIgnoreException(Result result) {
}
@Override
- public void buildMissingContainerSetAndValidate(
- Map container2BCSIDMap) {
- containerSet
- .buildMissingContainerSetAndValidate(container2BCSIDMap);
+ public void buildMissingContainerSetAndValidate(Map container2BCSIDMap) {
+ containerSet.buildMissingContainerSetAndValidate(container2BCSIDMap);
}
@Override
- public ContainerCommandResponseProto dispatch(
- ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
+ public ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
try {
return dispatcher.processRequest(msg,
req -> dispatchRequest(msg, dispatcherContext),
@@ -203,39 +190,35 @@ public ContainerCommandResponseProto dispatch(
}
@SuppressWarnings("methodlength")
- private ContainerCommandResponseProto dispatchRequest(
- ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
+ private ContainerCommandResponseProto dispatchRequest(ContainerCommandRequestProto msg,
+ DispatcherContext dispatcherContext) {
+
Preconditions.checkNotNull(msg);
- if (LOG.isTraceEnabled()) {
- LOG.trace("Command {}, trace ID: {} ", msg.getCmdType(),
- msg.getTraceID());
- }
+ LOG.trace("Command {}, trace ID: {} ", msg.getCmdType(), msg.getTraceID());
AuditAction action = getAuditAction(msg.getCmdType());
EventType eventType = getEventType(msg);
PerformanceStringBuilder perf = new PerformanceStringBuilder();
ContainerType containerType;
- ContainerCommandResponseProto responseProto = null;
+ ContainerCommandResponseProto responseProto;
long startTime = Time.monotonicNowNanos();
Type cmdType = msg.getCmdType();
long containerID = msg.getContainerID();
Container container = getContainer(containerID);
boolean isWriteStage =
(cmdType == Type.WriteChunk && dispatcherContext != null
- && dispatcherContext.getStage()
- == DispatcherContext.WriteChunkStage.WRITE_DATA)
+ && dispatcherContext.getStage() == DispatcherContext.WriteChunkStage.WRITE_DATA)
|| (cmdType == Type.StreamInit);
boolean isWriteCommitStage =
(cmdType == Type.WriteChunk && dispatcherContext != null
- && dispatcherContext.getStage()
- == DispatcherContext.WriteChunkStage.COMMIT_DATA);
+ && dispatcherContext.getStage() == DispatcherContext.WriteChunkStage.COMMIT_DATA);
if (dispatcherContext == null) {
- // increase all op not through ratis
+ // Increase all ops not through ratis
metrics.incContainerOpsMetrics(cmdType);
} else if (isWriteStage) {
- // increase WriteChunk in only WRITE_STAGE
+ // Increase WriteChunk in only WRITE_STAGE
metrics.incContainerOpsMetrics(cmdType);
} else if (cmdType != Type.WriteChunk) {
metrics.incContainerOpsMetrics(cmdType);
@@ -252,57 +235,50 @@ private ContainerCommandResponseProto dispatchRequest(
s, ioe, ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED);
return ContainerUtils.logAndReturnError(LOG, sce, msg);
}
- // if the command gets executed other than Ratis, the default write stage
- // is WriteChunkStage.COMBINED
+ // if the command gets executed other than Ratis, the default write stage is WriteChunkStage.COMBINED
boolean isCombinedStage =
cmdType == Type.WriteChunk && (dispatcherContext == null
- || dispatcherContext.getStage()
- == DispatcherContext.WriteChunkStage.COMBINED);
+ || dispatcherContext.getStage() == DispatcherContext.WriteChunkStage.COMBINED);
Map container2BCSIDMap = null;
if (dispatcherContext != null) {
container2BCSIDMap = dispatcherContext.getContainer2BCSIDMap();
}
if (isWriteCommitStage) {
- // check if the container Id exist in the loaded snapshot file. if
- // it does not , it infers that , this is a restart of dn where
- // the we are reapplying the transaction which was not captured in the
- // snapshot.
- // just add it to the list, and remove it from missing container set
- // as it might have been added in the list during "init".
+ // Check if the container id exist in the loaded snapshot file.
+ // If it does not, it infers that
+ // this is a restart of datanode
+ // where they are reapplying the transaction which was not captured in the snapshot.
+ // Add it to the list,
+ // and remove it from the missing container set as it might have been added in the list during "init".
Preconditions.checkNotNull(container2BCSIDMap);
if (container != null && container2BCSIDMap.get(containerID) == null) {
- container2BCSIDMap.put(
- containerID, container.getBlockCommitSequenceId());
+ container2BCSIDMap.put(containerID, container.getBlockCommitSequenceId());
getMissingContainerSet().remove(containerID);
}
}
if (getMissingContainerSet().contains(containerID)) {
StorageContainerException sce = new StorageContainerException(
- "ContainerID " + containerID
- + " has been lost and cannot be recreated on this DataNode",
+ "ContainerID " + containerID + " has been lost and cannot be recreated on this DataNode",
ContainerProtos.Result.CONTAINER_MISSING);
audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, sce);
return ContainerUtils.logAndReturnError(LOG, sce, msg);
}
if (cmdType != Type.CreateContainer) {
- /**
- * Create Container should happen only as part of Write_Data phase of
- * writeChunk.
- * In EC, we are doing empty putBlock. In the partial stripe writes, if
- * file size is less than chunkSize*(ECData-1), we are making empty block
- * to get the container created in non writing nodes. If replica index is
- * >0 then we know it's for ec container.
+ /*
+ Create Container should happen only as part of Write_Data phase of writeChunk.
+ In EC, we are doing empty putBlock.
+ In the partial stripe writes, if file size is less than chunkSize*(ECData-1),
+ we are making empty block to get the container created in non-writing nodes.
+ If replica index is > 0 then we know it's for ec container.
*/
if (container == null && ((isWriteStage || isCombinedStage)
|| cmdType == Type.PutSmallFile
|| cmdType == Type.PutBlock)) {
- // If container does not exist, create one for WriteChunk and
- // PutSmallFile request
+ // If a container does not exist, create one for WriteChunk and PutSmallFile request
responseProto = createContainer(msg);
metrics.incContainerOpsMetrics(Type.CreateContainer);
- metrics.incContainerOpsLatencies(Type.CreateContainer,
- Time.monotonicNowNanos() - startTime);
+ metrics.incContainerOpsLatencies(Type.CreateContainer, Time.monotonicNowNanos() - startTime);
if (responseProto.getResult() != Result.SUCCESS) {
StorageContainerException sce = new StorageContainerException(
@@ -315,14 +291,13 @@ private ContainerCommandResponseProto dispatchRequest(
|| dispatcherContext == null
|| cmdType == Type.PutBlock);
if (container2BCSIDMap != null) {
- // adds this container to list of containers created in the pipeline
- // with initial BCSID recorded as 0.
+ // Adds this container to list of containers created in the pipeline with initial BCSID recorded as 0.
container2BCSIDMap.putIfAbsent(containerID, 0L);
}
container = getContainer(containerID);
}
- // if container not found return error
+ // if container didn't find return error
if (container == null) {
StorageContainerException sce = new StorageContainerException(
"ContainerID " + containerID + " does not exist",
@@ -333,14 +308,13 @@ private ContainerCommandResponseProto dispatchRequest(
containerType = getContainerType(container);
} else {
if (!msg.hasCreateContainer()) {
- audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE,
- new Exception("MALFORMED_REQUEST"));
+ audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception("MALFORMED_REQUEST"));
return malformedRequest(msg);
}
containerType = msg.getCreateContainer().getContainerType();
}
- // Small performance optimization. We check if the operation is of type
- // write before trying to send CloseContainerAction.
+ // Small performance optimization.
+ // We check if the operation is of a type write before trying to send CloseContainerAction.
if (!HddsUtils.isReadOnly(msg)) {
sendCloseContainerActionIfNeeded(container);
}
@@ -359,75 +333,67 @@ private ContainerCommandResponseProto dispatchRequest(
if (responseProto != null) {
metrics.incContainerOpsLatencies(cmdType, opLatencyNs);
- // If the request is of Write Type and the container operation
- // is unsuccessful, it implies the applyTransaction on the container
- // failed. All subsequent transactions on the container should fail and
- // hence replica will be marked unhealthy here. In this case, a close
- // container action will be sent to SCM to close the container.
+ // If the request is of Write Type and the container operation is unsuccessful,
+ // it implies the applyTransaction on the container failed.
+ // All later transactions on the container should fail, and hence the replica will be marked unhealthy here.
+ // In this case, a close container action will be sent to SCM to close the container.
- // ApplyTransaction called on closed Container will fail with Closed
- // container exception. In such cases, ignore the exception here
- // If the container is already marked unhealthy, no need to change the
- // state here.
+ // ApplyTransaction called on closed Container will fail with Closed container exception.
+ // In such cases, ignore the exception here If the container is already marked unhealthy,
+ // no need to change the state here.
Result result = responseProto.getResult();
if (!HddsUtils.isReadOnly(msg) && !canIgnoreException(result)) {
- // If the container is open/closing and the container operation
- // has failed, it should be first marked unhealthy and the initiate the
- // close container action. This also implies this is the first
- // transaction which has failed, so the container is marked unhealthy
- // right here.
- // Once container is marked unhealthy, all the subsequent write
- // transactions will fail with UNHEALTHY_CONTAINER exception.
+ // If the container is open/closing and the container operation has failed,
+ // it should be first marked unhealthy and the initiate the close container action.
+ // This also implies this is the first transaction that has failed,
+ // so the container is marked unhealthy right here.
+ // Once a container is marked unhealthy,
+ // all the later writing transactions will fail with UNHEALTHY_CONTAINER exception.
if (container == null) {
- throw new NullPointerException(
- "Error on creating containers " + result + " " + responseProto
- .getMessage());
+ throw new NullPointerException("Error on creating containers " + result + " " + responseProto.getMessage());
}
- // For container to be moved to unhealthy state here, the container can
- // only be in open or closing state.
+ // For the container to be moved to unhealthy state here, the container can only be in open or closing state.
State containerState = container.getContainerData().getState();
Preconditions.checkState(
containerState == State.OPEN
|| containerState == State.CLOSING
|| containerState == State.RECOVERING);
- // mark and persist the container state to be unhealthy
+ // Mark and persist the container state to be unhealthy
try {
- // TODO HDDS-7096 + HDDS-8781: Use on demand scanning for the open
- // container instead.
+ // TODO HDDS-7096 + HDDS-8781: Use on demand scanning for the open container instead.
handler.markContainerUnhealthy(container,
ScanResult.unhealthy(ScanResult.FailureType.WRITE_FAILURE,
new File(container.getContainerData().getContainerPath()),
new StorageContainerException(result)));
LOG.info("Marked Container UNHEALTHY, ContainerID: {}", containerID);
} catch (IOException ioe) {
- // just log the error here in case marking the container fails,
- // Return the actual failure response to the client
- LOG.error("Failed to mark container " + containerID + " UNHEALTHY. ",
- ioe);
+ // Log the error here in case marking the container fails, return the actual failure response to the client
+ LOG.error("Failed to mark container {} UNHEALTHY. ", containerID, ioe);
}
- // in any case, the in memory state of the container should be unhealthy
- Preconditions.checkArgument(
- container.getContainerData().getState() == State.UNHEALTHY);
+ // In any case, the in memory state of the container should be unhealthy
+ Preconditions.checkArgument(container.getContainerData().getState() == State.UNHEALTHY);
sendCloseContainerActionIfNeeded(container);
}
- if (cmdType == Type.CreateContainer
- && result == Result.SUCCESS && dispatcherContext != null) {
+ if (cmdType == Type.CreateContainer && result == Result.SUCCESS && dispatcherContext != null) {
Preconditions.checkNotNull(dispatcherContext.getContainer2BCSIDMap());
- container2BCSIDMap.putIfAbsent(containerID, Long.valueOf(0));
+ container2BCSIDMap.putIfAbsent(containerID, 0L);
}
if (result == Result.SUCCESS) {
updateBCSID(container, dispatcherContext, cmdType);
audit(action, eventType, msg, dispatcherContext, AuditEventStatus.SUCCESS, null);
} else {
- //TODO HDDS-7096:
- // This is a too general place for on demand scanning.
- // Create a specific exception that signals for on demand scanning
- // and move this general scan to where it is more appropriate.
+ //TODO HDDS-7096: This is a too general place for on demand scanning.
+ // Create a specific exception
+ // that signals for on demand scanning and move this general scan to where it is more appropriate.
// Add integration tests to test the full functionality.
OnDemandContainerDataScanner.scanContainer(container);
- audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE,
+ audit(action,
+ eventType,
+ msg,
+ dispatcherContext,
+ AuditEventStatus.FAILURE,
new Exception(responseProto.getMessage()));
}
perf.appendOpLatencyNanos(opLatencyNs);
@@ -436,8 +402,7 @@ private ContainerCommandResponseProto dispatchRequest(
return responseProto;
} else {
// log failure
- audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE,
- new Exception("UNSUPPORTED_REQUEST"));
+ audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception("UNSUPPORTED_REQUEST"));
return unsupportedRequest(msg);
}
}
@@ -446,13 +411,11 @@ private long getSlowOpThresholdMs(ConfigurationSource config) {
return config.getTimeDuration(
HddsConfigKeys.HDDS_DATANODE_SLOW_OP_WARNING_THRESHOLD_KEY,
HddsConfigKeys.HDDS_DATANODE_SLOW_OP_WARNING_THRESHOLD_DEFAULT,
- TimeUnit.MILLISECONDS);
+ TimeUnit.MILLISECONDS);
}
- private void updateBCSID(Container container,
- DispatcherContext dispatcherContext, Type cmdType) {
- if (dispatcherContext != null && (cmdType == Type.PutBlock
- || cmdType == Type.PutSmallFile)) {
+ private void updateBCSID(Container container, DispatcherContext dispatcherContext, Type cmdType) {
+ if (dispatcherContext != null && (cmdType == Type.PutBlock || cmdType == Type.PutSmallFile)) {
Preconditions.checkNotNull(container);
long bcsID = container.getBlockCommitSequenceId();
long containerId = container.getContainerData().getContainerID();
@@ -460,35 +423,31 @@ private void updateBCSID(Container container,
container2BCSIDMap = dispatcherContext.getContainer2BCSIDMap();
Preconditions.checkNotNull(container2BCSIDMap);
Preconditions.checkArgument(container2BCSIDMap.containsKey(containerId));
- // updates the latest BCSID on every putBlock or putSmallFile
- // transaction over Ratis.
- container2BCSIDMap.computeIfPresent(containerId, (u, v) -> v = bcsID);
+ // updates the latest BCSID on every putBlock or putSmallFile transaction over Ratis.
+ container2BCSIDMap.computeIfPresent(containerId, (u, v) -> bcsID);
}
}
+
/**
* Create a container using the input container request.
+ *
* @param containerRequest - the container request which requires container
* to be created.
* @return ContainerCommandResponseProto container command response.
*/
@VisibleForTesting
- ContainerCommandResponseProto createContainer(
- ContainerCommandRequestProto containerRequest) {
+ ContainerCommandResponseProto createContainer(ContainerCommandRequestProto containerRequest) {
ContainerProtos.CreateContainerRequestProto.Builder createRequest =
ContainerProtos.CreateContainerRequestProto.newBuilder();
- ContainerType containerType =
- ContainerProtos.ContainerType.KeyValueContainer;
+ ContainerType containerType = ContainerProtos.ContainerType.KeyValueContainer;
createRequest.setContainerType(containerType);
if (containerRequest.hasWriteChunk()) {
- createRequest.setReplicaIndex(
- containerRequest.getWriteChunk().getBlockID().getReplicaIndex());
+ createRequest.setReplicaIndex(containerRequest.getWriteChunk().getBlockID().getReplicaIndex());
}
if (containerRequest.hasPutBlock()) {
- createRequest.setReplicaIndex(
- containerRequest.getPutBlock().getBlockData().getBlockID()
- .getReplicaIndex());
+ createRequest.setReplicaIndex(containerRequest.getPutBlock().getBlockData().getBlockID().getReplicaIndex());
}
ContainerCommandRequestProto.Builder requestBuilder =
@@ -506,26 +465,21 @@ ContainerCommandResponseProto createContainer(
return handler.handle(requestBuilder.build(), null, null);
}
- private void validateToken(
- ContainerCommandRequestProto msg) throws IOException {
- tokenVerifier.verify(
- msg,
- msg.getEncodedToken()
- );
+ private void validateToken(ContainerCommandRequestProto msg) throws IOException {
+ tokenVerifier.verify(msg, msg.getEncodedToken());
}
/**
- * This will be called as a part of creating the log entry during
- * startTransaction in Ratis on the leader node. In such cases, if the
- * container is not in open state for writing we should just fail.
+ * This will be called as a part of creating the log entry during startTransaction in Ratis on the leader node.
+ * In such cases, if the container is not in open state for writing, we should just fail.
* Leader will propagate the exception to client.
+ *
* @param msg container command proto
- * @throws StorageContainerException In case container state is open for write
- * requests and in invalid state for read requests.
+ * @throws StorageContainerException In case container state is open for write requests
+ * and in invalid state for read requests.
*/
@Override
- public void validateContainerCommand(
- ContainerCommandRequestProto msg) throws StorageContainerException {
+ public void validateContainerCommand(ContainerCommandRequestProto msg) throws StorageContainerException {
try {
validateToken(msg);
} catch (IOException ioe) {
@@ -555,19 +509,16 @@ public void validateContainerCommand(
State containerState = container.getContainerState();
String log = "Container " + containerID + " in " + containerState + " state";
- if (!HddsUtils.isReadOnly(msg)
- && !HddsUtils.isOpenToWriteState(containerState)) {
+ if (!HddsUtils.isReadOnly(msg) && !HddsUtils.isOpenToWriteState(containerState)) {
switch (cmdType) {
case CreateContainer:
// Create Container is idempotent. There is nothing to validate.
break;
case CloseContainer:
- // If the container is unhealthy, closeContainer will be rejected
- // while execution. Nothing to validate here.
+ // If the container is unhealthy, closeContainer will be rejected while execution. Nothing to validate here.
break;
default:
- // if the container is not open/recovering, no updates can happen. Just
- // throw an exception
+ // If the container is not open/recovering, no updates can happen. Throw an exception
ContainerNotOpenException cex = new ContainerNotOpenException(log);
audit(action, eventType, msg, null, AuditEventStatus.FAILURE, cex);
throw cex;
@@ -580,8 +531,9 @@ public void validateContainerCommand(
}
/**
- * If the container usage reaches the close threshold or the container is
- * marked unhealthy we send Close ContainerAction to SCM.
+ * If the container usage reaches the close threshold or the container is marked unhealthy,
+ * we send Close ContainerAction to SCM.
+ *
* @param container current state of container
*/
private void sendCloseContainerActionIfNeeded(Container container) {
@@ -590,12 +542,13 @@ private void sendCloseContainerActionIfNeeded(Container container) {
boolean shouldClose = isSpaceFull || isContainerUnhealthy(container);
if (shouldClose) {
ContainerData containerData = container.getContainerData();
- ContainerAction.Reason reason =
- isSpaceFull ? ContainerAction.Reason.CONTAINER_FULL :
- ContainerAction.Reason.CONTAINER_UNHEALTHY;
+ ContainerAction.Reason reason = isSpaceFull
+ ? ContainerAction.Reason.CONTAINER_FULL
+ : ContainerAction.Reason.CONTAINER_UNHEALTHY;
ContainerAction action = ContainerAction.newBuilder()
.setContainerID(containerData.getContainerID())
- .setAction(ContainerAction.Action.CLOSE).setReason(reason).build();
+ .setAction(ContainerAction.Action.CLOSE)
+ .setReason(reason).build();
context.addContainerActionIfAbsent(action);
}
}
@@ -606,8 +559,7 @@ private boolean isContainerFull(Container container) {
.orElse(Boolean.FALSE);
if (isOpen) {
ContainerData containerData = container.getContainerData();
- double containerUsedPercentage =
- 1.0f * containerData.getBytesUsed() / containerData.getMaxSize();
+ double containerUsedPercentage = 1.0f * containerData.getBytesUsed() / containerData.getMaxSize();
return containerUsedPercentage >= containerCloseThreshold;
} else {
return false;
@@ -620,11 +572,9 @@ private boolean isVolumeFull(Container container) {
.orElse(Boolean.FALSE);
if (isOpen) {
HddsVolume volume = container.getContainerData().getVolume();
- SpaceUsageSource precomputedVolumeSpace =
- volume.getCurrentUsage();
+ SpaceUsageSource precomputedVolumeSpace = volume.getCurrentUsage();
long volumeCapacity = precomputedVolumeSpace.getCapacity();
- long volumeFreeSpaceToSpare =
- freeSpaceCalculator.get(volumeCapacity);
+ long volumeFreeSpaceToSpare = freeSpaceCalculator.get(volumeCapacity);
long volumeFree = precomputedVolumeSpace.getAvailable();
long volumeCommitted = volume.getCommittedBytes();
long volumeAvailable = volumeFree - volumeCommitted;
@@ -634,9 +584,8 @@ private boolean isVolumeFull(Container container) {
}
private boolean isContainerUnhealthy(Container container) {
- return Optional.ofNullable(container).map(
- cont -> (cont.getContainerState() ==
- ContainerDataProto.State.UNHEALTHY))
+ return Optional.ofNullable(container)
+ .map(cont -> (cont.getContainerState() == ContainerDataProto.State.UNHEALTHY))
.orElse(Boolean.FALSE);
}
@@ -647,7 +596,7 @@ public Handler getHandler(ContainerProtos.ContainerType containerType) {
@Override
public void setClusterId(String clusterId) {
- Preconditions.checkNotNull(clusterId, "clusterId Cannot be null");
+ Preconditions.checkNotNull(clusterId, "clusterId cannot be null");
if (this.clusterId == null) {
this.clusterId = clusterId;
for (Map.Entry handlerMap : handlers.entrySet()) {
@@ -656,6 +605,12 @@ public void setClusterId(String clusterId) {
}
}
+ /**
+ * Retrieves a container from the container set based on the given container ID.
+ *
+ * @param containerID the unique identifier of the container to be retrieved.
+ * @return the container object corresponding to the specified container ID.
+ */
@VisibleForTesting
public Container getContainer(long containerID) {
return containerSet.getContainer(containerID);
@@ -670,6 +625,11 @@ private ContainerType getContainerType(Container container) {
return container.getContainerType();
}
+ /**
+ * Sets the container metrics for testing purposes.
+ *
+ * @param containerMetrics The ContainerMetrics instance to be used for testing.
+ */
@VisibleForTesting
public void setMetricsForTesting(ContainerMetrics containerMetrics) {
this.metrics = containerMetrics;
@@ -679,21 +639,19 @@ private EventType getEventType(ContainerCommandRequestProto msg) {
return HddsUtils.isReadOnly(msg) ? EventType.READ : EventType.WRITE;
}
- private void audit(AuditAction action, EventType eventType,
- ContainerCommandRequestProto msg, DispatcherContext dispatcherContext,
- AuditEventStatus result, Throwable exception) {
+ private void audit(AuditAction action, EventType eventType, ContainerCommandRequestProto msg,
+ DispatcherContext dispatcherContext, AuditEventStatus result, Throwable exception) {
+
Map params;
AuditMessage amsg;
switch (result) {
case SUCCESS:
if (isAllowed(action.getAction())) {
params = getAuditParams(msg, dispatcherContext);
- if (eventType == EventType.READ &&
- AUDIT.getLogger().isInfoEnabled(AuditMarker.READ.getMarker())) {
+ if (eventType == EventType.READ && AUDIT.getLogger().isInfoEnabled(AuditMarker.READ.getMarker())) {
amsg = buildAuditMessageForSuccess(action, params);
AUDIT.logReadSuccess(amsg);
- } else if (eventType == EventType.WRITE &&
- AUDIT.getLogger().isInfoEnabled(AuditMarker.WRITE.getMarker())) {
+ } else if (eventType == EventType.WRITE && AUDIT.getLogger().isInfoEnabled(AuditMarker.WRITE.getMarker())) {
amsg = buildAuditMessageForSuccess(action, params);
AUDIT.logWriteSuccess(amsg);
}
@@ -702,21 +660,17 @@ private void audit(AuditAction action, EventType eventType,
case FAILURE:
params = getAuditParams(msg, dispatcherContext);
- if (eventType == EventType.READ &&
- AUDIT.getLogger().isErrorEnabled(AuditMarker.READ.getMarker())) {
+ if (eventType == EventType.READ && AUDIT.getLogger().isErrorEnabled(AuditMarker.READ.getMarker())) {
amsg = buildAuditMessageForFailure(action, params, exception);
AUDIT.logReadFailure(amsg);
- } else if (eventType == EventType.WRITE &&
- AUDIT.getLogger().isErrorEnabled(AuditMarker.WRITE.getMarker())) {
+ } else if (eventType == EventType.WRITE && AUDIT.getLogger().isErrorEnabled(AuditMarker.WRITE.getMarker())) {
amsg = buildAuditMessageForFailure(action, params, exception);
AUDIT.logWriteFailure(amsg);
}
break;
default:
- if (LOG.isDebugEnabled()) {
- LOG.debug("Invalid audit event status - {}", result);
- }
+ LOG.debug("Invalid audit event status - {}", result);
}
}
@@ -724,12 +678,19 @@ private void performanceAudit(AuditAction action, ContainerCommandRequestProto m
DispatcherContext dispatcherContext, PerformanceStringBuilder performance, long opLatencyNs) {
if (isOperationSlow(opLatencyNs)) {
Map params = getAuditParams(msg, dispatcherContext);
- AuditMessage auditMessage =
- buildAuditMessageForPerformance(action, params, performance);
+ AuditMessage auditMessage = buildAuditMessageForPerformance(action, params, performance);
AUDIT.logPerformance(auditMessage);
}
}
+ /**
+ * Builds an audit message for performance tracking purposes.
+ *
+ * @param op The audit action representing the operation being performed.
+ * @param auditMap A map containing key-value pairs for audit parameters.
+ * @param performance The performance string builder encapsulating performance metrics.
+ * @return An AuditMessage instance configured with the provided parameters.
+ */
public AuditMessage buildAuditMessageForPerformance(AuditAction op,
Map auditMap, PerformanceStringBuilder performance) {
return new AuditMessage.Builder()
@@ -743,9 +704,7 @@ public AuditMessage buildAuditMessageForPerformance(AuditAction op,
//TODO: use GRPC to fetch user and ip details
@Override
- public AuditMessage buildAuditMessageForSuccess(AuditAction op,
- Map auditMap) {
-
+ public AuditMessage buildAuditMessageForSuccess(AuditAction op, Map auditMap) {
return new AuditMessage.Builder()
.setUser(null)
.atIp(null)
@@ -756,9 +715,7 @@ public AuditMessage buildAuditMessageForSuccess(AuditAction op,
}
@Override
- public AuditMessage buildAuditMessageForFailure(AuditAction op,
- Map auditMap, Throwable throwable) {
-
+ public AuditMessage buildAuditMessageForFailure(AuditAction op, Map auditMap, Throwable throwable) {
return new AuditMessage.Builder()
.setUser(null)
.atIp(null)
@@ -776,8 +733,12 @@ enum EventType {
/**
* Checks if the action is allowed for audit.
- * @param action
- * @return true or false accordingly.
+ *
+ * @param action The action to be checked for allowance. Valid actions include:
+ * "CLOSE_CONTAINER", "CREATE_CONTAINER", "LIST_CONTAINER",
+ * "DELETE_CONTAINER", "READ_CONTAINER", "UPDATE_CONTAINER",
+ * and "DELETE_BLOCK".
+ * @return {@code true} if the action is allowed; {@code false} otherwise.
*/
private boolean isAllowed(String action) {
switch (action) {
@@ -789,14 +750,15 @@ private boolean isAllowed(String action) {
case "UPDATE_CONTAINER":
case "DELETE_BLOCK":
return true;
- default: return false;
+ default:
+ return false;
}
}
@Override
- public StateMachine.DataChannel getStreamDataChannel(
- ContainerCommandRequestProto msg)
- throws StorageContainerException {
+ public StateMachine.DataChannel getStreamDataChannel(ContainerCommandRequestProto msg)
+ throws StorageContainerException {
+
long containerID = msg.getContainerID();
Container container = getContainer(containerID);
if (container != null) {
@@ -811,160 +773,153 @@ public StateMachine.DataChannel getStreamDataChannel(
private static DNAction getAuditAction(Type cmdType) {
switch (cmdType) {
- case CreateContainer : return DNAction.CREATE_CONTAINER;
- case ReadContainer : return DNAction.READ_CONTAINER;
- case UpdateContainer : return DNAction.UPDATE_CONTAINER;
- case DeleteContainer : return DNAction.DELETE_CONTAINER;
- case ListContainer : return DNAction.LIST_CONTAINER;
- case PutBlock : return DNAction.PUT_BLOCK;
- case GetBlock : return DNAction.GET_BLOCK;
- case DeleteBlock : return DNAction.DELETE_BLOCK;
- case ListBlock : return DNAction.LIST_BLOCK;
- case ReadChunk : return DNAction.READ_CHUNK;
- case DeleteChunk : return DNAction.DELETE_CHUNK;
- case WriteChunk : return DNAction.WRITE_CHUNK;
- case ListChunk : return DNAction.LIST_CHUNK;
- case CompactChunk : return DNAction.COMPACT_CHUNK;
- case PutSmallFile : return DNAction.PUT_SMALL_FILE;
- case GetSmallFile : return DNAction.GET_SMALL_FILE;
- case CloseContainer : return DNAction.CLOSE_CONTAINER;
- case GetCommittedBlockLength : return DNAction.GET_COMMITTED_BLOCK_LENGTH;
- case StreamInit : return DNAction.STREAM_INIT;
- case FinalizeBlock : return DNAction.FINALIZE_BLOCK;
- case Echo : return DNAction.ECHO;
- default :
+ case CreateContainer:
+ return DNAction.CREATE_CONTAINER;
+ case ReadContainer:
+ return DNAction.READ_CONTAINER;
+ case UpdateContainer:
+ return DNAction.UPDATE_CONTAINER;
+ case DeleteContainer:
+ return DNAction.DELETE_CONTAINER;
+ case ListContainer:
+ return DNAction.LIST_CONTAINER;
+ case PutBlock:
+ return DNAction.PUT_BLOCK;
+ case GetBlock:
+ return DNAction.GET_BLOCK;
+ case DeleteBlock:
+ return DNAction.DELETE_BLOCK;
+ case ListBlock:
+ return DNAction.LIST_BLOCK;
+ case ReadChunk:
+ return DNAction.READ_CHUNK;
+ case DeleteChunk:
+ return DNAction.DELETE_CHUNK;
+ case WriteChunk:
+ return DNAction.WRITE_CHUNK;
+ case ListChunk:
+ return DNAction.LIST_CHUNK;
+ case CompactChunk:
+ return DNAction.COMPACT_CHUNK;
+ case PutSmallFile:
+ return DNAction.PUT_SMALL_FILE;
+ case GetSmallFile:
+ return DNAction.GET_SMALL_FILE;
+ case CloseContainer:
+ return DNAction.CLOSE_CONTAINER;
+ case GetCommittedBlockLength:
+ return DNAction.GET_COMMITTED_BLOCK_LENGTH;
+ case StreamInit:
+ return DNAction.STREAM_INIT;
+ case FinalizeBlock:
+ return DNAction.FINALIZE_BLOCK;
+ case Echo:
+ return DNAction.ECHO;
+ case VerifyBlock:
+ return DNAction.VERIFY_BLOCK;
+ default:
LOG.debug("Invalid command type - {}", cmdType);
return null;
}
}
- private static Map getAuditParams(
- ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
+ private static Map getAuditParams(ContainerCommandRequestProto msg,
+ DispatcherContext dispatcherContext) {
Map auditParams = new TreeMap<>();
Type cmdType = msg.getCmdType();
String containerID = String.valueOf(msg.getContainerID());
switch (cmdType) {
case CreateContainer:
auditParams.put(AUDIT_PARAM_CONTAINER_ID, containerID);
- auditParams.put(AUDIT_PARAM_CONTAINER_TYPE,
- msg.getCreateContainer().getContainerType().toString());
+ auditParams.put(AUDIT_PARAM_CONTAINER_TYPE, msg.getCreateContainer().getContainerType().toString());
return auditParams;
case ReadContainer:
+ case CloseContainer:
auditParams.put(AUDIT_PARAM_CONTAINER_ID, containerID);
return auditParams;
case UpdateContainer:
auditParams.put(AUDIT_PARAM_CONTAINER_ID, containerID);
- auditParams.put(AUDIT_PARAM_FORCE_UPDATE,
- String.valueOf(msg.getUpdateContainer().getForceUpdate()));
+ auditParams.put(AUDIT_PARAM_FORCE_UPDATE, String.valueOf(msg.getUpdateContainer().getForceUpdate()));
return auditParams;
case DeleteContainer:
auditParams.put(AUDIT_PARAM_CONTAINER_ID, containerID);
- auditParams.put(AUDIT_PARAM_FORCE_DELETE,
- String.valueOf(msg.getDeleteContainer().getForceDelete()));
+ auditParams.put(AUDIT_PARAM_FORCE_DELETE, String.valueOf(msg.getDeleteContainer().getForceDelete()));
return auditParams;
case ListContainer:
auditParams.put(AUDIT_PARAM_START_CONTAINER_ID, containerID);
- auditParams.put(AUDIT_PARAM_COUNT,
- String.valueOf(msg.getListContainer().getCount()));
+ auditParams.put(AUDIT_PARAM_COUNT, String.valueOf(msg.getListContainer().getCount()));
return auditParams;
case PutBlock:
try {
auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockData.getFromProtoBuf(msg.getPutBlock().getBlockData())
- .toString());
+ BlockData.getFromProtoBuf(msg.getPutBlock().getBlockData()).toString());
} catch (IOException ex) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Encountered error parsing BlockData from protobuf: "
- + ex.getMessage());
- }
+ LOG.trace("Encountered error parsing BlockData from protobuf: {}", ex.getMessage(), ex);
return null;
}
return auditParams;
case GetBlock:
- auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getGetBlock().getBlockID()).toString());
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getGetBlock().getBlockID()).toString());
return auditParams;
case DeleteBlock:
- auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getDeleteBlock().getBlockID())
- .toString());
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getDeleteBlock().getBlockID()).toString());
return auditParams;
case ListBlock:
- auditParams.put(AUDIT_PARAM_START_LOCAL_ID,
- String.valueOf(msg.getListBlock().getStartLocalID()));
+ auditParams.put(AUDIT_PARAM_START_LOCAL_ID, String.valueOf(msg.getListBlock().getStartLocalID()));
auditParams.put(AUDIT_PARAM_COUNT, String.valueOf(msg.getListBlock().getCount()));
return auditParams;
case ReadChunk:
- auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getReadChunk().getBlockID()).toString());
- auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET,
- String.valueOf(msg.getReadChunk().getChunkData().getOffset()));
- auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE,
- String.valueOf(msg.getReadChunk().getChunkData().getLen()));
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getReadChunk().getBlockID()).toString());
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET, String.valueOf(msg.getReadChunk().getChunkData().getOffset()));
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE, String.valueOf(msg.getReadChunk().getChunkData().getLen()));
return auditParams;
case DeleteChunk:
- auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getDeleteChunk().getBlockID())
- .toString());
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getDeleteChunk().getBlockID()).toString());
return auditParams;
case WriteChunk:
- auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getWriteChunk().getBlockID())
- .toString());
- auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET,
- String.valueOf(msg.getWriteChunk().getChunkData().getOffset()));
- auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE,
- String.valueOf(msg.getWriteChunk().getChunkData().getLen()));
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getWriteChunk().getBlockID()).toString());
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET, String.valueOf(msg.getWriteChunk().getChunkData().getOffset()));
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE, String.valueOf(msg.getWriteChunk().getChunkData().getLen()));
if (dispatcherContext != null && dispatcherContext.getStage() != null) {
auditParams.put(AUDIT_PARAM_BLOCK_DATA_STAGE, dispatcherContext.getStage().toString());
}
return auditParams;
case ListChunk:
- auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getListChunk().getBlockID()).toString());
- auditParams.put(AUDIT_PARAM_PREV_CHUNKNAME, msg.getListChunk().getPrevChunkName());
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getListChunk().getBlockID()).toString());
+ auditParams.put(AUDIT_PARAM_PREV_CHUNK_NAME, msg.getListChunk().getPrevChunkName());
auditParams.put(AUDIT_PARAM_COUNT, String.valueOf(msg.getListChunk().getCount()));
return auditParams;
- case CompactChunk: return null; //CompactChunk operation
+ case CompactChunk:
+ return null; //CompactChunk operation
case PutSmallFile:
try {
auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockData.getFromProtoBuf(msg.getPutSmallFile()
- .getBlock().getBlockData()).toString());
+ BlockData.getFromProtoBuf(msg.getPutSmallFile().getBlock().getBlockData()).toString());
auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET,
String.valueOf(msg.getPutSmallFile().getChunkInfo().getOffset()));
- auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE,
- String.valueOf(msg.getPutSmallFile().getChunkInfo().getLen()));
+ auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE, String.valueOf(msg.getPutSmallFile().getChunkInfo().getLen()));
} catch (IOException ex) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Encountered error parsing BlockData from protobuf: "
- + ex.getMessage());
- }
+ LOG.trace("Encountered error parsing BlockData from protobuf: {}", ex.getMessage(), ex);
}
return auditParams;
case GetSmallFile:
auditParams.put(AUDIT_PARAM_BLOCK_DATA,
- BlockID.getFromProtobuf(msg.getGetSmallFile().getBlock().getBlockID())
- .toString());
- return auditParams;
-
- case CloseContainer:
- auditParams.put(AUDIT_PARAM_CONTAINER_ID, containerID);
+ BlockID.getFromProtobuf(msg.getGetSmallFile().getBlock().getBlockID()).toString());
return auditParams;
case GetCommittedBlockLength:
@@ -979,11 +934,15 @@ private static Map getAuditParams(
.toString());
return auditParams;
+ case VerifyBlock:
+ auditParams.put(
+ "verifyBlock",
+ BlockID.getFromProtobuf(msg.getVerifyBlock().getBlockID()).toString());
+
default :
LOG.debug("Invalid command type - {}", cmdType);
return null;
}
-
}
private boolean isOperationSlow(long opLatencyNs) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
index bfdff69be46..bba672a0c76 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -21,11 +21,12 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.util.Objects;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
@@ -54,12 +55,10 @@ public abstract class Handler {
protected String clusterId;
protected final ContainerMetrics metrics;
protected String datanodeId;
- private IncrementalReportSender icrSender;
+ private final IncrementalReportSender icrSender;
- protected Handler(ConfigurationSource config, String datanodeId,
- ContainerSet contSet, VolumeSet volumeSet,
- ContainerMetrics containerMetrics,
- IncrementalReportSender icrSender) {
+ protected Handler(ConfigurationSource config, String datanodeId, ContainerSet contSet, VolumeSet volumeSet,
+ ContainerMetrics containerMetrics, IncrementalReportSender icrSender) {
this.conf = config;
this.containerSet = contSet;
this.volumeSet = volumeSet;
@@ -68,28 +67,41 @@ protected Handler(ConfigurationSource config, String datanodeId,
this.icrSender = icrSender;
}
- public static Handler getHandlerForContainerType(
- final ContainerType containerType, final ConfigurationSource config,
- final String datanodeId, final ContainerSet contSet,
- final VolumeSet volumeSet, final ContainerMetrics metrics,
+ /**
+ * Returns a handler for the specified container type.
+ *
+ * @param containerType the type of container for which the handler is required
+ * @param config the configuration source
+ * @param datanodeId the ID of the data node
+ * @param contSet the set of containers
+ * @param volumeSet the set of volumes
+ * @param metrics metrics for the container
+ * @param icrSender the incremental report sender
+ * @return a Handler for the specified container type
+ * @throws IllegalArgumentException if the container type does not exist
+ */
+ public static Handler getHandlerForContainerType(ContainerType containerType, ConfigurationSource config,
+ String datanodeId, ContainerSet contSet, VolumeSet volumeSet, ContainerMetrics metrics,
IncrementalReportSender icrSender) {
- switch (containerType) {
- case KeyValueContainer:
- return new KeyValueHandler(config,
- datanodeId, contSet, volumeSet, metrics,
- icrSender);
- default:
- throw new IllegalArgumentException("Handler for ContainerType: " +
- containerType + "doesn't exist.");
+ if (Objects.requireNonNull(containerType) == ContainerType.KeyValueContainer) {
+ return new KeyValueHandler(config, datanodeId, contSet, volumeSet, metrics, icrSender);
}
+ throw new IllegalArgumentException("Handler for ContainerType: " + containerType + "doesn't exist.");
}
- public abstract StateMachine.DataChannel getStreamDataChannel(
- Container container, ContainerCommandRequestProto msg)
- throws StorageContainerException;
+ /**
+ * Retrieves the data channel stream for a given container based on the specified request message.
+ *
+ * @param container the container for which the data channel will be retrieved
+ * @param msg the command request message associated with the data channel retrieval
+ * @return the data channel stream corresponding to the given container and request message
+ * @throws StorageContainerException if an error occurs while retrieving the data channel
+ */
+ public abstract StateMachine.DataChannel getStreamDataChannel(Container container, ContainerCommandRequestProto msg)
+ throws StorageContainerException;
/**
- * Returns the Id of this datanode.
+ * Returns the id of this datanode.
*
* @return datanode Id
*/
@@ -98,41 +110,39 @@ protected String getDatanodeId() {
}
/**
- * This should be called whenever there is state change. It will trigger
- * an ICR to SCM.
+ * This should be called whenever there is state change. It will trigger an ICR to SCM.
*
* @param container Container for which ICR has to be sent
*/
- protected void sendICR(final Container container)
- throws StorageContainerException {
- if (container
- .getContainerState() == ContainerProtos.ContainerDataProto
- .State.RECOVERING) {
+ protected void sendICR(final Container container) throws StorageContainerException {
+ if (container.getContainerState() == State.RECOVERING) {
// Ignoring the recovering containers reports for now.
return;
}
icrSender.send(container);
}
- public abstract ContainerCommandResponseProto handle(
- ContainerCommandRequestProto msg, Container container,
+ /**
+ * Handles the given container command request.
+ *
+ * @param msg the container command request protocol message
+ * @param container the container to be handled
+ * @param dispatcherContext the context of the dispatcher handling the command
+ * @return the response protocol for the executed command
+ */
+ public abstract ContainerCommandResponseProto handle(ContainerCommandRequestProto msg, Container container,
DispatcherContext dispatcherContext);
/**
* Imports container from a raw input stream.
*/
- public abstract Container importContainer(
- ContainerData containerData, InputStream rawContainerStream,
- TarContainerPacker packer)
- throws IOException;
+ public abstract Container importContainer(ContainerData containerData, InputStream rawContainerStream,
+ TarContainerPacker packer) throws IOException;
/**
* Exports container to the output stream.
*/
- public abstract void exportContainer(
- Container container,
- OutputStream outputStream,
- TarContainerPacker packer)
+ public abstract void exportContainer(Container container, OutputStream outputStream, TarContainerPacker packer)
throws IOException;
/**
@@ -141,84 +151,84 @@ public abstract void exportContainer(
public abstract void stop();
/**
- * Marks the container for closing. Moves the container to CLOSING state.
+ * Marks the container for closing. Moves the container to {@link State#CLOSING} state.
*
* @param container container to update
* @throws IOException in case of exception
*/
- public abstract void markContainerForClose(Container container)
- throws IOException;
+ public abstract void markContainerForClose(Container container) throws IOException;
/**
- * Marks the container Unhealthy. Moves the container to UNHEALTHY state.
+ * Marks the container Unhealthy. Moves the container to {@link State#UNHEALTHY} state.
*
* @param container container to update
* @param reason The reason the container was marked unhealthy
* @throws IOException in case of exception
*/
- public abstract void markContainerUnhealthy(Container container,
- ScanResult reason)
- throws IOException;
+ public abstract void markContainerUnhealthy(Container container, ScanResult reason) throws IOException;
/**
- * Moves the Container to QUASI_CLOSED state.
+ * Moves the Container to {@link State#QUASI_CLOSED} state.
*
* @param container container to be quasi closed
- * @param reason The reason the container was quasi closed, for logging
- * purposes.
- * @throws IOException
+ * @param reason The reason the container was quasi closed, for logging purposes.
*/
- public abstract void quasiCloseContainer(Container container, String reason)
- throws IOException;
+ public abstract void quasiCloseContainer(Container container, String reason) throws IOException;
/**
- * Moves the Container to CLOSED state.
+ * Moves the Container to {@link State#CLOSED} state.
*
* @param container container to be closed
- * @throws IOException
*/
- public abstract void closeContainer(Container container)
- throws IOException;
+ public abstract void closeContainer(Container container) throws IOException;
/**
* Deletes the given container.
*
* @param container container to be deleted
- * @param force if this is set to true, we delete container without
- * checking
- * state of the container.
- * @throws IOException
+ * @param force if this is set to true, we delete container without checking state of the container.
*/
- public abstract void deleteContainer(Container container, boolean force)
- throws IOException;
+ public abstract void deleteContainer(Container container, boolean force) throws IOException;
/**
* Deletes the given files associated with a block of the container.
*
* @param container container whose block is to be deleted
* @param blockData block to be deleted
- * @throws IOException
*/
- public abstract void deleteBlock(Container container, BlockData blockData)
- throws IOException;
+ public abstract void deleteBlock(Container container, BlockData blockData) throws IOException;
/**
- * Deletes the possible onDisk but unreferenced blocks/chunks with localID
- * in the container.
+ * Deletes the possible onDisk but unreferenced blocks/chunks with localID in the container.
*
* @param container container whose block/chunk is to be deleted
* @param localID localId of the block/chunk
- * @throws IOException
*/
- public abstract void deleteUnreferenced(Container container, long localID)
- throws IOException;
+ public abstract void deleteUnreferenced(Container container, long localID) throws IOException;
+ /**
+ * Adds a finalized block to a container.
+ *
+ * @param container The container to which the finalized block will be added.
+ * @param localID The local identifier for the block.
+ */
public abstract void addFinalizedBlock(Container container, long localID);
+ /**
+ * Checks if a finalized block exists in the specified container with the given local ID.
+ *
+ * @param container the container to be checked
+ * @param localID the local ID of the block to be verified
+ * @return true if the finalized block exists, false otherwise
+ */
public abstract boolean isFinalizedBlockExist(Container container, long localID);
+ /**
+ * Sets the cluster ID for this handler.
+ *
+ * @param clusterID the new cluster ID to be set
+ */
public void setClusterID(String clusterID) {
this.clusterId = clusterID;
}
-
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index b3398de07ad..78e43c4bf48 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -26,7 +26,6 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@@ -41,7 +40,6 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
-import java.util.stream.Collectors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.hdds.HddsUtils;
@@ -100,56 +98,53 @@
import org.apache.ratis.util.LifeCycle;
import org.apache.ratis.util.TaskQueue;
import org.apache.ratis.util.function.CheckedSupplier;
-import org.apache.ratis.util.JavaUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.ratis.util.JavaUtils.completeExceptionally;
+
/**
- * A {@link StateMachine} for containers,
- * which is responsible for handling different types of container requests.
+ * A {@link StateMachine} for containers, which is responsible for handling different types of container requests.
*
* The container requests can be divided into readonly request, WriteChunk request and other write requests.
- * - Read only requests (see {@link HddsUtils#isReadOnly}) are handled by {@link #query(Message)}.
+ * - Read-only requests (see {@link HddsUtils#isReadOnly}) are handled by {@link #query(Message)}.
* - WriteChunk request contains user data
- * - Other write request does not contain user data.
+ * - Another writing request does not contain user data.
*
- * In order to optimize the write throughput, a WriteChunk request is processed :
- * (1) {@link #startTransaction(RaftClientRequest)} separate user data from the client request
- * (2) the user data is written directly into the state machine via {@link #write}
- * (3) transaction is committed via {@link #applyTransaction(TransactionContext)}
+ * In order to optimize the writing throughput, a WriteChunk request is processed:
+ *
+ * - {@link #startTransaction(RaftClientRequest)} separate user data from the client request
+ *
- the user data is written directly into the state machine via {@link #write}
+ *
- transaction is committed via {@link #applyTransaction(TransactionContext)}
+ *
*
- * For the other write requests,
+ * For the other writing requests,
* the transaction is directly committed via {@link #applyTransaction(TransactionContext)}.
*
- * There are 2 ordering operation which are enforced right now in the code,
- * 1) WriteChunk must be executed after the CreateContainer;
- * otherwise, WriteChunk will fail with container not found.
- * 2) WriteChunk commit is executed after WriteChunk write.
- * Then, WriteChunk commit and CreateContainer will be executed in the same order.
+ * There are two ordering operations that are enforced right now in the code,
+ *
+ * - WriteChunk must be executed after the CreateContainer; otherwise, WriteChunk will fail with container not found.
+ *
- WriteChunk commit is executed after WriteChunk write.
+ * Then, WriteChunk commit and CreateContainer will be executed in the same order.
+ *
*/
public class ContainerStateMachine extends BaseStateMachine {
- static final Logger LOG =
- LoggerFactory.getLogger(ContainerStateMachine.class);
+ static final Logger LOG = LoggerFactory.getLogger(ContainerStateMachine.class);
static class TaskQueueMap {
private final Map map = new HashMap<>();
- synchronized CompletableFuture submit(
- long containerId,
- CheckedSupplier task,
- ExecutorService executor) {
- final TaskQueue queue = map.computeIfAbsent(
- containerId, id -> new TaskQueue("container" + id));
- final CompletableFuture f
- = queue.submit(task, executor);
+ synchronized CompletableFuture submit(long containerId,
+ CheckedSupplier task, ExecutorService executor) {
+ final TaskQueue queue = map.computeIfAbsent(containerId, id -> new TaskQueue("container" + id));
+ final CompletableFuture f = queue.submit(task, executor);
// after the task is completed, remove the queue if the queue is empty.
f.thenAccept(dummy -> removeIfEmpty(containerId));
return f;
}
synchronized void removeIfEmpty(long containerId) {
- map.computeIfPresent(containerId,
- (id, q) -> q.isEmpty() ? null : q);
+ map.computeIfPresent(containerId, (id, q) -> q.isEmpty() ? null : q);
}
}
@@ -182,14 +177,12 @@ long getStartTime() {
}
}
- private final SimpleStateMachineStorage storage =
- new SimpleStateMachineStorage();
+ private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
private final RaftGroupId gid;
private final ContainerDispatcher dispatcher;
private final ContainerController containerController;
private final XceiverServerRatis ratisServer;
- private final ConcurrentHashMap> writeChunkFutureMap;
+ private final ConcurrentHashMap> writeChunkFutureMap;
// keeps track of the containers created per pipeline
private final Map container2BCSIDMap;
@@ -203,7 +196,7 @@ long getStartTime() {
private final Semaphore applyTransactionSemaphore;
private final boolean waitOnBothFollowers;
private final HddsDatanodeService datanodeService;
- private static Semaphore semaphore = new Semaphore(1);
+ private static final Semaphore SEMAPHORE = new Semaphore(1);
/**
* CSM metrics.
@@ -211,13 +204,16 @@ long getStartTime() {
private final CSMMetrics metrics;
@SuppressWarnings("parameternumber")
- public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupId gid,
+ public ContainerStateMachine(
+ HddsDatanodeService hddsDatanodeService,
+ RaftGroupId gid,
ContainerDispatcher dispatcher,
ContainerController containerController,
List chunkExecutors,
XceiverServerRatis ratisServer,
ConfigurationSource conf,
- String threadNamePrefix) {
+ String threadNamePrefix
+ ) {
this.datanodeService = hddsDatanodeService;
this.gid = gid;
this.dispatcher = dispatcher;
@@ -230,10 +226,9 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI
OzoneConfigKeys.HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT,
OzoneConfigKeys.HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT_DEFAULT,
StorageUnit.BYTES);
- // cache with FIFO eviction, and if element not found, this needs
- // to be obtained from disk for slow follower
+ // Cache with FIFO eviction, and if an element not found, this needs to be obtained from disk for slow follower
stateMachineDataCache = new ResourceCache<>(
- (index, data) -> ((ByteString)data).size(),
+ (index, data) -> data.size(),
pendingRequestsBytesLimit,
(p) -> {
if (p.wasEvicted()) {
@@ -249,23 +244,17 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI
OzoneConfigKeys.HDDS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_KEY,
OzoneConfigKeys.HDDS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_DEFAULT);
int maxPendingApplyTransactions = conf.getInt(
- ScmConfigKeys.
- HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS,
- ScmConfigKeys.
- HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT);
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS,
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT);
applyTransactionSemaphore = new Semaphore(maxPendingApplyTransactions);
stateMachineHealthy = new AtomicBoolean(true);
ThreadFactory threadFactory = new ThreadFactoryBuilder()
- .setNameFormat(
- threadNamePrefix + "ContainerOp-" + gid.getUuid() + "-%d")
+ .setNameFormat(threadNamePrefix + "ContainerOp-" + gid.getUuid() + "-%d")
.build();
- this.executor = Executors.newFixedThreadPool(numContainerOpExecutors,
- threadFactory);
-
- this.waitOnBothFollowers = conf.getObject(
- DatanodeConfiguration.class).waitOnAllFollowers();
+ this.executor = Executors.newFixedThreadPool(numContainerOpExecutors, threadFactory);
+ this.waitOnBothFollowers = conf.getObject(DatanodeConfiguration.class).waitOnAllFollowers();
}
@Override
@@ -273,14 +262,17 @@ public StateMachineStorage getStateMachineStorage() {
return storage;
}
+ /**
+ * Retrieves the current metrics for the {@link ContainerStateMachine}.
+ *
+ * @return The CSMMetrics instance containing the metrics data.
+ */
public CSMMetrics getMetrics() {
return metrics;
}
@Override
- public void initialize(
- RaftServer server, RaftGroupId id, RaftStorage raftStorage)
- throws IOException {
+ public void initialize(RaftServer server, RaftGroupId id, RaftStorage raftStorage) throws IOException {
super.initialize(server, id, raftStorage);
storage.init(raftStorage);
ratisServer.notifyGroupAdd(gid);
@@ -288,59 +280,66 @@ public void initialize(
loadSnapshot(storage.getLatestSnapshot());
}
- private long loadSnapshot(SingleFileSnapshotInfo snapshot)
- throws IOException {
+ private void loadSnapshot(SingleFileSnapshotInfo snapshot) throws IOException {
if (snapshot == null) {
TermIndex empty = TermIndex.valueOf(0, RaftLog.INVALID_LOG_INDEX);
- LOG.info("{}: The snapshot info is null. Setting the last applied index " +
- "to:{}", gid, empty);
+ LOG.info("{}: The snapshot info is null. Setting the last applied index to:{}", gid, empty);
setLastAppliedTermIndex(empty);
- return empty.getIndex();
+ empty.getIndex();
+ return;
}
final File snapshotFile = snapshot.getFile().getPath().toFile();
- final TermIndex last =
- SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
+ final TermIndex last = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
LOG.info("{}: Setting the last applied index to {}", gid, last);
setLastAppliedTermIndex(last);
- // initialize the dispatcher with snapshot so that it build the missing
- // container list
+ // Initialize the dispatcher with snapshot so that it builds the missing container list
buildMissingContainerSet(snapshotFile);
- return last.getIndex();
+ last.getIndex();
}
+ /**
+ * Initializes the dispatcher with the provided snapshot file and builds the
+ * missing container list by reading the container data from the snapshot file
+ * and updating the internal container-to-BCSID map.
+ *
+ * @param snapshotFile the snapshot file containing the container and block
+ * container mapping information used to build the missing
+ * container set.
+ * @throws IOException if an I/O error occurs while reading the snapshot file
+ * or updating the container-to-BCSID map.
+ */
@VisibleForTesting
public void buildMissingContainerSet(File snapshotFile) throws IOException {
- // initialize the dispatcher with snapshot so that it build the missing
- // container list
+ // Initialize the dispatcher with snapshot so that it builds the missing container list
try (FileInputStream fin = new FileInputStream(snapshotFile)) {
- ContainerProtos.Container2BCSIDMapProto proto =
- ContainerProtos.Container2BCSIDMapProto
- .parseFrom(fin);
- // read the created containers list from the snapshot file and add it to
- // the container2BCSIDMap here.
+ ContainerProtos.Container2BCSIDMapProto proto = ContainerProtos.Container2BCSIDMapProto.parseFrom(fin);
+ // Read the created containers list from the snapshot file and add it to the container2BCSIDMap here.
// container2BCSIDMap will further grow as and when containers get created
container2BCSIDMap.putAll(proto.getContainer2BCSIDMap());
dispatcher.buildMissingContainerSetAndValidate(container2BCSIDMap);
}
}
/**
- * As a part of taking snapshot with Ratis StateMachine, it will persist
- * the existing container set in the snapshotFile.
+ * As a part of taking snapshot with Ratis StateMachine,
+ * it will persist the existing container set in the snapshotFile.
+ *
* @param out OutputStream mapped to the Ratis snapshot file
- * @throws IOException
*/
public void persistContainerSet(OutputStream out) throws IOException {
- Container2BCSIDMapProto.Builder builder =
- Container2BCSIDMapProto.newBuilder();
+ Container2BCSIDMapProto.Builder builder = Container2BCSIDMapProto.newBuilder();
builder.putAllContainer2BCSID(container2BCSIDMap);
- // TODO : while snapshot is being taken, deleteContainer call should not
- // should not happen. Lock protection will be required if delete
- // container happens outside of Ratis.
+ // TODO: while snapshot is being taken, deleteContainer call should not happen.
+ // Lock protection will be required if delete container happens outside of Ratis.
builder.build().writeTo(out);
}
+ /**
+ * Checks if the state machine is currently healthy.
+ *
+ * @return {@code true} if the state machine is healthy, {@code false} otherwise.
+ */
public boolean isStateMachineHealthy() {
return stateMachineHealthy.get();
}
@@ -350,29 +349,29 @@ public long takeSnapshot() throws IOException {
TermIndex ti = getLastAppliedTermIndex();
long startTime = Time.monotonicNow();
if (!isStateMachineHealthy()) {
- String msg =
- "Failed to take snapshot " + " for " + gid + " as the stateMachine"
- + " is unhealthy. The last applied index is at " + ti;
+ String msg = "Failed to take snapshot for " + gid + " as the stateMachine is unhealthy." +
+ " The last applied index is at " + ti;
StateMachineException sme = new StateMachineException(msg);
LOG.error(msg);
throw sme;
}
if (ti != null && ti.getIndex() != RaftLog.INVALID_LOG_INDEX) {
- final File snapshotFile =
- storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
+ final File snapshotFile = storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
LOG.info("{}: Taking a snapshot at:{} file {}", gid, ti, snapshotFile);
try (FileOutputStream fos = new FileOutputStream(snapshotFile)) {
persistContainerSet(fos);
fos.flush();
- // make sure the snapshot file is synced
+ // Make sure the snapshot file is synced
fos.getFD().sync();
} catch (IOException ioe) {
- LOG.error("{}: Failed to write snapshot at:{} file {}", gid, ti,
- snapshotFile);
+ LOG.error("{}: Failed to write snapshot at:{} file {}", gid, ti, snapshotFile);
throw ioe;
}
LOG.info("{}: Finished taking a snapshot at:{} file:{} took: {} ms",
- gid, ti, snapshotFile, (Time.monotonicNow() - startTime));
+ gid,
+ ti,
+ snapshotFile,
+ (Time.monotonicNow() - startTime));
return ti.getIndex();
}
return -1;
@@ -394,13 +393,13 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftPeerRole rol
final ContainerCommandRequestProto requestProto;
if (logProto.getCmdType() == Type.WriteChunk) {
- // combine state machine data
+ // Combine state machine data
requestProto = ContainerCommandRequestProto.newBuilder(logProto)
.setWriteChunk(WriteChunkRequestProto.newBuilder(logProto.getWriteChunk())
.setData(stateMachineLogEntry.getStateMachineEntry().getStateMachineData()))
.build();
} else {
- // request and log are the same when there is no state machine data,
+ // Request and log are the same when there is no state machine data,
requestProto = logProto;
}
return trx.setStateMachineContext(new Context(requestProto, logProto));
@@ -408,11 +407,9 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftPeerRole rol
/** For the Leader to serve the given client request. */
@Override
- public TransactionContext startTransaction(RaftClientRequest request)
- throws IOException {
+ public TransactionContext startTransaction(RaftClientRequest request) throws IOException {
long startTime = Time.monotonicNowNanos();
- final ContainerCommandRequestProto proto =
- message2ContainerCommandRequestProto(request.getMessage());
+ final ContainerCommandRequestProto proto = message2ContainerCommandRequestProto(request.getMessage());
Preconditions.checkArgument(request.getRaftGroupId().equals(gid));
final TransactionContext.Builder builder = TransactionContext.newBuilder()
@@ -455,8 +452,7 @@ public TransactionContext startTransaction(RaftClientRequest request)
builder.setStateMachineData(write.getData());
}
} else if (proto.getCmdType() == Type.FinalizeBlock) {
- containerController.addFinalizedBlock(proto.getContainerID(),
- proto.getFinalizeBlock().getBlockID().getLocalID());
+ containerController.addFinalizedBlock(proto.getContainerID(), proto.getFinalizeBlock().getBlockID().getLocalID());
}
if (blockAlreadyFinalized) {
@@ -478,39 +474,35 @@ private boolean shouldRejectRequest(ContainerProtos.DatanodeBlockID blockID) {
return containerController.isFinalizedBlockExist(blockID.getContainerID(), blockID.getLocalID());
}
- private static ContainerCommandRequestProto getContainerCommandRequestProto(
- RaftGroupId id, ByteString request)
+ private static ContainerCommandRequestProto getContainerCommandRequestProto(RaftGroupId id, ByteString request)
throws InvalidProtocolBufferException {
- // TODO: We can avoid creating new builder and set pipeline Id if
- // the client is already sending the pipeline id, then we just have to
- // validate the pipeline Id.
+ // TODO: We can avoid creating new builder and set pipeline Id if the client is already sending the pipeline id,
+ // then we just have to validate the pipeline Id.
return ContainerCommandRequestProto.newBuilder(
ContainerCommandRequestProto.parseFrom(request))
.setPipelineID(id.getUuid().toString()).build();
}
- private ContainerCommandRequestProto message2ContainerCommandRequestProto(
- Message message) throws InvalidProtocolBufferException {
+ private ContainerCommandRequestProto message2ContainerCommandRequestProto(Message message)
+ throws InvalidProtocolBufferException {
return ContainerCommandRequestMessage.toProto(message.getContent(), gid);
}
private ContainerCommandResponseProto dispatchCommand(
ContainerCommandRequestProto requestProto, DispatcherContext context) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid,
- requestProto.getCmdType(), requestProto.getContainerID(),
- requestProto.getPipelineID(), requestProto.getTraceID());
- }
- ContainerCommandResponseProto response =
- dispatcher.dispatch(requestProto, context);
- if (LOG.isTraceEnabled()) {
- LOG.trace("{}: response {}", gid, response);
- }
+ LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}",
+ gid,
+ requestProto.getCmdType(),
+ requestProto.getContainerID(),
+ requestProto.getPipelineID(),
+ requestProto.getTraceID());
+ ContainerCommandResponseProto response = dispatcher.dispatch(requestProto, context);
+ LOG.trace("{}: response {}", gid, response);
return response;
}
- private CompletableFuture link(
- ContainerCommandRequestProto requestProto, LogEntryProto entry) {
+ private CompletableFuture link(ContainerCommandRequestProto requestProto,
+ LogEntryProto entry) {
return CompletableFuture.supplyAsync(() -> {
final DispatcherContext context = DispatcherContext
.newBuilder(DispatcherContext.Op.STREAM_LINK)
@@ -524,9 +516,8 @@ private CompletableFuture link(
}, executor);
}
- private CompletableFuture writeStateMachineData(
- ContainerCommandRequestProto requestProto, long entryIndex, long term,
- long startTime) {
+ private CompletableFuture writeStateMachineData(ContainerCommandRequestProto requestProto, long entryIndex,
+ long term, long startTime) {
final WriteChunkRequestProto write = requestProto.getWriteChunk();
RaftServer server = ratisServer.getServer();
Preconditions.checkArgument(!write.getData().isEmpty());
@@ -549,18 +540,19 @@ private CompletableFuture writeStateMachineData(
.setContainer2BCSIDMap(container2BCSIDMap)
.build();
CompletableFuture raftFuture = new CompletableFuture<>();
- // ensure the write chunk happens asynchronously in writeChunkExecutor pool
- // thread.
+ // Ensure the writing chunk happens asynchronously in writeChunkExecutor pool thread.
CompletableFuture writeChunkFuture =
CompletableFuture.supplyAsync(() -> {
try {
- metrics.recordWriteStateMachineQueueingLatencyNs(
- Time.monotonicNowNanos() - startTime);
+ metrics.recordWriteStateMachineQueueingLatencyNs(Time.monotonicNowNanos() - startTime);
return dispatchCommand(requestProto, context);
} catch (Exception e) {
- LOG.error("{}: writeChunk writeStateMachineData failed: blockId" +
- "{} logIndex {} chunkName {}", gid, write.getBlockID(),
- entryIndex, write.getChunkData().getChunkName(), e);
+ LOG.error("{}: writeChunk writeStateMachineData failed: blockId{} logIndex {} chunkName {}",
+ gid,
+ write.getBlockID(),
+ entryIndex,
+ write.getChunkData().getChunkName(),
+ e);
metrics.incNumWriteDataFails();
// write chunks go in parallel. It's possible that one write chunk
// see the stateMachine is marked unhealthy by other parallel thread
@@ -571,13 +563,12 @@ private CompletableFuture writeStateMachineData(
}, getChunkExecutor(requestProto.getWriteChunk()));
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
- if (LOG.isDebugEnabled()) {
- LOG.debug("{}: writeChunk writeStateMachineData : blockId" +
- "{} logIndex {} chunkName {}", gid, write.getBlockID(),
- entryIndex, write.getChunkData().getChunkName());
- }
- // Remove the future once it finishes execution from the
- // writeChunkFutureMap.
+ LOG.debug("{}: writeChunk writeStateMachineData : blockId{} logIndex {} chunkName {}",
+ gid,
+ write.getBlockID(),
+ entryIndex,
+ write.getChunkData().getChunkName());
+ // Remove the future once it finishes execution from the writeChunkFutureMap.
writeChunkFuture.thenApply(r -> {
if (r.getResult() != ContainerProtos.Result.SUCCESS
&& r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN
@@ -585,30 +576,30 @@ private CompletableFuture writeStateMachineData(
// After concurrent flushes are allowed on the same key, chunk file inconsistencies can happen and
// that should not crash the pipeline.
&& r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) {
- StorageContainerException sce =
- new StorageContainerException(r.getMessage(), r.getResult());
- LOG.error(gid + ": writeChunk writeStateMachineData failed: blockId" +
- write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
- write.getChunkData().getChunkName() + " Error message: " +
- r.getMessage() + " Container Result: " + r.getResult());
+ StorageContainerException sce = new StorageContainerException(r.getMessage(), r.getResult());
+ LOG.error("{}: writeChunk writeStateMachineData failed: blockId{} logIndex {} chunkName {} Error message: {}" +
+ " Container Result: {}",
+ gid,
+ write.getBlockID(),
+ entryIndex,
+ write.getChunkData().getChunkName(),
+ r.getMessage(),
+ r.getResult());
metrics.incNumWriteDataFails();
- // If the write fails currently we mark the stateMachine as unhealthy.
- // This leads to pipeline close. Any change in that behavior requires
- // handling the entry for the write chunk in cache.
+ // If the writing fails currently, we mark the stateMachine as unhealthy.
+ // This leads to the pipeline close.
+ // Any change in that behavior requires handling the entry for the writing chunk in cache.
stateMachineHealthy.set(false);
raftFuture.completeExceptionally(sce);
} else {
- metrics.incNumBytesWrittenCount(
- requestProto.getWriteChunk().getChunkData().getLen());
- if (LOG.isDebugEnabled()) {
- LOG.debug(gid +
- ": writeChunk writeStateMachineData completed: blockId" +
- write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
- write.getChunkData().getChunkName());
- }
+ metrics.incNumBytesWrittenCount(requestProto.getWriteChunk().getChunkData().getLen());
+ LOG.debug("{}: writeChunk writeStateMachineData completed: blockId{} logIndex {} chunkName {}",
+ gid,
+ write.getBlockID(),
+ entryIndex,
+ write.getChunkData().getChunkName());
raftFuture.complete(r::toByteString);
- metrics.recordWriteStateMachineCompletionNs(
- Time.monotonicNowNanos() - startTime);
+ metrics.recordWriteStateMachineCompletionNs(Time.monotonicNowNanos() - startTime);
}
writeChunkFutureMap.remove(entryIndex);
@@ -617,15 +608,14 @@ private CompletableFuture writeStateMachineData(
return raftFuture;
}
- private StateMachine.DataChannel getStreamDataChannel(
- ContainerCommandRequestProto requestProto,
- DispatcherContext context) throws StorageContainerException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{}: getStreamDataChannel {} containerID={} pipelineID={} " +
- "traceID={}", gid, requestProto.getCmdType(),
- requestProto.getContainerID(), requestProto.getPipelineID(),
- requestProto.getTraceID());
- }
+ private StateMachine.DataChannel getStreamDataChannel(ContainerCommandRequestProto requestProto,
+ DispatcherContext context) throws StorageContainerException {
+ LOG.debug("{}: getStreamDataChannel {} containerID={} pipelineID={} traceID={}",
+ gid,
+ requestProto.getCmdType(),
+ requestProto.getContainerID(),
+ requestProto.getPipelineID(),
+ requestProto.getTraceID());
dispatchCommand(requestProto, context); // stream init
return dispatcher.getStreamDataChannel(requestProto);
}
@@ -634,8 +624,7 @@ private StateMachine.DataChannel getStreamDataChannel(
public CompletableFuture stream(RaftClientRequest request) {
return CompletableFuture.supplyAsync(() -> {
try {
- ContainerCommandRequestProto requestProto =
- message2ContainerCommandRequestProto(request.getMessage());
+ ContainerCommandRequestProto requestProto = message2ContainerCommandRequestProto(request.getMessage());
DispatcherContext context =
DispatcherContext
.newBuilder(DispatcherContext.Op.STREAM_INIT)
@@ -643,8 +632,9 @@ public CompletableFuture stream(RaftClientRequest request) {
.setContainer2BCSIDMap(container2BCSIDMap)
.build();
DataChannel channel = getStreamDataChannel(requestProto, context);
- final ExecutorService chunkExecutor = requestProto.hasWriteChunk() ?
- getChunkExecutor(requestProto.getWriteChunk()) : null;
+ final ExecutorService chunkExecutor = requestProto.hasWriteChunk()
+ ? getChunkExecutor(requestProto.getWriteChunk())
+ : null;
return new LocalStream(channel, chunkExecutor);
} catch (IOException e) {
throw new CompletionException("Failed to create data stream", e);
@@ -655,46 +645,40 @@ public CompletableFuture stream(RaftClientRequest request) {
@Override
public CompletableFuture> link(DataStream stream, LogEntryProto entry) {
if (stream == null) {
- return JavaUtils.completeExceptionally(new IllegalStateException(
- "DataStream is null"));
+ return completeExceptionally(new IllegalStateException("DataStream is null"));
} else if (!(stream instanceof LocalStream)) {
- return JavaUtils.completeExceptionally(new IllegalStateException(
- "Unexpected DataStream " + stream.getClass()));
+ return completeExceptionally(new IllegalStateException("Unexpected DataStream " + stream.getClass()));
}
final DataChannel dataChannel = stream.getDataChannel();
if (dataChannel.isOpen()) {
- return JavaUtils.completeExceptionally(new IllegalStateException(
- "DataStream: " + stream + " is not closed properly"));
+ return completeExceptionally(new IllegalStateException("DataStream: " + stream + " is not closed properly"));
}
if (!(dataChannel instanceof KeyValueStreamDataChannel)) {
- return JavaUtils.completeExceptionally(new IllegalStateException(
- "Unexpected DataChannel " + dataChannel.getClass()));
+ return completeExceptionally(new IllegalStateException("Unexpected DataChannel " + dataChannel.getClass()));
}
- final KeyValueStreamDataChannel kvStreamDataChannel =
- (KeyValueStreamDataChannel) dataChannel;
+ final KeyValueStreamDataChannel kvStreamDataChannel = (KeyValueStreamDataChannel) dataChannel;
- final ContainerCommandRequestProto request =
- kvStreamDataChannel.getPutBlockRequest();
+ final ContainerCommandRequestProto request = kvStreamDataChannel.getPutBlockRequest();
return link(request, entry).whenComplete((response, e) -> {
if (e != null) {
- LOG.warn("Failed to link logEntry {} for request {}",
- TermIndex.valueOf(entry), request, e);
+ LOG.warn("Failed to link logEntry {} for request {}", TermIndex.valueOf(entry), request, e);
}
if (response != null) {
final ContainerProtos.Result result = response.getResult();
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} to link logEntry {} for request {}, response: {}",
- result, TermIndex.valueOf(entry), request, response);
- }
+ LOG.debug("{} to link logEntry {} for request {}, response: {}",
+ result,
+ TermIndex.valueOf(entry),
+ request,
+ response);
if (result == ContainerProtos.Result.SUCCESS) {
kvStreamDataChannel.setLinked();
return;
}
}
- // failed to link, cleanup
+ // Failed to link, cleanup
kvStreamDataChannel.cleanUp();
});
}
@@ -704,9 +688,18 @@ private ExecutorService getChunkExecutor(WriteChunkRequestProto req) {
return chunkExecutors.get(i);
}
- /*
- * writeStateMachineData calls are not synchronized with each other
- * and also with applyTransaction.
+ /**
+ * Processes a log entry to write data to the state machine.
+ * Incidents of the number of write state machine operations are recorded.
+ * {@link #writeStateMachineData(ContainerCommandRequestProto, long, long, long)}
+ * calls are not synchronized with each other and also with {@link #applyTransaction(TransactionContext)}.
+ *
+ * @param entry The log entry to be written.
+ * @param trx The transaction context associated with this write operation.
+ * @return A CompletableFuture representing the asynchronous operation of writing the data,
+ * which will complete with the resulting message or exceptionally if an error occurs.
+ * @throws NullPointerException if the transaction context is null.
+ * @throws IllegalStateException if the command type is not supported for state machine data.
*/
@Override
public CompletableFuture write(LogEntryProto entry, TransactionContext trx) {
@@ -720,14 +713,10 @@ public CompletableFuture write(LogEntryProto entry, TransactionContext
// For only writeChunk, there will be writeStateMachineData call.
// CreateContainer will happen as a part of writeChunk only.
- switch (cmdType) {
- case WriteChunk:
- return writeStateMachineData(requestProto, entry.getIndex(),
- entry.getTerm(), writeStateMachineStartTime);
- default:
- throw new IllegalStateException("Cmd Type:" + cmdType
- + " should not have state machine data");
+ if (Objects.requireNonNull(cmdType) == Type.WriteChunk) {
+ return writeStateMachineData(requestProto, entry.getIndex(), entry.getTerm(), writeStateMachineStartTime);
}
+ throw new IllegalStateException("Cmd Type:" + cmdType + " should not have state machine data");
} catch (Exception e) {
metrics.incNumWriteStateMachineFails();
return completeExceptionally(e);
@@ -738,26 +727,21 @@ public CompletableFuture write(LogEntryProto entry, TransactionContext
public CompletableFuture query(Message request) {
try {
metrics.incNumQueryStateMachineOps();
- final ContainerCommandRequestProto requestProto =
- message2ContainerCommandRequestProto(request);
- return CompletableFuture.completedFuture(
- dispatchCommand(requestProto, null)::toByteString);
+ final ContainerCommandRequestProto requestProto = message2ContainerCommandRequestProto(request);
+ return CompletableFuture.completedFuture(dispatchCommand(requestProto, null)::toByteString);
} catch (IOException e) {
metrics.incNumQueryStateMachineFails();
return completeExceptionally(e);
}
}
- private ByteString readStateMachineData(
- ContainerCommandRequestProto requestProto, long term, long index)
+ private ByteString readStateMachineData(ContainerCommandRequestProto requestProto, long term, long index)
throws IOException {
- // the stateMachine data is not present in the stateMachine cache,
- // increment the stateMachine cache miss count
+ // The stateMachine data is not present in the stateMachine cache, increment the stateMachine cache miss count
metrics.incNumReadStateMachineMissCount();
- WriteChunkRequestProto writeChunkRequestProto =
- requestProto.getWriteChunk();
+ WriteChunkRequestProto writeChunkRequestProto = requestProto.getWriteChunk();
ContainerProtos.ChunkInfo chunkInfo = writeChunkRequestProto.getChunkData();
- // prepare the chunk to be read
+ // Prepare the chunk to be read
ReadChunkRequestProto.Builder readChunkRequestProto =
ReadChunkRequestProto.newBuilder()
.setBlockID(writeChunkRequestProto.getBlockID())
@@ -772,16 +756,16 @@ private ByteString readStateMachineData(
.setTerm(term)
.setLogIndex(index)
.build();
- // read the chunk
- ContainerCommandResponseProto response =
- dispatchCommand(dataContainerCommandProto, context);
+ // Read the chunk
+ ContainerCommandResponseProto response = dispatchCommand(dataContainerCommandProto, context);
if (response.getResult() != ContainerProtos.Result.SUCCESS) {
- StorageContainerException sce =
- new StorageContainerException(response.getMessage(),
- response.getResult());
- LOG.error("gid {} : ReadStateMachine failed. cmd {} logIndex {} msg : "
- + "{} Container Result: {}", gid, response.getCmdType(), index,
- response.getMessage(), response.getResult());
+ StorageContainerException sce = new StorageContainerException(response.getMessage(), response.getResult());
+ LOG.error("gid {} : ReadStateMachine failed. cmd {} logIndex {} msg : {} Container Result: {}",
+ gid,
+ response.getCmdType(),
+ index,
+ response.getMessage(),
+ response.getResult());
stateMachineHealthy.set(false);
throw sce;
}
@@ -791,45 +775,42 @@ private ByteString readStateMachineData(
if (responseProto.hasData()) {
data = responseProto.getData();
} else {
- data = BufferUtils.concatByteStrings(
- responseProto.getDataBuffers().getBuffersList());
+ data = BufferUtils.concatByteStrings(responseProto.getDataBuffers().getBuffersList());
}
- // assert that the response has data in it.
- Preconditions
- .checkNotNull(data, "read chunk data is null for chunk: %s",
- chunkInfo);
+ // Assert that the response has data in it.
+ Preconditions.checkNotNull(data, "read chunk data is null for chunk: %s", chunkInfo);
Preconditions.checkState(data.size() == chunkInfo.getLen(),
"read chunk len=%s does not match chunk expected len=%s for chunk:%s",
- data.size(), chunkInfo.getLen(), chunkInfo);
+ data.size(),
+ chunkInfo.getLen(),
+ chunkInfo);
return data;
}
/**
- * Returns the combined future of all the writeChunks till the given log
- * index. The Raft log worker will wait for the stateMachineData to complete
- * flush as well.
+ * Returns the combined future of all the writeChunks till the given log index.
+ * The Raft log worker will wait for the stateMachineData to complete flush as well.
*
* @param index log index till which the stateMachine data needs to be flushed
* @return Combined future of all writeChunks till the log index given.
*/
@Override
public CompletableFuture flush(long index) {
- List> futureList =
- writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index)
- .map(Map.Entry::getValue).collect(Collectors.toList());
- return CompletableFuture.allOf(
- futureList.toArray(new CompletableFuture[futureList.size()]));
+ return CompletableFuture.allOf(writeChunkFutureMap.entrySet().stream()
+ .filter(x -> x.getKey() <= index)
+ .map(Map.Entry::getValue)
+ .toArray(CompletableFuture[]::new));
}
/**
- * This method is used by the Leader to read state machine date for sending appendEntries to followers.
+ * The Leader uses this method to read state machine date for sending appendEntries to followers.
* It will first get the data from {@link #stateMachineDataCache}.
* If the data is not in the cache, it will read from the file by dispatching a command
*
* @param trx the transaction context,
- * which can be null if this method is invoked after {@link #applyTransaction(TransactionContext)}.
+ * which can be null if this method is invoked after {@link #applyTransaction(TransactionContext)}.
*/
@Override
public CompletableFuture read(LogEntryProto entry, TransactionContext trx) {
@@ -856,7 +837,8 @@ public CompletableFuture read(LogEntryProto entry, TransactionContex
final Context context = (Context) Optional.ofNullable(trx)
.map(TransactionContext::getStateMachineContext)
.orElse(null);
- final ContainerCommandRequestProto requestProto = context != null ? context.getLogProto()
+ final ContainerCommandRequestProto requestProto = context != null
+ ? context.getLogProto()
: getContainerCommandRequestProto(gid, entry.getStateMachineLogEntry().getLogData());
if (requestProto.getCmdType() != Type.WriteChunk) {
@@ -897,33 +879,32 @@ private synchronized void updateLastApplied() {
}
/**
- * Notifies the state machine about index updates because of entries
- * which do not cause state machine update, i.e. conf entries, metadata
- * entries
+ * Notifies the state machine about index updates because of entries which do not cause state machine update,
+ * i.e., conf entries, metadata entries.
+ *
* @param term term of the log entry
* @param index index of the log entry
*/
@Override
public void notifyTermIndexUpdated(long term, long index) {
applyTransactionCompletionMap.put(index, term);
- // We need to call updateLastApplied here because now in ratis when a
- // node becomes leader, it is checking stateMachineIndex >=
- // placeHolderIndex (when a node becomes leader, it writes a conf entry
- // with some information like its peers and termIndex). So, calling
- // updateLastApplied updates lastAppliedTermIndex.
+ // We need to call updateLastApplied here because now in ratis when a node becomes leader,
+ // it is checking stateMachineIndex >= placeHolderIndex
+ // (when a node becomes leader, it writes a conf entry with some information like its peers and termIndex).
+ // So, calling updateLastApplied updates lastAppliedTermIndex.
updateLastApplied();
removeStateMachineDataIfNeeded(index);
}
@Override
public void notifyServerShutdown(RaftProtos.RoleInfoProto roleInfo, boolean allServer) {
- // if datanodeService is stopped , it indicates this `close` originates
- // from `HddsDatanodeService.stop()`, otherwise, it indicates this `close` originates from ratis.
+ // If datanodeService is stopped, it indicates this `close` originates from `HddsDatanodeService.stop()`,
+ // otherwise, it indicates this `close` originates from ratis.
if (allServer) {
if (datanodeService != null && !datanodeService.isStopped()) {
LOG.info("{} is closed by ratis", gid);
- if (semaphore.tryAcquire()) {
- // run with a different thread, so this raft group can be closed
+ if (SEMAPHORE.tryAcquire()) {
+ // Run with a different thread, so this raft group can be closed
Runnable runnable = () -> {
try {
int closed = 0, total = 0;
@@ -932,9 +913,7 @@ public void notifyServerShutdown(RaftProtos.RoleInfoProto roleInfo, boolean allS
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
- Iterator iterator = ratisServer.getServer().getGroupIds().iterator();
- while (iterator.hasNext()) {
- RaftGroupId id = iterator.next();
+ for (RaftGroupId id : ratisServer.getServer().getGroupIds()) {
RaftServer.Division division = ratisServer.getServer().getDivision(id);
if (division.getRaftServer().getLifeCycleState() == LifeCycle.State.CLOSED) {
closed++;
@@ -959,37 +938,38 @@ public void notifyServerShutdown(RaftProtos.RoleInfoProto roleInfo, boolean allS
}
private CompletableFuture applyTransaction(
- ContainerCommandRequestProto request, DispatcherContext context,
- Consumer exceptionHandler) {
+ ContainerCommandRequestProto request, DispatcherContext context, Consumer exceptionHandler) {
final long containerId = request.getContainerID();
- final CheckedSupplier task
- = () -> {
- try {
- long timeNow = Time.monotonicNowNanos();
- long queueingDelay = timeNow - context.getStartTime();
- metrics.recordQueueingDelay(request.getCmdType(), queueingDelay);
- // TODO: add a counter to track number of executing applyTransaction
- // and queue size
- return dispatchCommand(request, context);
- } catch (Exception e) {
- exceptionHandler.accept(e);
- throw e;
- }
- };
+ final CheckedSupplier task = () -> {
+ try {
+ long timeNow = Time.monotonicNowNanos();
+ long queueingDelay = timeNow - context.getStartTime();
+ metrics.recordQueueingDelay(request.getCmdType(), queueingDelay);
+ // TODO: add a counter to track number of executing applyTransaction
+ // and queue size
+ return dispatchCommand(request, context);
+ } catch (Exception e) {
+ exceptionHandler.accept(e);
+ throw e;
+ }
+ };
return containerTaskQueues.submit(containerId, task, executor);
}
- // Removes the stateMachine data from cache once both followers catch up
- // to the particular index.
+ /**
+ * Removes state machine data from the cache if certain conditions are met.
+ * This method will remove data corresponding to the given log index up to a certain minimum index
+ * if the current division is the leader and waitOnBothFollowers is set to true.
+ *
+ * @param index the log index up to which data may be removed from the cache
+ */
private void removeStateMachineDataIfNeeded(long index) {
if (waitOnBothFollowers) {
try {
RaftServer.Division division = ratisServer.getServer().getDivision(gid);
if (division.getInfo().isLeader()) {
- long minIndex = Arrays.stream(division.getInfo()
- .getFollowerNextIndices()).min().getAsLong();
- LOG.debug("Removing data corresponding to log index {} min index {} "
- + "from cache", index, minIndex);
+ long minIndex = Arrays.stream(division.getInfo().getFollowerNextIndices()).min().getAsLong();
+ LOG.debug("Removing data corresponding to log index {} min index {} from cache", index, minIndex);
removeCacheDataUpTo(Math.min(minIndex, index));
}
} catch (Exception e) {
@@ -998,20 +978,23 @@ private void removeStateMachineDataIfNeeded(long index) {
}
}
- /*
- * ApplyTransaction calls in Ratis are sequential.
+ /**
+ * Applies a transaction to the state machine.
+ *
+ * @param trx The transaction context containing information about the transaction to be applied.
+ * @return A CompletableFuture that will complete with the resultant message once the transaction is applied.
*/
@Override
public CompletableFuture applyTransaction(TransactionContext trx) {
long index = trx.getLogEntry().getIndex();
try {
- // Remove the stateMachine data once both followers have caught up. If any
- // one of the follower is behind, the pending queue will max out as
- // configurable limit on pending request size and count and then will
- // block and client will backoff as a result of that.
+ // Remove the stateMachine data once both followers have caught up.
+ // If any one of the followers is behind,
+ // the pending queue will max out as configurable limit on pending request size and count
+ // and then will block and client will backoff as a result of that.
removeStateMachineDataIfNeeded(index);
- // if waitOnBothFollower is false, remove the entry from the cache
- // as soon as its applied and such entry exists in the cache.
+ // If waitOnBothFollower is false,
+ // remove the entry from the cache as soon as its applied and such entry exists in the cache.
removeStateMachineDataIfMajorityFollowSync(index);
final DispatcherContext.Builder builder = DispatcherContext
.newBuilder(DispatcherContext.Op.APPLY_TRANSACTION)
@@ -1024,14 +1007,12 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
applyTransactionSemaphore.acquire();
metrics.incNumApplyTransactionsOps();
-
Objects.requireNonNull(context, "context == null");
final ContainerCommandRequestProto requestProto = context.getLogProto();
final Type cmdType = requestProto.getCmdType();
- // Make sure that in write chunk, the user data is not set
+ // Make sure that in writes chunk, the user data is not set
if (cmdType == Type.WriteChunk) {
- Preconditions
- .checkArgument(requestProto.getWriteChunk().getData().isEmpty());
+ Preconditions.checkArgument(requestProto.getWriteChunk().getData().isEmpty());
builder.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA);
}
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile
@@ -1039,65 +1020,58 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
|| cmdType == Type.StreamInit) {
builder.setContainer2BCSIDMap(container2BCSIDMap);
}
- CompletableFuture applyTransactionFuture =
- new CompletableFuture<>();
+ CompletableFuture applyTransactionFuture = new CompletableFuture<>();
final Consumer exceptionHandler = e -> {
- LOG.error(gid + ": failed to applyTransaction at logIndex " + index
- + " for " + requestProto.getCmdType(), e);
+ LOG.error("{}: failed to applyTransaction at logIndex {} for {}", gid, index, requestProto.getCmdType(), e);
stateMachineHealthy.compareAndSet(true, false);
metrics.incNumApplyTransactionsFails();
applyTransactionFuture.completeExceptionally(e);
};
- // Ensure the command gets executed in a separate thread than
- // stateMachineUpdater thread which is calling applyTransaction here.
+ // Ensure the command gets executed in a separate thread than stateMachineUpdater thread
+ // which is calling applyTransaction here.
final CompletableFuture future =
applyTransaction(requestProto, builder.build(), exceptionHandler);
future.thenApply(r -> {
// TODO: add metrics for non-leader case
if (trx.getServerRole() == RaftPeerRole.LEADER) {
final long startTime = context.getStartTime();
- metrics.incPipelineLatencyMs(cmdType,
- (Time.monotonicNowNanos() - startTime) / 1000000L);
+ metrics.incPipelineLatencyMs(cmdType, (Time.monotonicNowNanos() - startTime) / 1000000L);
}
- // ignore close container exception while marking the stateMachine
- // unhealthy
+ // Ignore close container exception while marking the stateMachine unhealthy
if (r.getResult() != ContainerProtos.Result.SUCCESS
&& r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN
&& r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO
&& r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) {
- StorageContainerException sce =
- new StorageContainerException(r.getMessage(), r.getResult());
- LOG.error(
- "gid {} : ApplyTransaction failed. cmd {} logIndex {} msg : "
- + "{} Container Result: {}", gid, r.getCmdType(), index,
- r.getMessage(), r.getResult());
+ StorageContainerException sce = new StorageContainerException(r.getMessage(), r.getResult());
+ LOG.error("gid {} : ApplyTransaction failed. cmd {} logIndex {} msg : {} Container Result: {}",
+ gid,
+ r.getCmdType(),
+ index,
+ r.getMessage(),
+ r.getResult());
metrics.incNumApplyTransactionsFails();
// Since the applyTransaction now is completed exceptionally,
- // before any further snapshot is taken , the exception will be
- // caught in stateMachineUpdater in Ratis and ratis server will
- // shutdown.
+ // before any further snapshot is taken,
+ // the exception will be caught in stateMachineUpdater in Ratis and ratis server will shut down.
applyTransactionFuture.completeExceptionally(sce);
stateMachineHealthy.compareAndSet(true, false);
ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole());
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
- + "{} Container Result: {}", gid, r.getCmdType(), index,
- r.getMessage(), r.getResult());
- }
+ LOG.debug("gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : {} Container Result: {}",
+ gid,
+ r.getCmdType(),
+ index,
+ r.getMessage(),
+ r.getResult());
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
- metrics.incNumBytesCommittedCount(
- requestProto.getWriteChunk().getChunkData().getLen());
+ metrics.incNumBytesCommittedCount(requestProto.getWriteChunk().getChunkData().getLen());
}
applyTransactionFuture.complete(r::toByteString);
- // add the entry to the applyTransactionCompletionMap only if the
- // stateMachine is healthy i.e, there has been no applyTransaction
- // failures before.
+ // Add the entry to the applyTransactionCompletionMap only if the stateMachine is healthy,
+ // i.e., there have been no applyTransaction failures before.
if (isStateMachineHealthy()) {
- final Long previous = applyTransactionCompletionMap
- .put(index, trx.getLogEntry().getTerm());
+ final Long previous = applyTransactionCompletionMap.put(index, trx.getLogEntry().getTerm());
Preconditions.checkState(previous == null);
updateLastApplied();
}
@@ -1108,8 +1082,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
exceptionHandler.accept(t);
}
applyTransactionSemaphore.release();
- metrics.recordApplyTransactionCompletionNs(
- Time.monotonicNowNanos() - applyTxnStartTime);
+ metrics.recordApplyTransactionCompletionNs(Time.monotonicNowNanos() - applyTxnStartTime);
if (trx.getServerRole() == RaftPeerRole.LEADER) {
metrics.decPendingApplyTransactions();
}
@@ -1127,8 +1100,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
private void removeStateMachineDataIfMajorityFollowSync(long index) {
if (!waitOnBothFollowers) {
- // if majority follow in sync, remove all cache previous to current index
- // including current index
+ // If the majority follow in sync, remove all caches previous to current index including current index
removeCacheDataUpTo(index);
}
}
@@ -1137,15 +1109,9 @@ private void removeCacheDataUpTo(long index) {
stateMachineDataCache.removeIf(k -> k <= index);
}
- private static CompletableFuture completeExceptionally(Exception e) {
- final CompletableFuture future = new CompletableFuture<>();
- future.completeExceptionally(e);
- return future;
- }
-
@Override
public void notifyNotLeader(Collection pendingEntries) {
- // once the leader steps down , clear the cache
+ // Once the leader steps down, clear the cache
evictStateMachineCache();
}
@@ -1155,6 +1121,11 @@ public CompletableFuture truncate(long index) {
return CompletableFuture.completedFuture(null);
}
+ /**
+ * Evicts the state machine cache by clearing all cached data.
+ * This method is primarily used for testing and is typically invoked
+ * when there is a need to reset or invalidate the current state of the cache.
+ */
@VisibleForTesting
public void evictStateMachineCache() {
stateMachineDataCache.clear();
@@ -1172,16 +1143,18 @@ public void notifyExtendedNoLeader(RoleInfoProto roleInfoProto) {
@Override
public void notifyLogFailed(Throwable t, LogEntryProto failedEntry) {
- LOG.error("{}: {} {}", gid, TermIndex.valueOf(failedEntry),
- toStateMachineLogEntryString(failedEntry.getStateMachineLogEntry()), t);
+ LOG.error("{}: {} {}",
+ gid,
+ TermIndex.valueOf(failedEntry),
+ toStateMachineLogEntryString(failedEntry.getStateMachineLogEntry()),
+ t);
ratisServer.handleNodeLogFailure(gid, t);
}
@Override
- public CompletableFuture notifyInstallSnapshotFromLeader(
- RoleInfoProto roleInfoProto, TermIndex firstTermIndexInLog) {
- ratisServer.handleInstallSnapshotFromLeader(gid, roleInfoProto,
- firstTermIndexInLog);
+ public CompletableFuture notifyInstallSnapshotFromLeader(RoleInfoProto roleInfoProto,
+ TermIndex firstTermIndexInLog) {
+ ratisServer.handleInstallSnapshotFromLeader(gid, roleInfoProto, firstTermIndexInLog);
final CompletableFuture future = new CompletableFuture<>();
future.complete(firstTermIndexInLog);
return future;
@@ -1190,14 +1163,12 @@ public CompletableFuture notifyInstallSnapshotFromLeader(
@Override
public void notifyGroupRemove() {
ratisServer.notifyGroupRemove(gid);
- // Make best effort to quasi-close all the containers on group removal.
- // Containers already in terminal state like CLOSED or UNHEALTHY will not
- // be affected.
+ // Make the best effort to quasi-close all the containers on group removal.
+ // Containers already in terminal state like CLOSED or UNHEALTHY will not be affected.
for (Long cid : container2BCSIDMap.keySet()) {
try {
containerController.markContainerForClose(cid);
- containerController.quasiCloseContainer(cid,
- "Ratis group removed");
+ containerController.quasiCloseContainer(cid, "Ratis group removed. Group id: " + gid);
} catch (IOException e) {
LOG.debug("Failed to quasi-close container {}", cid);
}
@@ -1212,8 +1183,7 @@ public void close() {
}
@Override
- public void notifyLeaderChanged(RaftGroupMemberId groupMemberId,
- RaftPeerId raftPeerId) {
+ public void notifyLeaderChanged(RaftGroupMemberId groupMemberId, RaftPeerId raftPeerId) {
ratisServer.handleLeaderChangedNotification(groupMemberId, raftPeerId);
}
@@ -1222,13 +1192,19 @@ public String toStateMachineLogEntryString(StateMachineLogEntryProto proto) {
return smProtoToString(gid, containerController, proto);
}
- public static String smProtoToString(RaftGroupId gid,
- ContainerController containerController,
- StateMachineLogEntryProto proto) {
+ /**
+ * Converts the given {@link StateMachineLogEntryProto} to a string representation.
+ *
+ * @param gid the Raft group ID associated with the state machine entry.
+ * @param containerController the controller used to manage container operations, can be null.
+ * @param proto the StateMachineLogEntryProto instance to be converted to string.
+ * @return a string representation of the state machine log entry.
+ */
+ public static String smProtoToString(RaftGroupId gid, ContainerController containerController,
+ StateMachineLogEntryProto proto) {
StringBuilder builder = new StringBuilder();
try {
- ContainerCommandRequestProto requestProto =
- getContainerCommandRequestProto(gid, proto.getLogData());
+ ContainerCommandRequestProto requestProto = getContainerCommandRequestProto(gid, proto.getLogData());
long contId = requestProto.getContainerID();
builder.append(TextFormat.shortDebugString(requestProto));
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java
index 6d32f3a3f3e..a50a125f6d4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java
@@ -46,6 +46,16 @@ public ECReconstructionCoordinatorTask(
debugString = reconstructionCommandInfo.toString();
}
+ @Override
+ public String getMetricName() {
+ return "ECReconstructions";
+ }
+
+ @Override
+ public String getMetricDescriptionSegment() {
+ return "EC reconstructions";
+ }
+
@Override
public void runTask() {
// Implement the coordinator logic to handle a container group
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 98d81c15d0a..ed9c4a7ede0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -25,7 +25,6 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
-import java.nio.file.StandardCopyOption;
import java.time.Instant;
import java.util.Collections;
import java.util.HashSet;
@@ -68,6 +67,8 @@
import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_ALREADY_EXISTS;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_FILES_CREATE_ERROR;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_INTERNAL_ERROR;
@@ -84,122 +85,113 @@
import org.slf4j.LoggerFactory;
/**
- * Class to perform KeyValue Container operations. Any modifications to
- * KeyValueContainer object should ideally be done via api exposed in
- * KeyValueHandler class.
+ * Class to perform KeyValue Container operations.
+ * Any modifications to KeyValueContainer object should ideally be done via api exposed in KeyValueHandler class.
*/
public class KeyValueContainer implements Container {
- private static final Logger LOG =
- LoggerFactory.getLogger(KeyValueContainer.class);
+ private static final Logger LOG = LoggerFactory.getLogger(KeyValueContainer.class);
- // Use a non-fair RW lock for better throughput, we may revisit this decision
- // if this causes fairness issues.
+ // Use a non-fair RW lock for better throughput, we may revisit this decision if this causes fairness issues.
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
// Simple lock to synchronize container metadata dump operation.
private final Object dumpLock = new Object();
private final KeyValueContainerData containerData;
- private ConfigurationSource config;
-
- // Cache of Blocks (LocalIDs) awaiting final PutBlock call after the stream
- // is closed. When a block is added to the DB as part of putBlock, it is
- // added to the cache here. It is cleared from the Cache when the putBlock
- // is called on the block as part of stream.close() (with endOfBlock = true
- // in BlockManagerImpl#putBlock). Or when the container is marked for
- // close, the whole cache is cleared as there can be no more writes to this
- // container.
- // We do not need to explicitly synchronize this cache as the writes to
- // container are synchronous.
+ private final ConfigurationSource config;
+
+ // Cache of Blocks (LocalIDs) awaiting final PutBlock call after the stream is closed.
+ // When a block is added to the DB as part of putBlock, it is added to the cache here.
+ // It is cleared from the Cache when the putBlock is called on the block as part of stream.close()
+ // (with endOfBlock = true in BlockManagerImpl#putBlock).
+ // Or when the container is marked for close,
+ // the whole cache is cleared as there can be no more writes to this container.
+ // We do not need to explicitly synchronize this cache as the writings to container are synchronous.
private Set pendingPutBlockCache;
private boolean bCheckChunksFilePath;
- public KeyValueContainer(KeyValueContainerData containerData,
- ConfigurationSource ozoneConfig) {
- Preconditions.checkNotNull(containerData,
- "KeyValueContainerData cannot be null");
- Preconditions.checkNotNull(ozoneConfig,
- "Ozone configuration cannot be null");
+ public KeyValueContainer(KeyValueContainerData containerData, ConfigurationSource ozoneConfig) {
+ Preconditions.checkNotNull(containerData, "KeyValueContainerData cannot be null");
+ Preconditions.checkNotNull(ozoneConfig, "Ozone configuration cannot be null");
this.config = ozoneConfig;
this.containerData = containerData;
if (this.containerData.isOpen() || this.containerData.isClosing()) {
- // If container is not in OPEN or CLOSING state, there cannot be block
- // writes to the container. So pendingPutBlockCache is not needed.
+ // If the container is not in OPEN or CLOSING state, there cannot be block writes to the container.
+ // So pendingPutBlockCache is unnecessary.
this.pendingPutBlockCache = new HashSet<>();
} else {
this.pendingPutBlockCache = Collections.emptySet();
}
- DatanodeConfiguration dnConf =
- config.getObject(DatanodeConfiguration.class);
+ DatanodeConfiguration dnConf = config.getObject(DatanodeConfiguration.class);
bCheckChunksFilePath = dnConf.getCheckEmptyContainerDir();
}
+ /**
+ * Sets the flag to check the chunk directory file path.
+ *
+ * @param bCheckChunksDirFilePath the flag indicating whether to check the chunks directory file path
+ */
@VisibleForTesting
public void setCheckChunksFilePath(boolean bCheckChunksDirFilePath) {
this.bCheckChunksFilePath = bCheckChunksDirFilePath;
}
@Override
- public void create(VolumeSet volumeSet, VolumeChoosingPolicy
- volumeChoosingPolicy, String clusterId) throws StorageContainerException {
- Preconditions.checkNotNull(volumeChoosingPolicy, "VolumeChoosingPolicy " +
- "cannot be null");
+ public void create(VolumeSet volumeSet, VolumeChoosingPolicy volumeChoosingPolicy, String clusterId)
+ throws StorageContainerException {
+ Preconditions.checkNotNull(volumeChoosingPolicy, "VolumeChoosingPolicy cannot be null");
Preconditions.checkNotNull(volumeSet, "VolumeSet cannot be null");
Preconditions.checkNotNull(clusterId, "clusterId cannot be null");
File containerMetaDataPath = null;
- //acquiring volumeset read lock
+ // Acquiring volumeset read lock
long maxSize = containerData.getMaxSize();
volumeSet.readLock();
try {
- List volumes
- = StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList());
+ List volumes = StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList());
while (true) {
HddsVolume containerVolume;
try {
containerVolume = volumeChoosingPolicy.chooseVolume(volumes, maxSize);
} catch (DiskOutOfSpaceException ex) {
- throw new StorageContainerException("Container creation failed, " +
- "due to disk out of space", ex, DISK_OUT_OF_SPACE);
+ throw new StorageContainerException(
+ "Container creation failed, due to disk out of space",
+ ex,
+ DISK_OUT_OF_SPACE);
} catch (IOException ex) {
throw new StorageContainerException(
- "Container creation failed. " + ex.getMessage(), ex,
+ "Container creation failed. " + ex.getMessage(),
+ ex,
CONTAINER_INTERNAL_ERROR);
}
try {
String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
- // Set volume before getContainerDBFile(), because we may need the
- // volume to deduce the db file.
+ // Set volume before getContainerDBFile(), because we may need the volume to deduce the db file.
containerData.setVolume(containerVolume);
long containerID = containerData.getContainerID();
- String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
- containerVolume, clusterId);
- // Set schemaVersion before the dbFile since we have to
- // choose the dbFile location based on schema version.
- String schemaVersion = VersionedDatanodeFeatures.SchemaV3
- .chooseSchemaVersion(config);
+ String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(containerVolume, clusterId);
+ // Set schemaVersion before the dbFile since we have to choose the dbFile location based on a schema version.
+ String schemaVersion = VersionedDatanodeFeatures.SchemaV3.chooseSchemaVersion(config);
containerData.setSchemaVersion(schemaVersion);
- containerMetaDataPath = KeyValueContainerLocationUtil
- .getContainerMetaDataPath(hddsVolumeDir, idDir, containerID);
+ containerMetaDataPath =
+ KeyValueContainerLocationUtil.getContainerMetaDataPath(hddsVolumeDir, idDir, containerID);
containerData.setMetadataPath(containerMetaDataPath.getPath());
- File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
- hddsVolumeDir, idDir, containerID);
+ File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(hddsVolumeDir, idDir, containerID);
- // Check if it is new Container.
+ // Check if it is a new Container.
ContainerUtils.verifyIsNewContainer(containerMetaDataPath);
- //Create Metadata path chunks path and metadata db
+ // Create Metadata path chunks path and metadata db
File dbFile = getContainerDBFile();
- createContainerMetaData(containerMetaDataPath, chunksPath, dbFile,
- containerData.getSchemaVersion(), config);
+ createContainerMetaData(containerMetaDataPath, chunksPath, dbFile, containerData.getSchemaVersion(), config);
- //Set containerData for the KeyValueContainer.
+ // Set containerData for the KeyValueContainer.
containerData.setChunksPath(chunksPath.getPath());
containerData.setDbFile(dbFile);
@@ -209,32 +201,34 @@ public void create(VolumeSet volumeSet, VolumeChoosingPolicy
return;
} catch (StorageContainerException ex) {
- if (containerMetaDataPath != null
- && containerMetaDataPath.getParentFile().exists()) {
+ if (containerMetaDataPath != null && containerMetaDataPath.getParentFile().exists()) {
FileUtil.fullyDelete(containerMetaDataPath.getParentFile());
}
throw ex;
} catch (FileAlreadyExistsException ex) {
- throw new StorageContainerException("Container creation failed " +
- "because ContainerFile already exists", ex,
+ throw new StorageContainerException(
+ "Container creation failed because ContainerFile already exists",
+ ex,
CONTAINER_ALREADY_EXISTS);
} catch (IOException ex) {
- // This is a general catch all - no space left of device, which should
- // not happen as the volume Choosing policy should filter out full
- // disks, but it may still be possible if the disk quickly fills,
- // or some IO error on the disk etc. In this case we try again with a
- // different volume if there are any left to try.
- if (containerMetaDataPath != null &&
- containerMetaDataPath.getParentFile().exists()) {
+ // This is a general catch-all - no space left of device,
+ // which should not happen as the volume choosing policy should filter out full disks,
+ // but it may still be possible if the disk quickly fills, or some IO error on the disk etc.
+ // In this case we try again with a different volume if there are any left to try.
+ if (containerMetaDataPath != null && containerMetaDataPath.getParentFile().exists()) {
FileUtil.fullyDelete(containerMetaDataPath.getParentFile());
}
volumes.remove(containerVolume);
- LOG.error("Exception attempting to create container {} on volume {}" +
- " remaining volumes to try {}", containerData.getContainerID(),
- containerVolume.getHddsRootDir(), volumes.size(), ex);
- if (volumes.size() == 0) {
+ LOG.error(
+ "Exception attempting to create container {} on volume {} remaining volumes to try {}",
+ containerData.getContainerID(),
+ containerVolume.getHddsRootDir(),
+ volumes.size(),
+ ex);
+ if (volumes.isEmpty()) {
throw new StorageContainerException(
- "Container creation failed. " + ex.getMessage(), ex,
+ "Container creation failed. " + ex.getMessage(),
+ ex,
CONTAINER_INTERNAL_ERROR);
}
}
@@ -244,28 +238,27 @@ public void create(VolumeSet volumeSet, VolumeChoosingPolicy
}
}
-
/**
- * The Static method call is wrapped in a protected instance method so it can
- * be overridden in tests.
+ * The Static method call is wrapped in a protected instance method so it can be overridden in tests.
*/
@VisibleForTesting
- protected void createContainerMetaData(File containerMetaDataPath,
- File chunksPath, File dbFile, String schemaVersion,
+ protected void createContainerMetaData(File containerMetaDataPath, File chunksPath, File dbFile, String schemaVersion,
ConfigurationSource configuration) throws IOException {
- KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
- chunksPath, dbFile, schemaVersion, configuration);
+ KeyValueContainerUtil.createContainerMetaData(
+ containerMetaDataPath,
+ chunksPath,
+ dbFile,
+ schemaVersion,
+ configuration);
}
/**
- * Set all of the path realted container data fields based on the name
- * conventions.
+ * Set all the path related container data fields based on the name conventions.
*
- * @param clusterId
- * @param containerVolume
+ * @param clusterId the ID of the cluster
+ * @param containerVolume the volume of the container
*/
- public void populatePathFields(String clusterId,
- HddsVolume containerVolume) {
+ public void populatePathFields(String clusterId, HddsVolume containerVolume) {
long containerId = containerData.getContainerID();
String hddsVolumeDir = containerVolume.getHddsRootDir().getAbsolutePath();
@@ -273,10 +266,9 @@ public void populatePathFields(String clusterId,
File containerMetaDataPath = KeyValueContainerLocationUtil
.getContainerMetaDataPath(hddsVolumeDir, clusterId, containerId);
- File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
- hddsVolumeDir, clusterId, containerId);
+ File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(hddsVolumeDir, clusterId, containerId);
- //Set containerData for the KeyValueContainer.
+ // Set containerData for the KeyValueContainer.
containerData.setMetadataPath(containerMetaDataPath.getPath());
containerData.setChunksPath(chunksPath.getPath());
containerData.setVolume(containerVolume);
@@ -287,78 +279,66 @@ public void populatePathFields(String clusterId,
* Writes to .container file.
*
* @param containerFile container file name
- * @param isCreate True if creating a new file. False is updating an
- * existing container file.
- * @throws StorageContainerException
+ * @param isCreate True if creating a new file. False is updating an existing container file.
+ * @throws StorageContainerException If an error occurs while writing to the container file.
*/
- private void writeToContainerFile(File containerFile, boolean isCreate)
- throws StorageContainerException {
+ private void writeToContainerFile(File containerFile, boolean isCreate) throws StorageContainerException {
File tempContainerFile = null;
long containerId = containerData.getContainerID();
try {
tempContainerFile = createTempFile(containerFile);
- ContainerDataYaml.createContainerFile(
- ContainerType.KeyValueContainer, containerData, tempContainerFile);
+ ContainerDataYaml.createContainerFile(ContainerType.KeyValueContainer, containerData, tempContainerFile);
- // NativeIO.renameTo is an atomic function. But it might fail if the
- // container file already exists. Hence, we handle the two cases
- // separately.
+ // NativeIO.renameTo is an atomic function.
+ // But it might fail if the container file already exists.
+ // Hence, we handle the two cases separately.
if (isCreate) {
NativeIO.renameTo(tempContainerFile, containerFile);
} else {
- Files.move(tempContainerFile.toPath(), containerFile.toPath(),
- StandardCopyOption.REPLACE_EXISTING);
+ Files.move(tempContainerFile.toPath(), containerFile.toPath(), REPLACE_EXISTING);
}
} catch (IOException ex) {
onFailure(containerData.getVolume());
- String containerExceptionMessage = "Error while creating/updating" +
- " container file. ContainerID: " + containerId +
- ", container path: " + containerFile.getAbsolutePath();
+ String containerExceptionMessage = "Error while creating/updating container file. ContainerID: "
+ + containerId
+ + ", container path: "
+ + containerFile.getAbsolutePath();
if (tempContainerFile == null) {
containerExceptionMessage += " Temporary file could not be created.";
}
- throw new StorageContainerException(containerExceptionMessage, ex,
- CONTAINER_FILES_CREATE_ERROR);
+ throw new StorageContainerException(containerExceptionMessage, ex, CONTAINER_FILES_CREATE_ERROR);
} finally {
- if (tempContainerFile != null && tempContainerFile.exists()) {
- if (!tempContainerFile.delete()) {
- LOG.warn("Unable to delete container temporary file: {}.",
- tempContainerFile.getAbsolutePath());
- }
+ if (tempContainerFile != null && tempContainerFile.exists() && !tempContainerFile.delete()) {
+ LOG.warn("Unable to delete container temporary file: {}.", tempContainerFile.getAbsolutePath());
}
}
}
- private void createContainerFile(File containerFile)
- throws StorageContainerException {
+ private void createContainerFile(File containerFile) throws StorageContainerException {
writeToContainerFile(containerFile, true);
}
- private void updateContainerFile(File containerFile)
- throws StorageContainerException {
+ private void updateContainerFile(File containerFile) throws StorageContainerException {
writeToContainerFile(containerFile, false);
}
-
@Override
public void delete() throws StorageContainerException {
long containerId = containerData.getContainerID();
try {
// Delete the Container from tmp directory.
- File tmpDirectoryPath = KeyValueContainerUtil.getTmpDirectoryPath(
- containerData, containerData.getVolume()).toFile();
+ File tmpDirectoryPath =
+ KeyValueContainerUtil.getTmpDirectoryPath(containerData, containerData.getVolume()).toFile();
FileUtils.deleteDirectory(tmpDirectoryPath);
} catch (StorageContainerException ex) {
// Disk needs replacement.
throw ex;
} catch (IOException ex) {
- // Container will be removed from tmp directory under the volume.
- // On datanode shutdown/restart any partial artifacts left
- // will be wiped from volume's tmp directory.
+ // The Container will be removed from tmp directory under the volume.
+ // On datanode shutdown/restart any partial artifacts left will be wiped from volume's tmp directory.
onFailure(containerData.getVolume());
- String errMsg = String.format("Failed to cleanup container. ID: %d",
- containerId);
+ String errMsg = String.format("Failed to cleanup container. ID: %d", containerId);
LOG.error(errMsg, ex);
throw new StorageContainerException(errMsg, ex, CONTAINER_INTERNAL_ERROR);
}
@@ -367,8 +347,7 @@ public void delete() throws StorageContainerException {
@Override
public boolean hasBlocks() throws IOException {
try (DBHandle db = BlockUtils.getDB(containerData, config)) {
- return !KeyValueContainerUtil.noBlocksInContainer(db.getStore(),
- containerData, bCheckChunksFilePath);
+ return !KeyValueContainerUtil.noBlocksInContainer(db.getStore(), containerData, bCheckChunksFilePath);
}
}
@@ -381,11 +360,10 @@ public void markContainerForClose() throws StorageContainerException {
"Attempting to close a " + getContainerState() + " container.",
CONTAINER_NOT_OPEN);
}
- updateContainerData(() ->
- containerData.setState(ContainerDataProto.State.CLOSING));
- // Do not clear the pendingBlockCache here as a follower can still
- // receive transactions from leader in CLOSING state. Refer to
- // KeyValueHandler#checkContainerOpen()
+ updateContainerData(() -> containerData.setState(ContainerDataProto.State.CLOSING));
+ // Do not clear the pendingBlockCache here
+ // as a follower can still receive transactions from leader in CLOSING state.
+ // Refer to KeyValueHandler#checkContainerOpen()
} finally {
writeUnlock();
}
@@ -396,15 +374,16 @@ public void markContainerUnhealthy() throws StorageContainerException {
writeLock();
ContainerDataProto.State prevState = containerData.getState();
try {
- updateContainerData(() ->
- containerData.setState(ContainerDataProto.State.UNHEALTHY));
+ updateContainerData(() -> containerData.setState(ContainerDataProto.State.UNHEALTHY));
clearPendingPutBlockCache();
} finally {
writeUnlock();
}
- LOG.warn("Moving container {} to state {} from state:{}",
- containerData.getContainerPath(), containerData.getState(),
- prevState);
+ LOG.warn(
+ "Moving container {} to state {} from state:{}",
+ containerData.getContainerPath(),
+ containerData.getState(),
+ prevState);
}
@Override
@@ -414,7 +393,7 @@ public void markContainerForDelete() {
try {
containerData.setState(ContainerDataProto.State.DELETED);
File containerFile = getContainerFile();
- // update the new container data to .container File
+ // Update the new container data to .container File
updateContainerFile(containerFile);
} catch (IOException ioe) {
LOG.error("Exception occur while update container {} state",
@@ -422,8 +401,10 @@ public void markContainerForDelete() {
} finally {
writeUnlock();
}
- LOG.info("Moving container {} to state {} from state:{}",
- containerData.getContainerPath(), containerData.getState(),
+ LOG.info(
+ "Moving container {} to state {} from state:{}",
+ containerData.getContainerPath(),
+ containerData.getState(),
prevState);
}
@@ -441,14 +422,14 @@ public void close() throws StorageContainerException {
throw new StorageContainerException(ex, IO_EXCEPTION);
}
closeAndFlushIfNeeded(containerData::closeContainer);
- LOG.info("Container {} is closed with bcsId {}.",
+ LOG.info(
+ "Container {} is closed with bcsId {}.",
containerData.getContainerID(),
containerData.getBlockCommitSequenceId());
}
@Override
- public void updateDataScanTimestamp(Instant time)
- throws StorageContainerException {
+ public void updateDataScanTimestamp(Instant time) throws StorageContainerException {
writeLock();
try {
updateContainerData(() -> containerData.updateDataScanTime(time));
@@ -460,17 +441,15 @@ public void updateDataScanTimestamp(Instant time)
/**
* Sync RocksDB WAL on closing of a single container.
*
- * @param closer
- * @throws StorageContainerException
+ * @param closer the closer to use for closing the container
+ * @throws StorageContainerException if an error occurs during closing
*/
- private void closeAndFlushIfNeeded(Runnable closer)
- throws StorageContainerException {
+ private void closeAndFlushIfNeeded(Runnable closer) throws StorageContainerException {
flushAndSyncDB();
writeLock();
try {
- // Second sync should be a very light operation as sync has already
- // been done outside the lock.
+ // Second sync should be a very light operation as sync has already been done outside the lock.
flushAndSyncDB();
updateContainerData(closer);
clearPendingPutBlockCache();
@@ -480,14 +459,17 @@ private void closeAndFlushIfNeeded(Runnable closer)
}
/**
+ * Updates the state of the container data and persists it to a file.
+ * The method ensures that changes are made atomically; if the update fails,
+ * the container data state is reverted to its original state (unless the new state is unhealthy).
+ *
+ * It Must be invoked with the writeLock held.
*
- * Must be invoked with the writeLock held.
- *
- * @param update
- * @throws StorageContainerException
+ * @param update A Runnable representing the update operation to be performed on the container data.
+ * @throws StorageContainerException If an error occurs while updating the container data
+ * or writing to the container file.
*/
- private void updateContainerData(Runnable update)
- throws StorageContainerException {
+ private void updateContainerData(Runnable update) throws StorageContainerException {
Preconditions.checkState(hasWriteLock());
ContainerDataProto.State oldState = null;
try {
@@ -496,10 +478,8 @@ private void updateContainerData(Runnable update)
File containerFile = getContainerFile();
// update the new container data to .container File
updateContainerFile(containerFile);
-
} catch (StorageContainerException ex) {
- if (oldState != null
- && containerData.getState() != ContainerDataProto.State.UNHEALTHY) {
+ if (oldState != null && containerData.getState() != ContainerDataProto.State.UNHEALTHY) {
// Failed to update .container file. Reset the state to old state only
// if the current state is not unhealthy.
containerData.setState(oldState);
@@ -526,7 +506,8 @@ private void flushAndSyncDB() throws StorageContainerException {
try {
try (DBHandle db = BlockUtils.getDB(containerData, config)) {
db.getStore().flushLog(true);
- LOG.info("Container {} is synced with bcsId {}.",
+ LOG.info(
+ "Container {} is synced with bcsId {}.",
containerData.getContainerID(),
containerData.getBlockCommitSequenceId());
}
@@ -555,23 +536,20 @@ public ContainerType getContainerType() {
}
@Override
- public void update(
- Map metadata, boolean forceUpdate)
- throws StorageContainerException {
-
- // TODO: Now, when writing the updated data to .container file, we are
- // holding lock and writing data to disk. We can have async implementation
- // to flush the update container data to disk.
+ public void update(Map metadata, boolean forceUpdate) throws StorageContainerException {
+ // TODO: Now, when writing the updated data to .container file, we are holding lock and writing data to disk.
+ // We can have async implementation to flush the update container data to disk.
long containerId = containerData.getContainerID();
if (!containerData.isValid()) {
LOG.debug("Invalid container data. ContainerID: {}", containerId);
- throw new StorageContainerException("Invalid container data. " +
- "ContainerID: " + containerId, INVALID_CONTAINER_STATE);
+ throw new StorageContainerException(
+ "Invalid container data. ContainerID: " + containerId,
+ INVALID_CONTAINER_STATE);
}
if (!forceUpdate && !containerData.isOpen()) {
throw new StorageContainerException(
- "Updating a closed container without force option is not allowed. " +
- "ContainerID: " + containerId, UNSUPPORTED_REQUEST);
+ "Updating a closed container without force option is not allowed. ContainerID: " + containerId,
+ UNSUPPORTED_REQUEST);
}
Map oldMetadata = containerData.getMetadata();
@@ -582,7 +560,7 @@ public void update(
}
File containerFile = getContainerFile();
- // update the new container data to .container File
+ // Update the new container data to .container File
updateContainerFile(containerFile);
} catch (StorageContainerException ex) {
containerData.setMetadata(oldMetadata);
@@ -598,37 +576,30 @@ public void updateDeleteTransactionId(long deleteTransactionId) {
}
@Override
- public void importContainerData(InputStream input,
- ContainerPacker packer)
- throws IOException {
+ public void importContainerData(InputStream input, ContainerPacker packer) throws IOException {
HddsVolume hddsVolume = containerData.getVolume();
- String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
- hddsVolume, hddsVolume.getClusterID());
+ String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(hddsVolume, hddsVolume.getClusterID());
long containerId = containerData.getContainerID();
Path destContainerDir =
- Paths.get(KeyValueContainerLocationUtil.getBaseContainerLocation(
- hddsVolume.getHddsRootDir().toString(), idDir, containerId));
+ Paths.get(KeyValueContainerLocationUtil
+ .getBaseContainerLocation(hddsVolume.getHddsRootDir().toString(), idDir, containerId));
Path tmpDir = ContainerImporter.getUntarDirectory(hddsVolume);
writeLock();
try {
- //copy the values from the input stream to the final destination
- // directory.
- byte[] descriptorContent = packer.unpackContainerData(this, input, tmpDir,
- destContainerDir);
-
- Preconditions.checkNotNull(descriptorContent,
- "Container descriptor is missing from the container archive: "
- + getContainerData().getContainerID());
-
- //now, we have extracted the container descriptor from the previous
- //datanode. We can load it and upload it with the current data
- // (original metadata + current filepath fields)
+ // Copy the values from the input stream to the final destination directory.
+ byte[] descriptorContent = packer.unpackContainerData(this, input, tmpDir, destContainerDir);
+
+ Preconditions.checkNotNull(
+ descriptorContent,
+ "Container descriptor is missing from the container archive: " + getContainerData().getContainerID());
+
+ // Now, we have extracted the container descriptor from the previous datanode.
+ // We can load it and upload it with the current data (original metadata + current filepath fields)
KeyValueContainerData originalContainerData =
- (KeyValueContainerData) ContainerDataYaml
- .readContainer(descriptorContent);
+ (KeyValueContainerData) ContainerDataYaml.readContainer(descriptorContent);
importContainerData(originalContainerData);
} catch (Exception ex) {
- // clean data under tmp directory
+ // Clean data under tmp directory
try {
Path containerUntarDir = tmpDir.resolve(String.valueOf(containerId));
if (containerUntarDir.toFile().exists()) {
@@ -637,17 +608,18 @@ public void importContainerData(InputStream input,
} catch (Exception deleteex) {
LOG.error(
"Can not cleanup container directory under {} for container {}",
- tmpDir, containerId, deleteex);
+ tmpDir,
+ containerId,
+ deleteex);
}
// Throw exception for existed containers
- if (ex instanceof StorageContainerException &&
- ((StorageContainerException) ex).getResult() ==
- CONTAINER_ALREADY_EXISTS) {
+ if (ex instanceof StorageContainerException
+ && ((StorageContainerException) ex).getResult() == CONTAINER_ALREADY_EXISTS) {
throw ex;
}
- // delete all other temporary data in case of any exception.
+ // Delete all other temporary data in case of any exception.
try {
if (containerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
BlockUtils.removeContainerFromDB(containerData, config);
@@ -658,8 +630,9 @@ public void importContainerData(InputStream input,
new File(getContainerData().getContainerPath()));
} catch (Exception deleteex) {
LOG.error(
- "Can not cleanup destination directories after a container import"
- + " error (cid: {}", containerId, deleteex);
+ "Can not cleanup destination directories after a container import error (cid: {}",
+ containerId,
+ deleteex);
}
throw ex;
} finally {
@@ -667,48 +640,53 @@ public void importContainerData(InputStream input,
}
}
- public void importContainerData(KeyValueContainerData originalContainerData)
- throws IOException {
+ /**
+ * Imports metadata and other necessary data from the original container data into the current container.
+ * It updates the state, container type, and schema version before rewriting the YAML file with a new checksum.
+ * Depending on the schema version,
+ * it may also load metadata from received dump files and fill in memory statistics such as key count and byte usage.
+ *
+ * @param originalContainerData the data from the original container to import
+ * @throws IOException if an I/O error occurs during the import process
+ */
+ public void importContainerData(KeyValueContainerData originalContainerData) throws IOException {
containerData.setState(originalContainerData.getState());
- containerData
- .setContainerDBType(originalContainerData.getContainerDBType());
+ containerData.setContainerDBType(originalContainerData.getContainerDBType());
containerData.setSchemaVersion(originalContainerData.getSchemaVersion());
- //rewriting the yaml file with new checksum calculation.
+ // Rewriting the yaml file with new checksum calculation.
update(originalContainerData.getMetadata(), true);
if (containerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
- // load metadata from received dump files before we try to parse kv
+ // Load metadata from received dump files before we try to parse kv
BlockUtils.loadKVContainerDataFromFiles(containerData, config);
}
- //fill in memory stat counter (keycount, byte usage)
+ // Fill in memory stat counter (keycount, byte usage)
KeyValueContainerUtil.parseKVContainerData(containerData, config);
}
@Override
- public void exportContainerData(OutputStream destination,
- ContainerPacker packer) throws IOException {
+ public void exportContainerData(OutputStream destination, ContainerPacker packer)
+ throws IOException {
writeLock();
try {
- // Closed/ Quasi closed and unhealthy containers are considered for
- // replication by replication manager if they are under-replicated.
- ContainerProtos.ContainerDataProto.State state =
- getContainerData().getState();
- if (!(state == ContainerProtos.ContainerDataProto.State.CLOSED ||
- state == ContainerDataProto.State.QUASI_CLOSED
- || state == ContainerDataProto.State.UNHEALTHY)) {
- throw new IllegalStateException(
- "Only (quasi)closed and unhealthy containers can be exported. " +
- "ContainerId=" + getContainerData().getContainerID() +
- " is in state " + state);
+ // Replication manager considers closed/ Quasi closed and unhealthy containers for replication
+ // if they are under-replicated.
+ ContainerProtos.ContainerDataProto.State state = getContainerData().getState();
+ if (state != ContainerProtos.ContainerDataProto.State.CLOSED
+ && state != ContainerDataProto.State.QUASI_CLOSED
+ && state != ContainerDataProto.State.UNHEALTHY) {
+ throw new IllegalStateException("Only (quasi)closed and unhealthy containers can be exported. ContainerId="
+ + getContainerData().getContainerID()
+ + " is in state "
+ + state);
}
try {
if (!containerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
compactDB();
- // Close DB (and remove from cache) to avoid concurrent modification
- // while packing it.
+ // Close DB (and remove from cache) to avoid concurrent modification while packing it.
BlockUtils.removeDB(containerData, config);
}
} finally {
@@ -756,8 +734,7 @@ public boolean hasReadLock() {
*/
@Override
public void writeLock() {
- // TODO: The lock for KeyValueContainer object should not be exposed
- // publicly.
+ // TODO: The lock for KeyValueContainer object should not be exposed publicly.
this.lock.writeLock().lock();
}
@@ -780,7 +757,6 @@ public boolean hasWriteLock() {
/**
* Acquire read lock, unless interrupted while waiting.
- * @throws InterruptedException
*/
@Override
public void readLockInterruptibly() throws InterruptedException {
@@ -789,7 +765,6 @@ public void readLockInterruptibly() throws InterruptedException {
/**
* Acquire write lock, unless interrupted while waiting.
- * @throws InterruptedException
*/
@Override
public void writeLockInterruptibly() throws InterruptedException {
@@ -797,24 +772,37 @@ public void writeLockInterruptibly() throws InterruptedException {
}
- public boolean writeLockTryLock(long time, TimeUnit unit)
- throws InterruptedException {
+ /**
+ * Attempts to acquire the write lock within the given time frame.
+ *
+ * @param time the maximum time to wait for the lock
+ * @param unit the time unit of the time argument
+ * @return {@code true} if the lock was acquired, {@code false} otherwise
+ * @throws InterruptedException if the current thread is interrupted while waiting
+ */
+ public boolean writeLockTryLock(long time, TimeUnit unit) throws InterruptedException {
return this.lock.writeLock().tryLock(time, unit);
}
/**
* Returns containerFile.
+ *
* @return .container File name
*/
@Override
public File getContainerFile() {
- return getContainerFile(containerData.getMetadataPath(),
- containerData.getContainerID());
+ return getContainerFile(containerData.getMetadataPath(), containerData.getContainerID());
}
+ /**
+ * Retrieves the container file based on the provided metadata path and container ID.
+ *
+ * @param metadataPath the path to the metadata directory
+ * @param containerId the identifier of the container
+ * @return the container file represented as a {@link File} object
+ */
public static File getContainerFile(String metadataPath, long containerId) {
- return new File(metadataPath,
- containerId + OzoneConsts.CONTAINER_EXTENSION);
+ return new File(metadataPath, containerId + OzoneConsts.CONTAINER_EXTENSION);
}
@Override
@@ -828,8 +816,7 @@ public long getBlockCommitSequenceId() {
}
/**
- * Return whether the given localID of a block is present in the
- * pendingPutBlockCache or not.
+ * Return whether the given localID of a block is present in the pendingPutBlockCache or not.
*/
public boolean isBlockInPendingPutBlockCache(long localID) {
return pendingPutBlockCache.contains(localID);
@@ -838,21 +825,18 @@ public boolean isBlockInPendingPutBlockCache(long localID) {
/**
* Add the given localID of a block to the pendingPutBlockCache.
*/
- public void addToPendingPutBlockCache(long localID)
- throws StorageContainerException {
+ public void addToPendingPutBlockCache(long localID) throws StorageContainerException {
try {
pendingPutBlockCache.add(localID);
} catch (UnsupportedOperationException e) {
- // Getting an UnsupportedOperationException here implies that the
- // pendingPutBlockCache is an Empty Set. This should not happen if the
- // container is in OPEN or CLOSING state. Log the exception here and
- // throw a non-Runtime exception so that putBlock request fails.
- String msg = "Failed to add block " + localID + " to " +
- "pendingPutBlockCache of container " + containerData.getContainerID()
+ // Getting an UnsupportedOperationException here implies that the pendingPutBlockCache is an Empty Set.
+ // This should not happen if the container is in OPEN or CLOSING state.
+ // Log the exception here and throw a non-Runtime exception so that putBlock request fails.
+ String msg = "Failed to add block " + localID
+ + " to pendingPutBlockCache of container " + containerData.getContainerID()
+ " (state: " + containerData.getState() + ")";
LOG.error(msg, e);
- throw new StorageContainerException(msg,
- ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
+ throw new StorageContainerException(msg, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
}
}
@@ -864,8 +848,8 @@ public void removeFromPendingPutBlockCache(long localID) {
}
/**
- * When a container is closed, quasi-closed or marked unhealthy, clear the
- * pendingPutBlockCache as there won't be any more writes to the container.
+ * When a container is closed, quasi-closed or marked unhealthy,
+ * clear the pendingPutBlockCache as there won't be anymore writes to the container.
*/
private void clearPendingPutBlockCache() {
pendingPutBlockCache.clear();
@@ -876,11 +860,9 @@ private void clearPendingPutBlockCache() {
* Returns KeyValueContainerReport for the KeyValueContainer.
*/
@Override
- public ContainerReplicaProto getContainerReport()
- throws StorageContainerException {
- ContainerReplicaProto.Builder ciBuilder =
- ContainerReplicaProto.newBuilder();
- ciBuilder.setContainerID(containerData.getContainerID())
+ public ContainerReplicaProto getContainerReport() throws StorageContainerException {
+ return ContainerReplicaProto.newBuilder()
+ .setContainerID(containerData.getContainerID())
.setReadCount(containerData.getReadCount())
.setWriteCount(containerData.getWriteCount())
.setReadBytes(containerData.getReadBytes())
@@ -892,17 +874,16 @@ public ContainerReplicaProto getContainerReport()
.setDeleteTransactionId(containerData.getDeleteTransactionId())
.setBlockCommitSequenceId(containerData.getBlockCommitSequenceId())
.setOriginNodeId(containerData.getOriginNodeId())
- .setIsEmpty(containerData.isEmpty());
- return ciBuilder.build();
+ .setIsEmpty(containerData.isEmpty())
+ .build();
}
/**
* Returns LifeCycle State of the container.
+ *
* @return LifeCycle State of the container in HddsProtos format
- * @throws StorageContainerException
*/
- private ContainerReplicaProto.State getHddsState()
- throws StorageContainerException {
+ private ContainerReplicaProto.State getHddsState() throws StorageContainerException {
ContainerReplicaProto.State state;
switch (containerData.getState()) {
case OPEN:
@@ -924,15 +905,15 @@ private ContainerReplicaProto.State getHddsState()
state = ContainerReplicaProto.State.DELETED;
break;
default:
- throw new StorageContainerException("Invalid Container state found: " +
- containerData.getContainerID(), INVALID_CONTAINER_STATE);
+ throw new StorageContainerException(
+ "Invalid Container state found: " + containerData.getContainerID(),
+ INVALID_CONTAINER_STATE);
}
return state;
}
/**
* Returns container DB file.
- * @return
*/
public File getContainerDBFile() {
return KeyValueContainerLocationUtil.getContainerDBFile(containerData);
@@ -941,12 +922,12 @@ public File getContainerDBFile() {
@Override
public boolean shouldScanMetadata() {
- boolean shouldScan =
- getContainerState() != ContainerDataProto.State.UNHEALTHY;
+ boolean shouldScan = getContainerState() != ContainerDataProto.State.UNHEALTHY;
if (!shouldScan && LOG.isDebugEnabled()) {
- LOG.debug("Container {} in state {} should not have its metadata " +
- "scanned.",
- containerData.getContainerID(), containerData.getState());
+ LOG.debug(
+ "Container {} in state {} should not have its metadata scanned.",
+ containerData.getContainerID(),
+ containerData.getState());
}
return shouldScan;
}
@@ -954,61 +935,64 @@ public boolean shouldScanMetadata() {
@Override
public ScanResult scanMetaData() throws InterruptedException {
long containerId = containerData.getContainerID();
- KeyValueContainerCheck checker =
- new KeyValueContainerCheck(containerData.getMetadataPath(), config,
- containerId, containerData.getVolume(), this);
+ KeyValueContainerCheck checker = new KeyValueContainerCheck(
+ containerData.getMetadataPath(),
+ config,
+ containerId,
+ containerData.getVolume(),
+ this);
return checker.fastCheck();
}
@Override
public boolean shouldScanData() {
- boolean shouldScan =
- getContainerState() == ContainerDataProto.State.CLOSED
+ boolean shouldScan = getContainerState() == ContainerDataProto.State.CLOSED
|| getContainerState() == ContainerDataProto.State.QUASI_CLOSED;
if (!shouldScan && LOG.isDebugEnabled()) {
- LOG.debug("Container {} in state {} should not have its data scanned.",
- containerData.getContainerID(), containerData.getState());
+ LOG.debug(
+ "Container {} in state {} should not have its data scanned.",
+ containerData.getContainerID(),
+ containerData.getState());
}
return shouldScan;
}
@Override
- public ScanResult scanData(DataTransferThrottler throttler, Canceler canceler)
- throws InterruptedException {
+ public ScanResult scanData(DataTransferThrottler throttler, Canceler canceler) throws InterruptedException {
if (!shouldScanData()) {
- throw new IllegalStateException("The checksum verification can not be" +
- " done for container in state "
- + containerData.getState());
+ throw new IllegalStateException(
+ "The checksum verification can not be done for container in state " + containerData.getState());
}
long containerId = containerData.getContainerID();
- KeyValueContainerCheck checker =
- new KeyValueContainerCheck(containerData.getMetadataPath(), config,
- containerId, containerData.getVolume(), this);
+ KeyValueContainerCheck checker = new KeyValueContainerCheck(
+ containerData.getMetadataPath(),
+ config,
+ containerId,
+ containerData.getVolume(),
+ this);
return checker.fullCheck(throttler, canceler);
}
/**
- * Creates a temporary file.
- * @param file
- * @return
- * @throws IOException
+ * Creates a temporary file in the same directory as the given file,
+ * with a unique prefix based on the current system time.
+ *
+ * @param file the file whose directory will be used to create the temporary file
+ * @return a newly created temporary file
+ * @throws IOException if an I/O error occurs
*/
private File createTempFile(File file) throws IOException {
- return File.createTempFile("tmp_" + System.currentTimeMillis() + "_",
- file.getName(), file.getParentFile());
+ return File.createTempFile("tmp_" + System.currentTimeMillis() + "_", file.getName(), file.getParentFile());
}
- private void packContainerToDestination(OutputStream destination,
- ContainerPacker packer)
+ private void packContainerToDestination(OutputStream destination, ContainerPacker packer)
throws IOException {
if (containerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
- // Synchronize the dump and pack operation,
- // so concurrent exports don't get dump files overwritten.
- // We seldom got concurrent exports for a container,
- // so it should not influence performance much.
+ // Synchronize the dump and pack operation, so concurrent exports don't get dump files overwritten.
+ // We seldom got concurrent exports for a container, so it should not influence performance much.
synchronized (dumpLock) {
BlockUtils.dumpKVContainerDataToFiles(containerData, config);
packer.pack(this, destination);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
index c235109f2cb..fc910b02028 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
@@ -53,28 +53,36 @@
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE_ROCKSDB;
+import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.CORRUPT_CHUNK;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.CORRUPT_CONTAINER_FILE;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.INACCESSIBLE_DB;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.INCONSISTENT_CHUNK_LENGTH;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.MISSING_CHUNKS_DIR;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.MISSING_CHUNK_FILE;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.MISSING_CONTAINER_DIR;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.MISSING_CONTAINER_FILE;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.MISSING_METADATA_DIR;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.unhealthy;
/**
* Class to run integrity checks on Datanode Containers.
* Provide infra for Data Scrubbing
*/
-
public class KeyValueContainerCheck {
+ private static final Logger LOG = LoggerFactory.getLogger(KeyValueContainerCheck.class);
- private static final Logger LOG =
- LoggerFactory.getLogger(KeyValueContainerCheck.class);
-
- private long containerID;
+ private final long containerID;
private KeyValueContainerData onDiskContainerData; //loaded from fs/disk
- private ConfigurationSource checkConfig;
+ private final ConfigurationSource checkConfig;
- private String metadataPath;
- private HddsVolume volume;
- private KeyValueContainer container;
- private static final DirectBufferPool BUFFER_POOL = new DirectBufferPool();
+ private final String metadataPath;
+ private final HddsVolume volume;
+ private final KeyValueContainer container;
+ public static final DirectBufferPool BUFFER_POOL = new DirectBufferPool();
- public KeyValueContainerCheck(String metadataPath, ConfigurationSource conf,
- long containerID, HddsVolume volume, KeyValueContainer container) {
+ public KeyValueContainerCheck(String metadataPath, ConfigurationSource conf, long containerID, HddsVolume volume,
+ KeyValueContainer container) {
Preconditions.checkArgument(metadataPath != null);
this.checkConfig = conf;
@@ -90,7 +98,7 @@ public KeyValueContainerCheck(String metadataPath, ConfigurationSource conf,
* These checks do not look inside the metadata files.
* Applicable for OPEN containers.
*
- * @return true : integrity checks pass, false : otherwise.
+ * @return {@code true}: integrity checks pass, {@code false}: otherwise.
*/
public ScanResult fastCheck() throws InterruptedException {
LOG.debug("Running basic checks for container {};", containerID);
@@ -99,193 +107,155 @@ public ScanResult fastCheck() throws InterruptedException {
// Container directory should exist.
File containerDir = new File(metadataPath).getParentFile();
if (!containerDir.exists()) {
- return ScanResult.unhealthy(
- ScanResult.FailureType.MISSING_CONTAINER_DIR,
- containerDir, new FileNotFoundException("Container directory " +
- containerDir + " not found."));
+ return unhealthy(MISSING_CONTAINER_DIR, containerDir,
+ new FileNotFoundException("Container directory " + containerDir + " not found."));
}
// Metadata directory should exist.
File metadataDir = new File(metadataPath);
if (!metadataDir.exists()) {
- return ScanResult.unhealthy(ScanResult.FailureType.MISSING_METADATA_DIR,
- metadataDir, new FileNotFoundException("Metadata directory " +
- metadataDir + " not found."));
+ return unhealthy(MISSING_METADATA_DIR, metadataDir,
+ new FileNotFoundException("Metadata directory " + metadataDir + " not found."));
}
// Container file should be valid.
- File containerFile = KeyValueContainer
- .getContainerFile(metadataPath, containerID);
+ File containerFile = KeyValueContainer.getContainerFile(metadataPath, containerID);
try {
loadContainerData(containerFile);
} catch (FileNotFoundException ex) {
- return ScanResult.unhealthy(
- ScanResult.FailureType.MISSING_CONTAINER_FILE, containerFile, ex);
+ return unhealthy(MISSING_CONTAINER_FILE, containerFile, ex);
} catch (IOException ex) {
- return ScanResult.unhealthy(
- ScanResult.FailureType.CORRUPT_CONTAINER_FILE, containerFile, ex);
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, ex);
}
// Chunks directory should exist.
File chunksDir = new File(onDiskContainerData.getChunksPath());
if (!chunksDir.exists()) {
- return ScanResult.unhealthy(ScanResult.FailureType.MISSING_CHUNKS_DIR,
- chunksDir, new FileNotFoundException("Chunks directory " +
- chunksDir + " not found."));
+ return unhealthy(MISSING_CHUNKS_DIR, chunksDir,
+ new FileNotFoundException("Chunks directory " + chunksDir + " not found."));
}
return checkContainerFile(containerFile);
} finally {
if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedException("Metadata scan of container " +
- containerID + " interrupted.");
+ throw new InterruptedException("Metadata scan of container " + containerID + " interrupted.");
}
}
}
/**
- * full checks comprise scanning all metadata inside the container.
- * Including the KV database. These checks are intrusive, consume more
- * resources compared to fast checks and should only be done on Closed
- * or Quasi-closed Containers. Concurrency being limited to delete
- * workflows.
+ * Full checks comprise scanning all metadata inside the container.
+ * Including the KV database.
+ * These checks are intrusive,
+ * consume more resources compared to fast checks and should only be done on Closed or Quasi-closed Containers.
+ * Concurrency being limited to delete workflows.
*
* fullCheck is a superset of fastCheck
*
- * @return true : integrity checks pass, false : otherwise.
+ * @return {@code true}: integrity checks pass, {@code false}: otherwise.
*/
- public ScanResult fullCheck(DataTransferThrottler throttler,
- Canceler canceler) throws InterruptedException {
+ public ScanResult fullCheck(DataTransferThrottler throttler, Canceler canceler) throws InterruptedException {
ScanResult result = fastCheck();
if (result.isHealthy()) {
result = scanData(throttler, canceler);
}
if (!result.isHealthy() && Thread.currentThread().isInterrupted()) {
- throw new InterruptedException("Data scan of container " + containerID +
- " interrupted.");
+ throw new InterruptedException("Data scan of container " + containerID + " interrupted.");
}
return result;
}
private ScanResult checkContainerFile(File containerFile) {
- /*
- * compare the values in the container file loaded from disk,
- * with the values we are expecting
- */
+ // Compare the values in the container file loaded from the disk with the values we are expecting
String dbType;
- Preconditions
- .checkState(onDiskContainerData != null, "Container File not loaded");
+ Preconditions.checkState(onDiskContainerData != null, "Container File not loaded");
try {
ContainerUtils.verifyChecksum(onDiskContainerData, checkConfig);
} catch (IOException ex) {
- return ScanResult.unhealthy(ScanResult.FailureType.CORRUPT_CONTAINER_FILE,
- containerFile, ex);
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, ex);
}
- if (onDiskContainerData.getContainerType()
- != ContainerProtos.ContainerType.KeyValueContainer) {
- String errStr = "Bad Container type in Containerdata for " + containerID;
- return ScanResult.unhealthy(ScanResult.FailureType.CORRUPT_CONTAINER_FILE,
- containerFile, new IOException(errStr));
+ if (onDiskContainerData.getContainerType() != ContainerProtos.ContainerType.KeyValueContainer) {
+ String errStr = "Bad Container type in ContainerData for " + containerID;
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, new IOException(errStr));
}
if (onDiskContainerData.getContainerID() != containerID) {
- String errStr =
- "Bad ContainerID field in Containerdata for " + containerID;
- return ScanResult.unhealthy(ScanResult.FailureType.CORRUPT_CONTAINER_FILE,
- containerFile, new IOException(errStr));
+ String errStr = "Bad ContainerID field in ContainerData for " + containerID;
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, new IOException(errStr));
}
dbType = onDiskContainerData.getContainerDBType();
if (!dbType.equals(CONTAINER_DB_TYPE_ROCKSDB)) {
- String errStr = "Unknown DBType [" + dbType
- + "] in Container File for [" + containerID + "]";
- return ScanResult.unhealthy(ScanResult.FailureType.CORRUPT_CONTAINER_FILE,
- containerFile, new IOException(errStr));
+ String errStr = "Unknown DBType [" + dbType + "] in Container File for [" + containerID + "]";
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, new IOException(errStr));
}
KeyValueContainerData kvData = onDiskContainerData;
if (!metadataPath.equals(kvData.getMetadataPath())) {
- String errStr =
- "Bad metadata path in Containerdata for " + containerID + "Expected ["
- + metadataPath + "] Got [" + kvData.getMetadataPath()
- + "]";
- return ScanResult.unhealthy(ScanResult.FailureType.CORRUPT_CONTAINER_FILE,
- containerFile, new IOException(errStr));
+ String errStr = "Bad metadata path in Containerdata for " + containerID + "Expected [" + metadataPath
+ + "] Got [" + kvData.getMetadataPath() + "]";
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, new IOException(errStr));
}
return ScanResult.healthy();
}
- private ScanResult scanData(DataTransferThrottler throttler,
- Canceler canceler) {
+ private ScanResult scanData(DataTransferThrottler throttler, Canceler canceler) {
/*
* Check the integrity of the DB inside each container.
- * 1. iterate over each key (Block) and locate the chunks for the block
- * 2. garbage detection (TBD): chunks which exist in the filesystem,
- * but not in the DB. This function will be implemented in HDDS-1202
- * 3. chunk checksum verification.
+ * 1. Iterate over each key (Block) and locate the chunks for the block
+ * 2. Garbage detection (TBD): chunks which exist in the filesystem, but not in the DB.
+ * This function will be implemented in HDDS-1202.
+ * 3. Chunk checksum verification.
*/
- Preconditions.checkState(onDiskContainerData != null,
- "invoke loadContainerData prior to calling this function");
+ Preconditions
+ .checkState(onDiskContainerData != null, "invoke loadContainerData prior to calling this function");
- File dbFile = KeyValueContainerLocationUtil
- .getContainerDBFile(onDiskContainerData);
+ File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(onDiskContainerData);
if (!dbFile.exists() || !dbFile.canRead()) {
- String dbFileErrorMsg = "Unable to access DB File [" + dbFile.toString()
- + "] for Container [" + containerID + "] metadata path ["
- + metadataPath + "]";
- return ScanResult.unhealthy(ScanResult.FailureType.INACCESSIBLE_DB,
- dbFile, new IOException(dbFileErrorMsg));
+ String dbFileErrorMsg = "Unable to access DB File [" + dbFile + "] for Container [" + containerID
+ + "] metadata path [" + metadataPath + "]";
+ return unhealthy(INACCESSIBLE_DB, dbFile, new IOException(dbFileErrorMsg));
}
onDiskContainerData.setDbFile(dbFile);
try {
try (DBHandle db = BlockUtils.getDB(onDiskContainerData, checkConfig);
- BlockIterator kvIter = db.getStore().getBlockIterator(
- onDiskContainerData.getContainerID(),
- onDiskContainerData.getUnprefixedKeyFilter())) {
+ BlockIterator kvIter = db.getStore().getBlockIterator(onDiskContainerData.getContainerID(),
+ onDiskContainerData.getUnprefixedKeyFilter())) {
while (kvIter.hasNext()) {
BlockData block = kvIter.nextBlock();
- // If holding read lock for the entire duration, including wait()
- // calls in DataTransferThrottler, would effectively make other
- // threads throttled.
- // Here try optimistically and retry with the container lock to
- // make sure reading the latest record. If the record is just removed,
- // the block should be skipped to scan.
+ // If holding read lock for the entire duration,
+ // including wait() calls in DataTransferThrottler,
+ // would effectively make other threads throttled.
+ // Here try optimistically and retry with the container lock to make sure reading the latest record.
+ // If the record is just removed, the block should be skipped to scan.
ScanResult result = scanBlock(block, throttler, canceler);
if (!result.isHealthy()) {
- if (result.getFailureType() ==
- ScanResult.FailureType.MISSING_CHUNK_FILE) {
+ if (result.getFailureType() == MISSING_CHUNK_FILE) {
if (getBlockDataFromDBWithLock(db, block) != null) {
// Block was not deleted, the failure is legitimate.
return result;
} else {
- // If schema V3 and container details not in DB or
- // if containerDBPath is removed
- if ((onDiskContainerData.hasSchema(OzoneConsts.SCHEMA_V3) &&
- db.getStore().getMetadataTable().get(
- onDiskContainerData.getBcsIdKey()) == null) ||
- !new File(onDiskContainerData.getDbFile()
- .getAbsolutePath()).exists()) {
- // Container has been deleted. Skip the rest of the blocks.
- return ScanResult.unhealthy(
- ScanResult.FailureType.DELETED_CONTAINER,
+ // If schema V3 and container details not in DB or if containerDBPath is removed
+ if (onDiskContainerData.hasSchema(OzoneConsts.SCHEMA_V3)
+ && db.getStore().getMetadataTable().get(onDiskContainerData.getBcsIdKey()) == null
+ || !new File(onDiskContainerData.getDbFile().getAbsolutePath()).exists()) {
+ // The Container has been deleted. Skip the rest of the blocks.
+ return unhealthy(ScanResult.FailureType.DELETED_CONTAINER,
result.getUnhealthyFile(), result.getException());
}
- // Block may have been deleted during the scan.
- if (LOG.isDebugEnabled()) {
- LOG.debug("Scanned outdated blockData {} in container {}.",
- block, containerID);
- }
+ // The Block may have been deleted during the scan.
+ LOG.debug("Scanned outdated blockData {} in container {}.", block, containerID);
}
} else {
// All other failures should be treated as errors.
@@ -295,8 +265,7 @@ private ScanResult scanData(DataTransferThrottler throttler,
}
}
} catch (IOException ex) {
- return ScanResult.unhealthy(ScanResult.FailureType.INACCESSIBLE_DB,
- dbFile, ex);
+ return unhealthy(INACCESSIBLE_DB, dbFile, ex);
}
return ScanResult.healthy();
@@ -304,33 +273,28 @@ private ScanResult scanData(DataTransferThrottler throttler,
/**
* Attempt to read the block data without the container lock.
- * The block onDisk might be in modification by other thread and not yet
- * flushed to DB, so the content might be outdated.
+ * The block onDisk might be in modification by another thread and not yet flushed to DB,
+ * so the content might be outdated.
*
* @param db DB of container
* @param block last queried blockData
* @return blockData in DB
- * @throws IOException
*/
- private BlockData getBlockDataFromDB(DBHandle db, BlockData block)
- throws IOException {
- String blockKey =
- onDiskContainerData.getBlockKey(block.getBlockID().getLocalID());
+ private BlockData getBlockDataFromDB(DBHandle db, BlockData block) throws IOException {
+ String blockKey = onDiskContainerData.getBlockKey(block.getBlockID().getLocalID());
return db.getStore().getBlockDataTable().get(blockKey);
}
/**
* Attempt to read the block data with the container lock.
- * The container lock ensure the latest DB record could be retrieved, since
- * other block related write operation will acquire the container write lock.
+ * The container lock ensures the latest DB record could be retrieved,
+ * since another block related write operation will acquire the container write lock.
*
* @param db DB of container
* @param block last queried blockData
* @return blockData in DB
- * @throws IOException
*/
- private BlockData getBlockDataFromDBWithLock(DBHandle db, BlockData block)
- throws IOException {
+ private BlockData getBlockDataFromDBWithLock(DBHandle db, BlockData block) throws IOException {
container.readLock();
try {
return getBlockDataFromDB(db, block);
@@ -339,37 +303,28 @@ private BlockData getBlockDataFromDBWithLock(DBHandle db, BlockData block)
}
}
- private ScanResult scanBlock(BlockData block, DataTransferThrottler throttler,
- Canceler canceler) {
+ private ScanResult scanBlock(BlockData block, DataTransferThrottler throttler, Canceler canceler) {
ContainerLayoutVersion layout = onDiskContainerData.getLayoutVersion();
for (ContainerProtos.ChunkInfo chunk : block.getChunks()) {
File chunkFile;
try {
- chunkFile = layout.getChunkFile(onDiskContainerData,
- block.getBlockID(), chunk.getChunkName());
+ chunkFile = layout.getChunkFile(onDiskContainerData, block.getBlockID(), chunk.getChunkName());
} catch (IOException ex) {
- return ScanResult.unhealthy(
- ScanResult.FailureType.MISSING_CHUNK_FILE,
- new File(onDiskContainerData.getChunksPath()), ex);
+ return unhealthy(MISSING_CHUNK_FILE, new File(onDiskContainerData.getChunksPath()), ex);
}
if (!chunkFile.exists()) {
- // In EC, client may write empty putBlock in padding block nodes.
- // So, we need to make sure, chunk length > 0, before declaring
- // the missing chunk file.
- if (block.getChunks().size() > 0 && block
- .getChunks().get(0).getLen() > 0) {
- return ScanResult.unhealthy(ScanResult.FailureType.MISSING_CHUNK_FILE,
- chunkFile, new IOException("Missing chunk file " +
- chunkFile.getAbsolutePath()));
+ // In EC, a client may write empty putBlock in padding block nodes.
+ // So, we need to make sure, chunk length > 0, before declaring the missing chunk file.
+ if (!block.getChunks().isEmpty() && block.getChunks().get(0).getLen() > 0) {
+ return unhealthy(MISSING_CHUNK_FILE, chunkFile,
+ new IOException("Missing chunk file " + chunkFile.getAbsolutePath()));
}
- } else if (chunk.getChecksumData().getType()
- != ContainerProtos.ChecksumType.NONE) {
+ } else if (chunk.getChecksumData().getType() != ContainerProtos.ChecksumType.NONE) {
int bytesPerChecksum = chunk.getChecksumData().getBytesPerChecksum();
ByteBuffer buffer = BUFFER_POOL.getBuffer(bytesPerChecksum);
- ScanResult result = verifyChecksum(block, chunk, chunkFile, layout, buffer,
- throttler, canceler);
+ ScanResult result = verifyChecksum(block, chunk, chunkFile, layout, buffer, throttler, canceler);
buffer.clear();
BUFFER_POOL.returnBuffer(buffer);
if (!result.isHealthy()) {
@@ -381,27 +336,21 @@ chunkFile, new IOException("Missing chunk file " +
return ScanResult.healthy();
}
- private static ScanResult verifyChecksum(BlockData block,
- ContainerProtos.ChunkInfo chunk, File chunkFile,
- ContainerLayoutVersion layout, ByteBuffer buffer,
- DataTransferThrottler throttler, Canceler canceler) {
- ChecksumData checksumData =
- ChecksumData.getFromProtoBuf(chunk.getChecksumData());
+ public static ScanResult verifyChecksum(BlockData block, ContainerProtos.ChunkInfo chunk, File chunkFile,
+ ContainerLayoutVersion layout, ByteBuffer buffer, DataTransferThrottler throttler, Canceler canceler) {
+ ChecksumData checksumData = ChecksumData.getFromProtoBuf(chunk.getChecksumData());
int checksumCount = checksumData.getChecksums().size();
int bytesPerChecksum = checksumData.getBytesPerChecksum();
- Checksum cal = new Checksum(checksumData.getChecksumType(),
- bytesPerChecksum);
+ Checksum cal = new Checksum(checksumData.getChecksumType(), bytesPerChecksum);
long bytesRead = 0;
- try (FileChannel channel = FileChannel.open(chunkFile.toPath(),
- ChunkUtils.READ_OPTIONS, ChunkUtils.NO_ATTRIBUTES)) {
- if (layout == ContainerLayoutVersion.FILE_PER_BLOCK) {
+ try (FileChannel channel =
+ FileChannel.open(chunkFile.toPath(), ChunkUtils.READ_OPTIONS, ChunkUtils.NO_ATTRIBUTES)) {
+ if (layout == FILE_PER_BLOCK) {
channel.position(chunk.getOffset());
}
for (int i = 0; i < checksumCount; i++) {
- // limit last read for FILE_PER_BLOCK, to avoid reading next chunk
- if (layout == ContainerLayoutVersion.FILE_PER_BLOCK &&
- i == checksumCount - 1 &&
- chunk.getLen() % bytesPerChecksum != 0) {
+ // Limit last read for FILE_PER_BLOCK, to avoid reading the next chunk
+ if (layout == FILE_PER_BLOCK && i == checksumCount - 1 && chunk.getLen() % bytesPerChecksum != 0) {
buffer.limit((int) (chunk.getLen() % bytesPerChecksum));
}
@@ -415,46 +364,34 @@ private static ScanResult verifyChecksum(BlockData block,
throttler.throttle(v, canceler);
ByteString expected = checksumData.getChecksums().get(i);
- ByteString actual = cal.computeChecksum(buffer)
- .getChecksums().get(0);
+ ByteString actual = cal.computeChecksum(buffer).getChecksums().get(0);
if (!expected.equals(actual)) {
- String message = String
- .format("Inconsistent read for chunk=%s" +
- " checksum item %d" +
- " expected checksum %s" +
- " actual checksum %s" +
- " for block %s",
+ String message = String.format("Inconsistent read for chunk=%s checksum item %d "
+ + "expected checksum %s actual checksum %s for block %s",
ChunkInfo.getFromProtoBuf(chunk),
i,
StringUtils.bytes2Hex(expected.asReadOnlyByteBuffer()),
StringUtils.bytes2Hex(actual.asReadOnlyByteBuffer()),
block.getBlockID());
- return ScanResult.unhealthy(
- ScanResult.FailureType.CORRUPT_CHUNK, chunkFile,
- new IOException(message));
+ return unhealthy(CORRUPT_CHUNK, chunkFile, new IOException(message));
}
}
if (bytesRead != chunk.getLen()) {
- String message = String
- .format("Inconsistent read for chunk=%s expected length=%d"
- + " actual length=%d for block %s",
+ String message = String.format("Inconsistent read for chunk=%s "
+ + "expected length=%d actual length=%d for block %s",
chunk.getChunkName(),
chunk.getLen(), bytesRead, block.getBlockID());
- return ScanResult.unhealthy(
- ScanResult.FailureType.INCONSISTENT_CHUNK_LENGTH, chunkFile,
- new IOException(message));
+ return unhealthy(INCONSISTENT_CHUNK_LENGTH, chunkFile, new IOException(message));
}
} catch (IOException ex) {
- return ScanResult.unhealthy(
- ScanResult.FailureType.MISSING_CHUNK_FILE, chunkFile, ex);
+ return unhealthy(MISSING_CHUNK_FILE, chunkFile, ex);
}
return ScanResult.healthy();
}
private void loadContainerData(File containerFile) throws IOException {
- onDiskContainerData = (KeyValueContainerData) ContainerDataYaml
- .readContainerFile(containerFile);
+ onDiskContainerData = (KeyValueContainerData) ContainerDataYaml.readContainerFile(containerFile);
onDiskContainerData.setVolume(volume);
}
@@ -464,9 +401,7 @@ void setContainerData(KeyValueContainerData containerData) {
}
@VisibleForTesting
- ScanResult scanContainer(DataTransferThrottler throttler,
- Canceler canceler) {
+ ScanResult scanContainer(DataTransferThrottler throttler, Canceler canceler) {
return scanData(throttler, canceler);
}
-
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 47d4f3f9e70..97ea9e40e0a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -63,9 +63,8 @@
import static org.apache.hadoop.ozone.container.metadata.DatanodeSchemaThreeDBDefinition.getContainerKeyPrefix;
/**
- * This class represents the KeyValueContainer metadata, which is the
- * in-memory representation of container metadata and is represented on disk
- * by the .container file.
+ * This class represents the KeyValueContainer metadata,
+ * which is the in-memory representation of container metadata and is represented on disk by the .container file.
*/
public class KeyValueContainerData extends ContainerData {
@@ -78,7 +77,7 @@ public class KeyValueContainerData extends ContainerData {
// Path to Container metadata Level DB/RocksDB Store and .container file.
private String metadataPath;
- //Type of DB used to store key to chunks mapping
+ //Type of DB used to store key to chunk mapping
private String containerDBType = CONTAINER_DB_TYPE_ROCKSDB;
private File dbFile = null;
@@ -108,14 +107,14 @@ public class KeyValueContainerData extends ContainerData {
/**
* Constructs KeyValueContainerData object.
+ *
* @param id - ContainerId
* @param layoutVersion container layout
* @param size - maximum size of the container in bytes
*/
- public KeyValueContainerData(long id, ContainerLayoutVersion layoutVersion,
- long size, String originPipelineId, String originNodeId) {
- super(ContainerProtos.ContainerType.KeyValueContainer, id, layoutVersion,
- size, originPipelineId, originNodeId);
+ public KeyValueContainerData(long id, ContainerLayoutVersion layoutVersion, long size, String originPipelineId,
+ String originNodeId) {
+ super(ContainerProtos.ContainerType.KeyValueContainer, id, layoutVersion, size, originPipelineId, originNodeId);
this.numPendingDeletionBlocks = new AtomicLong(0);
this.deleteTransactionId = 0;
finalizedBlockSet = ConcurrentHashMap.newKeySet();
@@ -123,8 +122,7 @@ public KeyValueContainerData(long id, ContainerLayoutVersion layoutVersion,
public KeyValueContainerData(KeyValueContainerData source) {
super(source);
- Preconditions.checkArgument(source.getContainerType()
- == ContainerProtos.ContainerType.KeyValueContainer);
+ Preconditions.checkArgument(source.getContainerType() == ContainerProtos.ContainerType.KeyValueContainer);
this.numPendingDeletionBlocks = new AtomicLong(0);
this.deleteTransactionId = 0;
this.schemaVersion = source.getSchemaVersion();
@@ -132,25 +130,22 @@ public KeyValueContainerData(KeyValueContainerData source) {
}
/**
- * @param version The schema version indicating the table layout of the
- * container's database.
+ * @param version The schema version indicating the table layout of the container's database.
*/
public void setSchemaVersion(String version) {
schemaVersion = version;
}
/**
- * @return The schema version describing the container database's table
- * layout.
+ * @return The schema version describing the container database's table layout.
*/
public String getSchemaVersion() {
return schemaVersion;
}
/**
- * Returns schema version or the default value when the
- * {@link KeyValueContainerData#schemaVersion} is null. The default value can
- * be referred to {@link KeyValueContainerUtil#isSameSchemaVersion}.
+ * Returns schema version or the default value when the {@link KeyValueContainerData#schemaVersion} is null.
+ * The default value can be referred to {@link KeyValueContainerUtil#isSameSchemaVersion}.
*
* @return Schema version as a string.
* @throws UnsupportedOperationException If no valid schema version is found.
@@ -167,9 +162,7 @@ public String getSupportedSchemaVersionOrDefault() {
}
/**
- * Sets Container dbFile. This should be called only during creation of
- * KeyValue container.
- * @param containerDbFile
+ * Sets Container dbFile. This should be called only during the creation of KeyValue container.
*/
public void setDbFile(File containerDbFile) {
dbFile = containerDbFile;
@@ -177,6 +170,7 @@ public void setDbFile(File containerDbFile) {
/**
* Returns container DB file.
+ *
* @return dbFile
*/
public File getDbFile() {
@@ -185,7 +179,8 @@ public File getDbFile() {
/**
* Returns container metadata path.
- * @return - Physical path where container file and checksum is stored.
+ *
+ * @return - Physical path where container file and checksum are stored.
*/
public String getMetadataPath() {
return metadataPath;
@@ -202,6 +197,7 @@ public void setMetadataPath(String path) {
/**
* Returns the path to base dir of the container.
+ *
* @return Path to base dir
*/
@Override
@@ -218,7 +214,7 @@ public long getBlockCommitSequenceId() {
}
/**
- * updates the blockCommitSequenceId.
+ * Updates the blockCommitSequenceId.
*/
public void updateBlockCommitSequenceId(long id) {
this.blockCommitSequenceId = id;
@@ -226,6 +222,7 @@ public void updateBlockCommitSequenceId(long id) {
/**
* Returns the DBType used for the container.
+ *
* @return containerDBType
*/
public String getContainerDBType() {
@@ -234,7 +231,6 @@ public String getContainerDBType() {
/**
* Sets the DBType used for the container.
- * @param containerDBType
*/
public void setContainerDBType(String containerDBType) {
this.containerDBType = containerDBType;
@@ -288,17 +284,37 @@ public void addToFinalizedBlockSet(long localID) {
finalizedBlockSet.add(localID);
}
+ /**
+ * Returns a set of finalized block IDs associated with the container data.
+ *
+ * @return a Set of Long values representing the IDs of finalized blocks.
+ */
public Set getFinalizedBlockSet() {
return finalizedBlockSet;
}
+ /**
+ * Checks if a block with the specified local ID exists in the finalized block set.
+ *
+ * @param localID The ID of the block to check for existence in the finalized block set.
+ * @return true if the block exists in the finalized block set, false otherwise.
+ */
public boolean isFinalizedBlockExist(long localID) {
return finalizedBlockSet.contains(localID);
}
+ /**
+ * Clears the set of finalized blocks from both memory and the database.
+ * This operation will remove all finalized blocks associated with the current container's prefix.
+ * It first checks if the finalized block set is not empty,
+ * then deletes the corresponding entries from the database using batch operations and clears the in-memory set.
+ *
+ * @param db The database handle to use it for the batch operations. It must not be {@code null}.
+ * @throws IOException If any I/O error occurs during the batch operations.
+ */
public void clearFinalizedBlock(DBHandle db) throws IOException {
if (!finalizedBlockSet.isEmpty()) {
- // delete from db and clear memory
+ // Delete it from db and clear memory.
// Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here");
try (BatchOperation batch = db.getStore().getBatchHandler().initBatchOperation()) {
@@ -324,10 +340,8 @@ public ContainerDataProto getProtoBufMessage() {
builder.setBlockCount(this.getBlockCount());
for (Map.Entry entry : getMetadata().entrySet()) {
- ContainerProtos.KeyValue.Builder keyValBuilder =
- ContainerProtos.KeyValue.newBuilder();
- builder.addMetadata(keyValBuilder.setKey(entry.getKey())
- .setValue(entry.getValue()).build());
+ ContainerProtos.KeyValue.Builder keyValBuilder = ContainerProtos.KeyValue.newBuilder();
+ builder.addMetadata(keyValBuilder.setKey(entry.getKey()).setValue(entry.getValue()).build());
}
if (this.getBytesUsed() >= 0) {
@@ -341,35 +355,44 @@ public ContainerDataProto getProtoBufMessage() {
return builder.build();
}
+ /**
+ * Returns an unmodifiable list of YAML field names used in the key-value container.
+ *
+ * @return a List of Strings representing the YAML field names.
+ */
public static List getYamlFields() {
return Collections.unmodifiableList(KV_YAML_FIELDS);
}
/**
* Update DB counters related to block metadata.
+ *
* @param db - Reference to container DB.
* @param batchOperation - Batch Operation to batch DB operations.
* @param deletedBlockCount - Number of blocks deleted.
* @param releasedBytes - Number of bytes released.
- * @throws IOException
*/
- public void updateAndCommitDBCounters(DBHandle db,
- BatchOperation batchOperation, int deletedBlockCount,
+ public void updateAndCommitDBCounters(DBHandle db, BatchOperation batchOperation, int deletedBlockCount,
long releasedBytes) throws IOException {
Table metadataTable = db.getStore().getMetadataTable();
// Set Bytes used and block count key.
- metadataTable.putWithBatch(batchOperation, getBytesUsedKey(),
- getBytesUsed() - releasedBytes);
- metadataTable.putWithBatch(batchOperation, getBlockCountKey(),
- getBlockCount() - deletedBlockCount);
- metadataTable.putWithBatch(batchOperation,
+ metadataTable.putWithBatch(batchOperation, getBytesUsedKey(), getBytesUsed() - releasedBytes);
+ metadataTable.putWithBatch(batchOperation, getBlockCountKey(), getBlockCount() - deletedBlockCount);
+ metadataTable.putWithBatch(
+ batchOperation,
getPendingDeleteBlockCountKey(),
getNumPendingDeletionBlocks() - deletedBlockCount);
db.getStore().getBatchHandler().commitBatchOperation(batchOperation);
}
+ /**
+ * Resets the count of pending deletion blocks to zero.
+ *
+ * @param db The database handle used to access the container's metadata table.
+ * @throws IOException If an I/O error occurs while updating the metadata table on disk.
+ */
public void resetPendingDeleteBlockCount(DBHandle db) throws IOException {
// Reset the in memory metadata.
numPendingDeletionBlocks.set(0);
@@ -378,59 +401,116 @@ public void resetPendingDeleteBlockCount(DBHandle db) throws IOException {
metadataTable.put(getPendingDeleteBlockCountKey(), 0L);
}
- // NOTE: Below are some helper functions to format keys according
- // to container schemas, we should use them instead of using
- // raw const variables defined.
+ // NOTE: Below are some helper functions to format keys according to container schemas,
+ // we should use them instead of using raw const variables defined.
+ /**
+ * Generates a formatted key for accessing block data using the provided local block ID.
+ *
+ * @param localID The local ID of the block within the container.
+ * @return The formatted key string specific to the container's schema.
+ */
public String getBlockKey(long localID) {
return formatKey(Long.toString(localID));
}
+ /**
+ * Generates a deleting block key string using the provided local ID.
+ *
+ * @param localID The local ID of the block within the container.
+ * @return The formatted key string specific to the container's schema with the deleting block key prefix.
+ */
public String getDeletingBlockKey(long localID) {
return formatKey(DELETING_KEY_PREFIX + localID);
}
+ /**
+ * Generates a formatted key for accessing delete transaction data using the provided transaction ID.
+ *
+ * @param txnID The ID of the transaction to be deleted.
+ * @return The formatted key string specific to the container's schema.
+ */
public String getDeleteTxnKey(long txnID) {
return formatKey(Long.toString(txnID));
}
+ /**
+ * Retrieves the formatted key for the latest delete transaction.
+ *
+ * @return A String representing the formatted key for the latest delete transaction.
+ */
public String getLatestDeleteTxnKey() {
return formatKey(DELETE_TRANSACTION_KEY);
}
+ /**
+ * Retrieves the formatted key specific to the Block Commit Sequence ID.
+ *
+ * @return A String representing the formatted key for the Block Commit Sequence ID.
+ */
public String getBcsIdKey() {
return formatKey(BLOCK_COMMIT_SEQUENCE_ID);
}
+ /**
+ * Retrieves the formatted key specific to the Block Count.
+ *
+ * @return A String representing the formatted key for the Block Count.
+ */
public String getBlockCountKey() {
return formatKey(BLOCK_COUNT);
}
+ /**
+ * Retrieves the formatted key specific to the bytes used in the container.
+ * This key is used to store or query the bytes used information from the database specific to the container's schema.
+ *
+ * @return A String representing the formatted key for bytes used.
+ */
public String getBytesUsedKey() {
return formatKey(CONTAINER_BYTES_USED);
}
+ /**
+ * Retrieves the formatted key specific to the Pending Delete Block Count.
+ *
+ * @return A String representing the formatted key for the Pending Delete Block Count.
+ */
public String getPendingDeleteBlockCountKey() {
return formatKey(PENDING_DELETE_BLOCK_COUNT);
}
+ /**
+ * Retrieves the key prefix used for deleting blocks within the container.
+ *
+ * @return A String representing the formatted key prefix specific to the container's schema for deleting blocks.
+ */
public String getDeletingBlockKeyPrefix() {
return formatKey(DELETING_KEY_PREFIX);
}
+ /**
+ * Returns a KeyPrefixFilter that is configured to filter out keys with the container's schema-specific prefix.
+ *
+ * @return a KeyPrefixFilter object that filters out keys using the container's schema-specific prefix.
+ */
public KeyPrefixFilter getUnprefixedKeyFilter() {
String schemaPrefix = containerPrefix();
return new KeyPrefixFilter().addFilter(schemaPrefix + "#", true);
}
+ /**
+ * Generates and returns a {@link KeyPrefixFilter}
+ * configured to filter out keys that have the prefix used for deleting blocks within the container.
+ *
+ * @return a KeyPrefixFilter object configured to filter keys with the deleting block key prefix.
+ */
public KeyPrefixFilter getDeletingBlockKeyFilter() {
return new KeyPrefixFilter().addFilter(getDeletingBlockKeyPrefix());
}
/**
- * Schema v3 use a prefix as startKey,
- * for other schemas just return null.
- * @return
+ * Schema v3 use a prefix as startKey, for other schemas return {@code null}.
*/
public String startKeyEmpty() {
if (hasSchema(SCHEMA_V3)) {
@@ -440,9 +520,7 @@ public String startKeyEmpty() {
}
/**
- * Schema v3 use containerID as key prefix,
- * for other schemas just return null.
- * @return
+ * Schema v3 use containerID as key prefix, for other schemas {@code null}.
*/
public String containerPrefix() {
if (hasSchema(SCHEMA_V3)) {
@@ -452,9 +530,9 @@ public String containerPrefix() {
}
/**
- * Format the raw key to a schema specific format key.
- * Schema v3 use container ID as key prefix,
- * for other schemas just return the raw key.
+ * Format the raw key to a schema-specific format key.
+ * Schema v3 uses container ID as key prefix, for other schemas return the raw key.
+ *
* @param key raw key
* @return formatted key
*/
@@ -465,8 +543,13 @@ private String formatKey(String key) {
return key;
}
+ /**
+ * Checks if the provided version matches the schema version of the container.
+ *
+ * @param version The schema version to compare against the container's schema version.
+ * @return true if the provided version matches the container's schema version, false otherwise.
+ */
public boolean hasSchema(String version) {
return KeyValueContainerUtil.isSameSchemaVersion(schemaVersion, version);
}
-
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 1bcb64200b2..844ad7f29cf 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -18,11 +18,7 @@
package org.apache.hadoop.ozone.container.keyvalue;
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
+import java.io.*;
import java.nio.ByteBuffer;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
@@ -56,6 +52,8 @@
import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
import org.apache.hadoop.hdds.utils.FaultInjector;
import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.apache.hadoop.hdfs.util.Canceler;
+import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.common.Checksum;
import org.apache.hadoop.ozone.common.ChunkBuffer;
@@ -65,9 +63,7 @@
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.*;
import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
@@ -88,6 +84,7 @@
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerFactory;
import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.ozoneimpl.ContainerScannerConfiguration;
import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
import com.google.common.annotations.VisibleForTesting;
@@ -124,6 +121,10 @@
import static org.apache.hadoop.ozone.ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST;
import static org.apache.hadoop.ozone.OzoneConsts.INCREMENTAL_CHUNK_LIST;
import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.FailureType.*;
+import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult.unhealthy;
+import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerCheck.BUFFER_POOL;
+import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerCheck.verifyChecksum;
import org.apache.hadoop.util.Time;
import org.apache.ratis.statemachine.StateMachine;
@@ -136,8 +137,7 @@
*/
public class KeyValueHandler extends Handler {
- public static final Logger LOG = LoggerFactory.getLogger(
- KeyValueHandler.class);
+ public static final Logger LOG = LoggerFactory.getLogger(KeyValueHandler.class);
private final BlockManager blockManager;
private final ChunkManager chunkManager;
@@ -149,18 +149,13 @@ public class KeyValueHandler extends Handler {
private final Striped containerCreationLocks;
private static FaultInjector injector;
- public KeyValueHandler(ConfigurationSource config,
- String datanodeId,
- ContainerSet contSet,
- VolumeSet volSet,
- ContainerMetrics metrics,
- IncrementalReportSender icrSender) {
+ public KeyValueHandler(ConfigurationSource config, String datanodeId, ContainerSet contSet, VolumeSet volSet,
+ ContainerMetrics metrics, IncrementalReportSender icrSender) {
super(config, datanodeId, contSet, volSet, metrics, icrSender);
blockManager = new BlockManagerImpl(config);
- validateChunkChecksumData = conf.getObject(
- DatanodeConfiguration.class).isChunkDataValidationCheck();
- chunkManager = ChunkManagerFactory.createChunkManager(config, blockManager,
- volSet);
+ validateChunkChecksumData = conf.getObject(DatanodeConfiguration.class).isChunkDataValidationCheck();
+ chunkManager = ChunkManagerFactory.createChunkManager(config, blockManager, volSet);
+
try {
volumeChoosingPolicy = VolumeChoosingPolicyFactory.getPolicy(conf);
} catch (Exception e) {
@@ -170,49 +165,44 @@ public KeyValueHandler(ConfigurationSource config,
maxContainerSize = (long) config.getStorageSize(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
- // this striped handler lock is used for synchronizing createContainer
- // Requests.
+ // This striped handler lock is used for synchronizing createContainer Requests.
final int threadCountPerDisk = conf.getInt(
- OzoneConfigKeys
- .HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_KEY,
- OzoneConfigKeys
- .HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT);
- final int numberOfDisks =
- HddsServerUtil.getDatanodeStorageDirs(conf).size();
- containerCreationLocks = Striped.lazyWeakLock(
- threadCountPerDisk * numberOfDisks);
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_KEY,
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT);
+
+ final int numberOfDisks = HddsServerUtil.getDatanodeStorageDirs(conf).size();
+ containerCreationLocks = Striped.lazyWeakLock(threadCountPerDisk * numberOfDisks);
boolean isUnsafeByteBufferConversionEnabled =
conf.getBoolean(
OzoneConfigKeys.OZONE_UNSAFEBYTEOPERATIONS_ENABLED,
OzoneConfigKeys.OZONE_UNSAFEBYTEOPERATIONS_ENABLED_DEFAULT);
- byteBufferToByteString =
- ByteStringConversion
- .createByteBufferConversion(isUnsafeByteBufferConversionEnabled);
+ byteBufferToByteString = ByteStringConversion.createByteBufferConversion(isUnsafeByteBufferConversionEnabled);
}
+ /**
+ * Returns the {@link VolumeChoosingPolicy} used for testing purposes.
+ *
+ * @return the instance of VolumeChoosingPolicy.
+ */
@VisibleForTesting
public VolumeChoosingPolicy getVolumeChoosingPolicyForTesting() {
return volumeChoosingPolicy;
}
@Override
- public StateMachine.DataChannel getStreamDataChannel(
- Container container, ContainerCommandRequestProto msg)
+ public StateMachine.DataChannel getStreamDataChannel(Container container, ContainerCommandRequestProto msg)
throws StorageContainerException {
KeyValueContainer kvContainer = (KeyValueContainer) container;
checkContainerOpen(kvContainer);
if (msg.hasWriteChunk()) {
- BlockID blockID =
- BlockID.getFromProtobuf(msg.getWriteChunk().getBlockID());
+ BlockID blockID = BlockID.getFromProtobuf(msg.getWriteChunk().getBlockID());
- return chunkManager.getStreamDataChannel(kvContainer,
- blockID, metrics);
+ return chunkManager.getStreamDataChannel(kvContainer, blockID, metrics);
} else {
- throw new StorageContainerException("Malformed request.",
- ContainerProtos.Result.IO_EXCEPTION);
+ throw new StorageContainerException("Malformed request.", ContainerProtos.Result.IO_EXCEPTION);
}
}
@@ -223,25 +213,19 @@ public void stop() {
}
@Override
- public ContainerCommandResponseProto handle(
- ContainerCommandRequestProto request, Container container,
+ public ContainerCommandResponseProto handle(ContainerCommandRequestProto request, Container container,
DispatcherContext dispatcherContext) {
try {
- return KeyValueHandler
- .dispatchRequest(this, request, (KeyValueContainer) container,
- dispatcherContext);
+ return KeyValueHandler.dispatchRequest(this, request, (KeyValueContainer) container, dispatcherContext);
} catch (RuntimeException e) {
- return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException(e, CONTAINER_INTERNAL_ERROR),
- request);
+ return ContainerUtils.logAndReturnError(LOG, new StorageContainerException(e, CONTAINER_INTERNAL_ERROR), request);
}
}
@VisibleForTesting
- static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler,
- ContainerCommandRequestProto request, KeyValueContainer kvContainer,
- DispatcherContext dispatcherContext) {
+ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler, ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer, DispatcherContext dispatcherContext) {
Type cmdType = request.getCmdType();
switch (cmdType) {
@@ -254,6 +238,8 @@ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler,
case DeleteContainer:
return handler.handleDeleteContainer(request, kvContainer);
case ListContainer:
+ case ListChunk:
+ case CompactChunk:
return handler.handleUnsupportedOp(request);
case CloseContainer:
return handler.handleCloseContainer(request, kvContainer);
@@ -262,24 +248,19 @@ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler,
case GetBlock:
return handler.handleGetBlock(request, kvContainer);
case DeleteBlock:
- return handler.handleDeleteBlock(request, kvContainer);
+ return handler.handleDeleteBlock();
case ListBlock:
return handler.handleListBlock(request, kvContainer);
case ReadChunk:
return handler.handleReadChunk(request, kvContainer, dispatcherContext);
case DeleteChunk:
- return handler.handleDeleteChunk(request, kvContainer);
+ return handler.handleDeleteChunk();
case WriteChunk:
return handler.handleWriteChunk(request, kvContainer, dispatcherContext);
case StreamInit:
- return handler.handleStreamInit(request, kvContainer, dispatcherContext);
- case ListChunk:
- return handler.handleUnsupportedOp(request);
- case CompactChunk:
- return handler.handleUnsupportedOp(request);
+ return handler.handleStreamInit(request, kvContainer);
case PutSmallFile:
- return handler
- .handlePutSmallFile(request, kvContainer, dispatcherContext);
+ return handler.handlePutSmallFile(request, kvContainer, dispatcherContext);
case GetSmallFile:
return handler.handleGetSmallFile(request, kvContainer);
case GetCommittedBlockLength:
@@ -287,42 +268,48 @@ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler,
case FinalizeBlock:
return handler.handleFinalizeBlock(request, kvContainer);
case Echo:
- return handler.handleEcho(request, kvContainer);
+ return handler.handleEcho(request);
+ case VerifyBlock:
+ return handler.handleVerifyBlock(request, kvContainer);
default:
return null;
}
}
+ /**
+ * Returns the instance of {@link ChunkManager} used within the {@link KeyValueHandler}.
+ *
+ * @return the instance of {@link ChunkManager}.
+ */
@VisibleForTesting
public ChunkManager getChunkManager() {
return this.chunkManager;
}
+ /**
+ * Returns the instance of {@link BlockManager} used within the {@link KeyValueHandler}.
+ *
+ * @return the instance of {@link BlockManager}.
+ */
@VisibleForTesting
public BlockManager getBlockManager() {
return this.blockManager;
}
- ContainerCommandResponseProto handleStreamInit(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer,
- DispatcherContext dispatcherContext) {
+ ContainerCommandResponseProto handleStreamInit(ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
final BlockID blockID;
if (request.hasWriteChunk()) {
WriteChunkRequestProto writeChunk = request.getWriteChunk();
blockID = BlockID.getFromProtobuf(writeChunk.getBlockID());
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed {} request. trace ID: {}",
- request.getCmdType(), request.getTraceID());
- }
+ LOG.debug("Malformed {} request. trace ID: {}", request.getCmdType(), request.getTraceID());
return malformedRequest(request);
}
- String path = null;
+ String path;
try {
checkContainerOpen(kvContainer);
- path = chunkManager
- .streamInit(kvContainer, blockID);
+ path = chunkManager.streamInit(kvContainer, blockID);
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
}
@@ -333,45 +320,40 @@ ContainerCommandResponseProto handleStreamInit(
}
/**
- * Handles Create Container Request. If successful, adds the container to
- * ContainerSet and sends an ICR to the SCM.
+ * Handles Create Container Request.
+ * If successful, add the container to {@link ContainerSet} and send an ICR to the SCM.
*/
ContainerCommandResponseProto handleCreateContainer(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasCreateContainer()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Create Container request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Create Container request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
- // Create Container request should be passed a null container as the
- // container would be created here.
+ // Create Container request should be passed a null container as the container would be created here.
if (kvContainer != null) {
return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException(
- "Container creation failed because " + "key value container" +
+ new StorageContainerException("Container creation failed because " + "key value container" +
" already exists", null, CONTAINER_ALREADY_EXISTS), request);
}
long containerID = request.getContainerID();
- ContainerLayoutVersion layoutVersion =
- ContainerLayoutVersion.getConfiguredVersion(conf);
+ ContainerLayoutVersion layoutVersion = ContainerLayoutVersion.getConfiguredVersion(conf);
KeyValueContainerData newContainerData = new KeyValueContainerData(
- containerID, layoutVersion, maxContainerSize, request.getPipelineID(),
+ containerID,
+ layoutVersion,
+ maxContainerSize,
+ request.getPipelineID(),
getDatanodeId());
State state = request.getCreateContainer().getState();
if (state != null) {
newContainerData.setState(state);
}
- newContainerData.setReplicaIndex(request.getCreateContainer()
- .getReplicaIndex());
+ newContainerData.setReplicaIndex(request.getCreateContainer().getReplicaIndex());
- // TODO: Add support to add metadataList to ContainerData. Add metadata
- // to container during creation.
- KeyValueContainer newContainer = new KeyValueContainer(
- newContainerData, conf);
+ // TODO: Add support to add metadataList to ContainerData.
+ // Add metadata to container during creation.
+ KeyValueContainer newContainer = new KeyValueContainer(newContainerData, conf);
boolean created = false;
Lock containerIdLock = containerCreationLocks.get(containerID);
@@ -381,9 +363,9 @@ ContainerCommandResponseProto handleCreateContainer(
newContainer.create(volumeSet, volumeChoosingPolicy, clusterId);
created = containerSet.addContainer(newContainer);
} else {
- // The create container request for an already existing container can
- // arrive in case the ContainerStateMachine reapplies the transaction
- // on datanode restart. Just log a warning msg here.
+ // The creation container request for an already existing container can arrive
+ // in case the ContainerStateMachine reapplied the transaction on datanode restart.
+ // Just log a warning msg here.
LOG.debug("Container already exists. container Id {}", containerID);
}
} catch (StorageContainerException ex) {
@@ -404,12 +386,10 @@ ContainerCommandResponseProto handleCreateContainer(
return getSuccessResponse(request);
}
- private void populateContainerPathFields(KeyValueContainer container,
- HddsVolume hddsVolume) throws IOException {
+ private void populateContainerPathFields(KeyValueContainer container, HddsVolume hddsVolume) throws IOException {
volumeSet.readLock();
try {
- String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
- hddsVolume, clusterId);
+ String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(hddsVolume, clusterId);
container.populatePathFields(idDir, hddsVolume);
} finally {
volumeSet.readUnlock();
@@ -422,10 +402,7 @@ private void populateContainerPathFields(KeyValueContainer container,
ContainerCommandResponseProto handleReadContainer(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasReadContainer()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Read Container request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Read Container request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
@@ -436,23 +413,18 @@ ContainerCommandResponseProto handleReadContainer(
/**
- * Handles Update Container Request. If successful, the container metadata
- * is updated.
+ * Handles Update Container Request. If successful, the container metadata is updated.
*/
ContainerCommandResponseProto handleUpdateContainer(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasUpdateContainer()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Update Container request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Update Container request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
boolean forceUpdate = request.getUpdateContainer().getForceUpdate();
- List keyValueList =
- request.getUpdateContainer().getMetadataList();
+ List keyValueList = request.getUpdateContainer().getMetadataList();
Map metadata = new HashMap<>();
for (KeyValue keyValue : keyValueList) {
metadata.put(keyValue.getKey(), keyValue.getValue());
@@ -471,18 +443,14 @@ ContainerCommandResponseProto handleUpdateContainer(
/**
* Handles Delete Container Request.
* Open containers cannot be deleted.
- * Holds writeLock on ContainerSet till the container is removed from
- * containerMap. On disk deletion of container files will happen
- * asynchronously without the lock.
+ * Holds writeLock on ContainerSet till the container is removed from containerMap.
+ * On disk deletion of container files will happen asynchronously without the lock.
*/
- ContainerCommandResponseProto handleDeleteContainer(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleDeleteContainer(ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer) {
if (!request.hasDeleteContainer()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Delete container request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Delete container request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
@@ -499,14 +467,11 @@ ContainerCommandResponseProto handleDeleteContainer(
* Handles Close Container Request. An open container is closed.
* Close Container call is idempotent.
*/
- ContainerCommandResponseProto handleCloseContainer(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleCloseContainer(ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer) {
if (!request.hasCloseContainer()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Update Container request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Update Container request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
try {
@@ -516,8 +481,7 @@ ContainerCommandResponseProto handleCloseContainer(
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException("Close Container failed", ex,
- IO_EXCEPTION), request);
+ new StorageContainerException("Close Container failed", ex, IO_EXCEPTION), request);
}
return getSuccessResponse(request);
@@ -526,15 +490,11 @@ ContainerCommandResponseProto handleCloseContainer(
/**
* Handle Put Block operation. Calls BlockManager to process the request.
*/
- ContainerCommandResponseProto handlePutBlock(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer,
+ ContainerCommandResponseProto handlePutBlock(ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasPutBlock()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Put Key request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Put Key request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
@@ -548,20 +508,18 @@ ContainerCommandResponseProto handlePutBlock(
boolean endOfBlock = false;
if (!request.getPutBlock().hasEof() || request.getPutBlock().getEof()) {
- // There are two cases where client sends empty put block with eof.
- // (1) An EC empty file. In this case, the block/chunk file does not exist,
- // so no need to flush/close the file.
+ // There are two cases where a client sends an empty put block with eof.
+ // (1) An EC empty file. In this case, the block/chunk file does not exist, so no need to flush/close the file.
// (2) Ratis output stream in incremental chunk list mode may send empty put block
// to close the block, in which case we need to flush/close the file.
- if (!request.getPutBlock().getBlockData().getChunksList().isEmpty() ||
- blockData.getMetadata().containsKey(INCREMENTAL_CHUNK_LIST)) {
+ if (!request.getPutBlock().getBlockData().getChunksList().isEmpty()
+ || blockData.getMetadata().containsKey(INCREMENTAL_CHUNK_LIST)) {
chunkManager.finishWriteChunks(kvContainer, blockData);
}
endOfBlock = true;
}
- long bcsId =
- dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
+ long bcsId = dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
blockData.setBlockCommitSequenceId(bcsId);
blockManager.putBlock(kvContainer, blockData, endOfBlock);
@@ -572,72 +530,66 @@ ContainerCommandResponseProto handlePutBlock(
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
- return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException("Put Key failed", ex, IO_EXCEPTION),
+ return ContainerUtils.logAndReturnError(LOG, new StorageContainerException("Put Key failed", ex, IO_EXCEPTION),
request);
}
return putBlockResponseSuccess(request, blockDataProto);
}
- ContainerCommandResponseProto handleFinalizeBlock(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleFinalizeBlock(ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer) {
ContainerCommandResponseProto responseProto = checkFaultInjector(request);
if (responseProto != null) {
return responseProto;
}
if (!request.hasFinalizeBlock()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Finalize block request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Finalize block request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
ContainerProtos.BlockData responseData;
try {
if (!VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.HBASE_SUPPORT)) {
- throw new StorageContainerException("DataNode has not finalized " +
- "upgrading to a version that supports block finalization.", UNSUPPORTED_REQUEST);
+ throw new StorageContainerException(
+ "DataNode has not finalized upgrading to a version that supports block finalization.",
+ UNSUPPORTED_REQUEST);
}
checkContainerOpen(kvContainer);
- BlockID blockID = BlockID.getFromProtobuf(
- request.getFinalizeBlock().getBlockID());
+ BlockID blockID = BlockID.getFromProtobuf(request.getFinalizeBlock().getBlockID());
Preconditions.checkNotNull(blockID);
LOG.info("Finalized Block request received {} ", blockID);
- responseData = blockManager.getBlock(kvContainer, blockID)
- .getProtoBufMessage();
+ responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage();
chunkManager.finalizeWriteChunk(kvContainer, blockID);
blockManager.finalizeBlock(kvContainer, blockID);
- kvContainer.getContainerData()
- .addToFinalizedBlockSet(blockID.getLocalID());
+ kvContainer.getContainerData().addToFinalizedBlockSet(blockID.getLocalID());
LOG.info("Block has been finalized {} ", blockID);
-
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
- return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException(
- "Finalize Block failed", ex, IO_EXCEPTION), request);
+ return ContainerUtils.logAndReturnError(
+ LOG,
+ new StorageContainerException("Finalize Block failed", ex, IO_EXCEPTION),
+ request);
}
return getFinalizeBlockResponse(request, responseData);
}
- ContainerCommandResponseProto handleEcho(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleEcho(ContainerCommandRequestProto request) {
return getEchoResponse(request);
}
/**
* Checks if a replicaIndex needs to be checked based on the client version for a request.
+ *
* @param request ContainerCommandRequest object.
- * @return true if the validation is required for the client version else false.
+ * @return {@code true} if the validation is required for the client version else {@code false}.
*/
private boolean replicaIndexCheckRequired(ContainerCommandRequestProto request) {
return request.hasVersion() && request.getVersion() >= EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue();
@@ -650,29 +602,23 @@ ContainerCommandResponseProto handleGetBlock(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasGetBlock()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Get Key request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Get Key request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
ContainerProtos.BlockData responseData;
try {
- BlockID blockID = BlockID.getFromProtobuf(
- request.getGetBlock().getBlockID());
+ BlockID blockID = BlockID.getFromProtobuf(request.getGetBlock().getBlockID());
if (replicaIndexCheckRequired(request)) {
BlockUtils.verifyReplicaIdx(kvContainer, blockID);
}
responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage();
final long numBytes = responseData.getSerializedSize();
metrics.incContainerBytesStats(Type.GetBlock, numBytes);
-
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
- return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException("Get Key failed", ex, IO_EXCEPTION),
+ return ContainerUtils.logAndReturnError(LOG, new StorageContainerException("Get Key failed", ex, IO_EXCEPTION),
request);
}
@@ -683,8 +629,8 @@ ContainerCommandResponseProto handleGetBlock(
* Handles GetCommittedBlockLength operation.
* Calls BlockManager to process the request.
*/
- ContainerCommandResponseProto handleGetCommittedBlockLength(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleGetCommittedBlockLength(ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer) {
ContainerCommandResponseProto responseProto = checkFaultInjector(request);
if (responseProto != null) {
@@ -692,25 +638,22 @@ ContainerCommandResponseProto handleGetCommittedBlockLength(
}
if (!request.hasGetCommittedBlockLength()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Get Key request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Get Key request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
long blockLength;
try {
- BlockID blockID = BlockID
- .getFromProtobuf(request.getGetCommittedBlockLength().getBlockID());
+ BlockID blockID = BlockID.getFromProtobuf(request.getGetCommittedBlockLength().getBlockID());
BlockUtils.verifyBCSId(kvContainer, blockID);
blockLength = blockManager.getCommittedBlockLength(kvContainer, blockID);
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
- return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException("GetCommittedBlockLength failed", ex,
- IO_EXCEPTION), request);
+ return ContainerUtils.logAndReturnError(
+ LOG,
+ new StorageContainerException("GetCommittedBlockLength failed", ex, IO_EXCEPTION),
+ request);
}
return getBlockLengthResponse(request, blockLength);
@@ -719,14 +662,10 @@ ContainerCommandResponseProto handleGetCommittedBlockLength(
/**
* Handle List Block operation. Calls BlockManager to process the request.
*/
- ContainerCommandResponseProto handleListBlock(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleListBlock(ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasListBlock()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed list block request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed list block request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
@@ -737,8 +676,7 @@ ContainerCommandResponseProto handleListBlock(
if (request.getListBlock().hasStartLocalID()) {
startLocalId = request.getListBlock().getStartLocalID();
}
- List responseData =
- blockManager.listBlock(kvContainer, startLocalId, count);
+ List responseData = blockManager.listBlock(kvContainer, startLocalId, count);
for (BlockData responseDatum : responseData) {
returnData.add(responseDatum.getProtoBufMessage());
}
@@ -754,37 +692,31 @@ ContainerCommandResponseProto handleListBlock(
}
/**
- * Handle Delete Block operation. Calls BlockManager to process the request.
+ * Handle Delete Block operation.
+ * Calls BlockManager to process the request.
*/
@Deprecated
- ContainerCommandResponseProto handleDeleteBlock(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleDeleteBlock() {
// Block/ Chunk Deletion is handled by BlockDeletingService.
// SCM sends Block Deletion commands directly to Datanodes and not
// through a Pipeline.
- throw new UnsupportedOperationException("Datanode handles block deletion " +
- "using BlockDeletingService");
+ throw new UnsupportedOperationException("Datanode handles block deletion using BlockDeletingService");
}
/**
- * Handle Read Chunk operation. Calls ChunkManager to process the request.
+ * Handle Read Chunk operation.
+ * Calls ChunkManager to process the request.
*/
- ContainerCommandResponseProto handleReadChunk(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer,
+ ContainerCommandResponseProto handleReadChunk(ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasReadChunk()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Read Chunk request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Read Chunk request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
ChunkBuffer data;
try {
- BlockID blockID = BlockID.getFromProtobuf(
- request.getReadChunk().getBlockID());
- ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk()
- .getChunkData());
+ BlockID blockID = BlockID.getFromProtobuf(request.getReadChunk().getBlockID());
+ ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk().getChunkData());
Preconditions.checkNotNull(chunkInfo);
if (replicaIndexCheckRequired(request)) {
BlockUtils.verifyReplicaIdx(kvContainer, blockID);
@@ -795,21 +727,18 @@ ContainerCommandResponseProto handleReadChunk(
dispatcherContext = DispatcherContext.getHandleReadChunk();
}
- boolean isReadChunkV0 = getReadChunkVersion(request.getReadChunk())
- .equals(ContainerProtos.ReadChunkVersion.V0);
+ boolean isReadChunkV0 = getReadChunkVersion(request.getReadChunk()).equals(ContainerProtos.ReadChunkVersion.V0);
if (isReadChunkV0) {
- // For older clients, set ReadDataIntoSingleBuffer to true so that
- // all the data read from chunk file is returned as a single
- // ByteString. Older clients cannot process data returned as a list
- // of ByteStrings.
+ // For older clients,
+ // set ReadDataIntoSingleBuffer to true
+ // so that all the data read from a chunk file is returned as a single ByteString.
+ // Older clients cannot process data returned as a list of ByteStrings.
chunkInfo.setReadDataIntoSingleBuffer(true);
}
- data = chunkManager.readChunk(kvContainer, blockID, chunkInfo,
- dispatcherContext);
+ data = chunkManager.readChunk(kvContainer, blockID, chunkInfo, dispatcherContext);
LOG.debug("read chunk from block {} chunk {}", blockID, chunkInfo);
- // Validate data only if the read chunk is issued by Ratis for its
- // internal logic.
+ // Validate data only if the read chunk is issued by Ratis for its internal logic.
// For client reads, the client is expected to validate.
if (DispatcherContext.op(dispatcherContext).readFromTmpFile()) {
validateChunkChecksumData(data, chunkInfo);
@@ -834,13 +763,10 @@ ContainerCommandResponseProto handleReadChunk(
* Handle Delete Chunk operation. Calls ChunkManager to process the request.
*/
@Deprecated
- ContainerCommandResponseProto handleDeleteChunk(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleDeleteChunk() {
// Block/ Chunk Deletion is handled by BlockDeletingService.
- // SCM sends Block Deletion commands directly to Datanodes and not
- // through a Pipeline.
- throw new UnsupportedOperationException("Datanode handles chunk deletion " +
- "using BlockDeletingService");
+ // SCM sends Block Deletion commands directly to Datanodes and not through a Pipeline.
+ throw new UnsupportedOperationException("Datanode handles chunk deletion using BlockDeletingService");
}
private void validateChunkChecksumData(ChunkBuffer data, ChunkInfo info)
@@ -855,17 +781,14 @@ private void validateChunkChecksumData(ChunkBuffer data, ChunkInfo info)
}
/**
- * Handle Write Chunk operation. Calls ChunkManager to process the request.
+ * Handle Write Chunk operation.
+ * Calls ChunkManager to process the request.
*/
- ContainerCommandResponseProto handleWriteChunk(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer,
+ ContainerCommandResponseProto handleWriteChunk(ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasWriteChunk()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Write Chunk request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Write Chunk request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
@@ -886,26 +809,20 @@ ContainerCommandResponseProto handleWriteChunk(
}
final boolean isWrite = dispatcherContext.getStage().isWrite();
if (isWrite) {
- data =
- ChunkBuffer.wrap(writeChunk.getData().asReadOnlyByteBufferList());
+ data = ChunkBuffer.wrap(writeChunk.getData().asReadOnlyByteBufferList());
validateChunkChecksumData(data, chunkInfo);
}
- chunkManager
- .writeChunk(kvContainer, blockID, chunkInfo, data, dispatcherContext);
+ chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data, dispatcherContext);
final boolean isCommit = dispatcherContext.getStage().isCommit();
if (isCommit && writeChunk.hasBlock()) {
long startTime = Time.monotonicNowNanos();
metrics.incContainerOpsMetrics(Type.PutBlock);
- BlockData blockData = BlockData.getFromProtoBuf(
- writeChunk.getBlock().getBlockData());
- // optimization for hsync when WriteChunk is in commit phase:
- //
- // block metadata is piggybacked in the same message.
- // there will not be an additional PutBlock request.
- //
- // do not do this in WRITE_DATA phase otherwise PutBlock will be out
- // of order.
+ BlockData blockData = BlockData.getFromProtoBuf(writeChunk.getBlock().getBlockData());
+ // Optimization for hsync when WriteChunk is in commit phase:
+ // Block metadata is piggybacked in the same message.
+ // There will not be an additional PutBlock request.
+ // Do not do this in the WRITE_DATA phase, otherwise PutBlock will be out of order.
blockData.setBlockCommitSequenceId(dispatcherContext.getLogIndex());
boolean eob = writeChunk.getBlock().getEof();
if (eob) {
@@ -920,8 +837,7 @@ ContainerCommandResponseProto handleWriteChunk(
// We should increment stats after writeChunk
if (isWrite) {
- metrics.incContainerBytesStats(Type.WriteChunk, writeChunk
- .getChunkData().getLen());
+ metrics.incContainerBytesStats(Type.WriteChunk, writeChunk.getChunkData().getLen());
}
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
@@ -935,19 +851,15 @@ ContainerCommandResponseProto handleWriteChunk(
}
/**
- * Handle Put Small File operation. Writes the chunk and associated key
- * using a single RPC. Calls BlockManager and ChunkManager to process the
- * request.
+ * Handle Put Small File operation.
+ * Writes the chunk and associated key using a single RPC.
+ * Call BlockManager and ChunkManager to process the request.
*/
- ContainerCommandResponseProto handlePutSmallFile(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer,
+ ContainerCommandResponseProto handlePutSmallFile(ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasPutSmallFile()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Put Small File request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Put Small File request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
@@ -956,27 +868,24 @@ ContainerCommandResponseProto handlePutSmallFile(
try {
checkContainerOpen(kvContainer);
- BlockData blockData = BlockData.getFromProtoBuf(
- putSmallFileReq.getBlock().getBlockData());
+ BlockData blockData = BlockData.getFromProtoBuf(putSmallFileReq.getBlock().getBlockData());
Preconditions.checkNotNull(blockData);
ContainerProtos.ChunkInfo chunkInfoProto = putSmallFileReq.getChunkInfo();
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfoProto);
Preconditions.checkNotNull(chunkInfo);
- ChunkBuffer data = ChunkBuffer.wrap(
- putSmallFileReq.getData().asReadOnlyByteBufferList());
+ ChunkBuffer data = ChunkBuffer.wrap(putSmallFileReq.getData().asReadOnlyByteBufferList());
if (dispatcherContext == null) {
dispatcherContext = DispatcherContext.getHandlePutSmallFile();
}
BlockID blockID = blockData.getBlockID();
- // chunks will be committed as a part of handling putSmallFile
- // here. There is no need to maintain this info in openContainerBlockMap.
+ // Chunks will be committed as a part of handling putSmallFile here.
+ // There is no need to maintain this info in openContainerBlockMap.
validateChunkChecksumData(data, chunkInfo);
- chunkManager
- .writeChunk(kvContainer, blockID, chunkInfo, data, dispatcherContext);
+ chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data, dispatcherContext);
chunkManager.finishWriteChunks(kvContainer, blockData);
List chunks = new LinkedList<>();
@@ -992,107 +901,92 @@ ContainerCommandResponseProto handlePutSmallFile(
return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) {
return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException("Read Chunk failed", ex,
- PUT_SMALL_FILE_ERROR), request);
+ new StorageContainerException("Read Chunk failed", ex, PUT_SMALL_FILE_ERROR),
+ request);
}
return getPutFileResponseSuccess(request, blockDataProto);
}
/**
- * Handle Get Small File operation. Gets a data stream using a key. This
- * helps in reducing the RPC overhead for small files. Calls BlockManager and
- * ChunkManager to process the request.
+ * Handle Get Small File operation.
+ * Gets a data stream using a key.
+ * This helps in reducing the RPC overhead for small files.
+ * Call BlockManager and ChunkManager to process the request.
*/
- ContainerCommandResponseProto handleGetSmallFile(
- ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+ ContainerCommandResponseProto handleGetSmallFile(ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer) {
if (!request.hasGetSmallFile()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Malformed Get Small File request. trace ID: {}",
- request.getTraceID());
- }
+ LOG.debug("Malformed Get Small File request. trace ID: {}", request.getTraceID());
return malformedRequest(request);
}
GetSmallFileRequestProto getSmallFileReq = request.getGetSmallFile();
try {
- BlockID blockID = BlockID.getFromProtobuf(getSmallFileReq.getBlock()
- .getBlockID());
+ BlockID blockID = BlockID.getFromProtobuf(getSmallFileReq.getBlock().getBlockID());
BlockData responseData = blockManager.getBlock(kvContainer, blockID);
ContainerProtos.ChunkInfo chunkInfoProto = null;
List dataBuffers = new ArrayList<>();
- final DispatcherContext dispatcherContext
- = DispatcherContext.getHandleGetSmallFile();
+ final DispatcherContext dispatcherContext = DispatcherContext.getHandleGetSmallFile();
for (ContainerProtos.ChunkInfo chunk : responseData.getChunks()) {
- // if the block is committed, all chunks must have been committed.
+ // If the block is committed, all chunks must have been committed.
// Tmp chunk files won't exist here.
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunk);
boolean isReadChunkV0 = getReadChunkVersion(request.getGetSmallFile())
.equals(ContainerProtos.ReadChunkVersion.V0);
if (isReadChunkV0) {
- // For older clients, set ReadDataIntoSingleBuffer to true so that
- // all the data read from chunk file is returned as a single
- // ByteString. Older clients cannot process data returned as a list
- // of ByteStrings.
+ // For older clients,
+ // set ReadDataIntoSingleBuffer to true
+ // so that all the data read from a chunk file is returned as a single ByteString.
+ // Older clients cannot process data returned as a list of ByteStrings.
chunkInfo.setReadDataIntoSingleBuffer(true);
}
- ChunkBuffer data = chunkManager.readChunk(kvContainer, blockID,
- chunkInfo, dispatcherContext);
- dataBuffers.addAll(data.toByteStringList(byteBufferToByteString));
- chunkInfoProto = chunk;
+ try (ChunkBuffer data = chunkManager.readChunk(kvContainer, blockID, chunkInfo, dispatcherContext)) {
+ dataBuffers.addAll(data.toByteStringList(byteBufferToByteString));
+ chunkInfoProto = chunk;
+ }
}
- metrics.incContainerBytesStats(Type.GetSmallFile,
- BufferUtils.getBuffersLen(dataBuffers));
- return getGetSmallFileResponseSuccess(request, dataBuffers,
- chunkInfoProto);
+ metrics.incContainerBytesStats(Type.GetSmallFile, BufferUtils.getBuffersLen(dataBuffers));
+ return getGetSmallFileResponseSuccess(request, dataBuffers, chunkInfoProto);
} catch (StorageContainerException e) {
return ContainerUtils.logAndReturnError(LOG, e, request);
} catch (IOException ex) {
return ContainerUtils.logAndReturnError(LOG,
- new StorageContainerException("Write Chunk failed", ex,
- GET_SMALL_FILE_ERROR), request);
+ new StorageContainerException("Write Chunk failed", ex, GET_SMALL_FILE_ERROR),
+ request);
}
}
/**
* Handle unsupported operation.
*/
- ContainerCommandResponseProto handleUnsupportedOp(
- ContainerCommandRequestProto request) {
+ ContainerCommandResponseProto handleUnsupportedOp(ContainerCommandRequestProto request) {
// TODO : remove all unsupported operations or handle them.
return unsupportedRequest(request);
}
/**
- * Check if container is open. Throw exception otherwise.
- * @param kvContainer
- * @throws StorageContainerException
+ * Check if the container is open. Throw exception otherwise.
*/
- private void checkContainerOpen(KeyValueContainer kvContainer)
- throws StorageContainerException {
-
+ private void checkContainerOpen(KeyValueContainer kvContainer) throws StorageContainerException {
final State containerState = kvContainer.getContainerState();
/*
* In a closing state, follower will receive transactions from leader.
- * Once the leader is put to closing state, it will reject further requests
- * from clients. Only the transactions which happened before the container
- * in the leader goes to closing state, will arrive here even the container
- * might already be in closing state here.
+ * Once the leader is put to closing state, it will reject further requests from clients.
+ * Only the transactions which happened before the container in the leader goes to closing state,
+ * will arrive here even the container might already be in closing state here.
*/
- if (containerState == State.OPEN || containerState == State.CLOSING
- || containerState == State.RECOVERING) {
+ if (containerState == State.OPEN || containerState == State.CLOSING || containerState == State.RECOVERING) {
return;
}
final ContainerProtos.Result result;
switch (containerState) {
case QUASI_CLOSED:
- result = CLOSED_CONTAINER_IO;
- break;
case CLOSED:
result = CLOSED_CONTAINER_IO;
break;
@@ -1105,23 +999,19 @@ private void checkContainerOpen(KeyValueContainer kvContainer)
default:
result = CONTAINER_INTERNAL_ERROR;
}
- String msg = "Requested operation not allowed as ContainerState is " +
- containerState;
+ String msg = "Requested operation not allowed as ContainerState is " + containerState;
throw new StorageContainerException(msg, result);
}
@Override
- public Container importContainer(ContainerData originalContainerData,
- final InputStream rawContainerStream,
- final TarContainerPacker packer) throws IOException {
- Preconditions.checkState(originalContainerData instanceof
- KeyValueContainerData, "Should be KeyValueContainerData instance");
+ public Container importContainer(ContainerData originalContainerData, InputStream rawContainerStream,
+ TarContainerPacker packer) throws IOException {
+ Preconditions.checkState(originalContainerData instanceof KeyValueContainerData,
+ "Should be KeyValueContainerData instance");
- KeyValueContainerData containerData = new KeyValueContainerData(
- (KeyValueContainerData) originalContainerData);
+ KeyValueContainerData containerData = new KeyValueContainerData((KeyValueContainerData) originalContainerData);
- KeyValueContainer container = new KeyValueContainer(containerData,
- conf);
+ KeyValueContainer container = new KeyValueContainer(containerData, conf);
HddsVolume targetVolume = originalContainerData.getVolume();
populateContainerPathFields(container, targetVolume);
@@ -1129,13 +1019,10 @@ public Container importContainer(ContainerData originalContainerData,
ContainerLogger.logImported(containerData);
sendICR(container);
return container;
-
}
@Override
- public void exportContainer(final Container container,
- final OutputStream outputStream,
- final TarContainerPacker packer)
+ public void exportContainer(Container container, OutputStream outputStream, TarContainerPacker packer)
throws IOException {
final KeyValueContainer kvc = (KeyValueContainer) container;
kvc.exportContainerData(outputStream, packer);
@@ -1143,17 +1030,14 @@ public void exportContainer(final Container container,
}
@Override
- public void markContainerForClose(Container container)
- throws IOException {
+ public void markContainerForClose(Container container) throws IOException {
container.writeLock();
try {
- ContainerProtos.ContainerDataProto.State state =
- container.getContainerState();
+ ContainerProtos.ContainerDataProto.State state = container.getContainerState();
// Move the container to CLOSING state only if it's OPEN/RECOVERING
if (HddsUtils.isOpenToWriteState(state)) {
if (state == RECOVERING) {
- containerSet.removeRecoveringContainer(
- container.getContainerData().getContainerID());
+ containerSet.removeRecoveringContainer(container.getContainerData().getContainerID());
ContainerLogger.logRecovered(container.getContainerData());
}
container.markContainerForClose();
@@ -1166,35 +1050,32 @@ public void markContainerForClose(Container container)
}
@Override
- public void markContainerUnhealthy(Container container, ScanResult reason)
- throws StorageContainerException {
+ public void markContainerUnhealthy(Container container, ScanResult reason) throws StorageContainerException {
container.writeLock();
try {
long containerID = container.getContainerData().getContainerID();
if (container.getContainerState() == State.UNHEALTHY) {
- LOG.debug("Call to mark already unhealthy container {} as unhealthy",
- containerID);
+ LOG.debug("Call to mark already unhealthy container {} as unhealthy", containerID);
return;
}
- // If the volume is unhealthy, no action is needed. The container has
- // already been discarded and SCM notified. Once a volume is failed, it
- // cannot be restored without a restart.
+ // If the volume is unhealthy, no action is necessary.
+ // The container has already been discarded and SCM notified.
+ // Once a volume is failed, it cannot be restored without a restart.
HddsVolume containerVolume = container.getContainerData().getVolume();
if (containerVolume.isFailed()) {
- LOG.debug("Ignoring unhealthy container {} detected on an " +
- "already failed volume {}", containerID, containerVolume);
+ LOG.debug("Ignoring unhealthy container {} detected on an already failed volume {}",
+ containerID, containerVolume);
return;
}
try {
container.markContainerUnhealthy();
} catch (StorageContainerException ex) {
- LOG.warn("Unexpected error while marking container {} unhealthy",
- containerID, ex);
+ LOG.warn("Unexpected error while marking container {} unhealthy", containerID, ex);
} finally {
- // Even if the container file is corrupted/missing and the unhealthy
- // update fails, the unhealthy state is kept in memory and sent to
- // SCM. Write a corresponding entry to the container log as well.
+ // Even if the container file is corrupted/missing and the unhealthy update fails,
+ // the unhealthy state is kept in memory and sent to SCM.
+ // Write a corresponding entry to the container log as well.
ContainerLogger.logUnhealthy(container.getContainerData(), reason);
sendICR(container);
}
@@ -1204,8 +1085,7 @@ public void markContainerUnhealthy(Container container, ScanResult reason)
}
@Override
- public void quasiCloseContainer(Container container, String reason)
- throws IOException {
+ public void quasiCloseContainer(Container container, String reason) throws IOException {
container.writeLock();
try {
final State state = container.getContainerState();
@@ -1215,12 +1095,12 @@ public void quasiCloseContainer(Container container, String reason)
}
// The container has to be in CLOSING state.
if (state != State.CLOSING) {
- ContainerProtos.Result error =
- state == State.INVALID ? INVALID_CONTAINER_STATE :
- CONTAINER_INTERNAL_ERROR;
+ ContainerProtos.Result error = state == State.INVALID ? INVALID_CONTAINER_STATE : CONTAINER_INTERNAL_ERROR;
throw new StorageContainerException(
- "Cannot quasi close container #" + container.getContainerData()
- .getContainerID() + " while in " + state + " state.", error);
+ String.format("Cannot quasi close container #%s while in %s state.",
+ container.getContainerData().getContainerID(),
+ state),
+ error);
}
container.quasiClose();
ContainerLogger.logQuasiClosed(container.getContainerData(), reason);
@@ -1231,8 +1111,7 @@ public void quasiCloseContainer(Container container, String reason)
}
@Override
- public void closeContainer(Container container)
- throws IOException {
+ public void closeContainer(Container container) throws IOException {
container.writeLock();
try {
final State state = container.getContainerState();
@@ -1242,18 +1121,19 @@ public void closeContainer(Container container)
}
if (state == State.UNHEALTHY) {
throw new StorageContainerException(
- "Cannot close container #" + container.getContainerData()
- .getContainerID() + " while in " + state + " state.",
+ String.format("Cannot quasi close container #%s while in %s state.",
+ container.getContainerData().getContainerID(),
+ state),
ContainerProtos.Result.CONTAINER_UNHEALTHY);
}
// The container has to be either in CLOSING or in QUASI_CLOSED state.
if (state != State.CLOSING && state != State.QUASI_CLOSED) {
- ContainerProtos.Result error =
- state == State.INVALID ? INVALID_CONTAINER_STATE :
- CONTAINER_INTERNAL_ERROR;
+ ContainerProtos.Result error = state == State.INVALID ? INVALID_CONTAINER_STATE : CONTAINER_INTERNAL_ERROR;
throw new StorageContainerException(
- "Cannot close container #" + container.getContainerData()
- .getContainerID() + " while in " + state + " state.", error);
+ String.format("Cannot quasi close container #%s while in %s state.",
+ container.getContainerData().getContainerID(),
+ state),
+ error);
}
container.close();
ContainerLogger.logClosed(container.getContainerData());
@@ -1264,18 +1144,16 @@ public void closeContainer(Container container)
}
@Override
- public void deleteContainer(Container container, boolean force)
- throws IOException {
+ public void deleteContainer(Container container, boolean force) throws IOException {
deleteInternal(container, force);
}
/**
- * Called by BlockDeletingService to delete all the chunks in a block
- * before proceeding to delete the block info from DB.
+ * Called by {@link BlockDeletingService}
+ * to delete all the chunks in a block before proceeding to delete the block info from DB.
*/
@Override
- public void deleteBlock(Container container, BlockData blockData)
- throws IOException {
+ public void deleteBlock(Container container, BlockData blockData) throws IOException {
chunkManager.deleteChunks(container, blockData);
if (LOG.isDebugEnabled()) {
for (ContainerProtos.ChunkInfo chunkInfo : blockData.getChunks()) {
@@ -1286,13 +1164,10 @@ public void deleteBlock(Container container, BlockData blockData)
}
@Override
- public void deleteUnreferenced(Container container, long localID)
- throws IOException {
- // Since the block/chunk is already checked that is unreferenced, no
- // need to lock the container here.
+ public void deleteUnreferenced(Container container, long localID) throws IOException {
+ // Since the block/chunk is already checked that is unreferenced, no need to lock the container here.
StringBuilder prefixBuilder = new StringBuilder();
- ContainerLayoutVersion layoutVersion = container.getContainerData().
- getLayoutVersion();
+ ContainerLayoutVersion layoutVersion = container.getContainerData().getLayoutVersion();
long containerID = container.getContainerData().getContainerID();
// Only supports the default chunk/block name format now
switch (layoutVersion) {
@@ -1303,19 +1178,18 @@ public void deleteUnreferenced(Container container, long localID)
prefixBuilder.append(localID).append("_chunk_");
break;
default:
- throw new IOException("Unsupported container layout version " +
- layoutVersion + " for the container " + containerID);
+ throw new IOException(String.format("Unsupported container layout version %s for the container %s",
+ layoutVersion, containerID));
}
String prefix = prefixBuilder.toString();
File chunkDir = ContainerUtils.getChunkDir(container.getContainerData());
- // chunkNames here is an array of file/dir name, so if we cannot find any
- // matching one, it means the client did not write any chunk into the block.
+ // chunkNames here is an array of file/dir name,
+ // so if we cannot find any matching one, it means the client did not write any chunk into the block.
// Since the putBlock request may fail, we don't know if the chunk exists,
// thus we need to check it when receiving the request to delete such blocks
String[] chunkNames = getFilesWithPrefix(prefix, chunkDir);
if (chunkNames.length == 0) {
- LOG.warn("Missing delete block(Container = {}, Block = {}",
- containerID, localID);
+ LOG.warn("Missing delete block(Container = {}, Block = {}", containerID, localID);
return;
}
for (String name: chunkNames) {
@@ -1324,16 +1198,18 @@ public void deleteUnreferenced(Container container, long localID)
continue;
}
FileUtil.fullyDelete(file);
- LOG.info("Deleted unreferenced chunk/block {} in container {}", name,
- containerID);
+ LOG.info("Deleted unreferenced chunk/block {} in container {}", name, containerID);
}
}
+ @Override
public void addFinalizedBlock(Container container, long localID) {
KeyValueContainer keyValueContainer = (KeyValueContainer)container;
keyValueContainer.getContainerData().addToFinalizedBlockSet(localID);
}
+
+ @Override
public boolean isFinalizedBlockExist(Container container, long localID) {
KeyValueContainer keyValueContainer = (KeyValueContainer)container;
return keyValueContainer.getContainerData().isFinalizedBlockExist(localID);
@@ -1344,17 +1220,12 @@ private String[] getFilesWithPrefix(String prefix, File chunkDir) {
return chunkDir.list(filter);
}
- private boolean logBlocksIfNonZero(Container container)
- throws IOException {
+ private void logBlocksIfNonZero(Container container) throws IOException {
boolean nonZero = false;
- try (DBHandle dbHandle
- = BlockUtils.getDB(
- (KeyValueContainerData) container.getContainerData(),
- conf)) {
+ try (DBHandle dbHandle = BlockUtils.getDB((KeyValueContainerData) container.getContainerData(), conf)) {
StringBuilder stringBuilder = new StringBuilder();
- try (BlockIterator
- blockIterator = dbHandle.getStore().
- getBlockIterator(container.getContainerData().getContainerID())) {
+ try (BlockIterator blockIterator =
+ dbHandle.getStore().getBlockIterator(container.getContainerData().getContainerID())) {
while (blockIterator.hasNext()) {
nonZero = true;
stringBuilder.append(blockIterator.nextBlock());
@@ -1364,21 +1235,17 @@ private boolean logBlocksIfNonZero(Container container)
}
}
if (nonZero) {
- LOG.error("blocks in rocksDB on container delete: {}",
- stringBuilder.toString());
+ LOG.error("blocks in rocksDB on container delete: {}", stringBuilder);
}
}
- return nonZero;
}
- private boolean logBlocksFoundOnDisk(Container container) throws IOException {
+ private void logBlocksFoundOnDisk(Container container) throws IOException {
// List files left over
- File chunksPath = new
- File(container.getContainerData().getChunksPath());
+ File chunksPath = new File(container.getContainerData().getChunksPath());
Preconditions.checkArgument(chunksPath.isDirectory());
boolean notEmpty = false;
- try (DirectoryStream dir
- = Files.newDirectoryStream(chunksPath.toPath())) {
+ try (DirectoryStream dir = Files.newDirectoryStream(chunksPath.toPath())) {
StringBuilder stringBuilder = new StringBuilder();
for (Path block : dir) {
if (notEmpty) {
@@ -1391,68 +1258,57 @@ private boolean logBlocksFoundOnDisk(Container container) throws IOException {
}
}
if (notEmpty) {
- LOG.error("Files still part of the container on delete: {}",
- stringBuilder.toString());
+ LOG.error("Files still part of the container on delete: {}", stringBuilder);
}
}
- return notEmpty;
}
- private void deleteInternal(Container container, boolean force)
- throws StorageContainerException {
+ private void deleteInternal(Container container, boolean force) throws StorageContainerException {
container.writeLock();
try {
if (container.getContainerData().getVolume().isFailed()) {
- // if the volume in which the container resides fails
- // don't attempt to delete/move it. When a volume fails,
- // failedVolumeListener will pick it up and clear the container
- // from the container set.
- LOG.info("Delete container issued on containerID {} which is in a " +
- "failed volume. Skipping", container.getContainerData()
- .getContainerID());
+ // If the volume in which the container resides fails, don't attempt to delete/move it.
+ // When a volume fails, failedVolumeListener will pick it up and clear the container from the container set.
+ LOG.info("Delete container issued on containerID {} which is in a failed volume. Skipping",
+ container.getContainerData().getContainerID());
return;
}
- // If force is false, we check container state.
+ // If force is false, we check the container state.
if (!force) {
- // Check if container is open
+ // Check if the container is open
if (container.getContainerData().isOpen()) {
- throw new StorageContainerException(
- "Deletion of Open Container is not allowed.",
- DELETE_ON_OPEN_CONTAINER);
+ throw new StorageContainerException("Deletion of Open Container is not allowed.", DELETE_ON_OPEN_CONTAINER);
}
// Safety check that the container is empty.
- // If the container is not empty, it should not be deleted unless the
- // container is being forcefully deleted (which happens when
- // container is unhealthy or over-replicated).
+ // If the container is not empty, it should not be deleted unless the container is being forcefully deleted
+ // (which happens when the container is unhealthy or over-replicated).
if (container.hasBlocks()) {
metrics.incContainerDeleteFailedNonEmpty();
- LOG.error("Received container deletion command for container {} but" +
- " the container is not empty with blockCount {}",
+ LOG.error(
+ "Received container deletion command for container {} but the container is not empty with blockCount {}",
container.getContainerData().getContainerID(),
container.getContainerData().getBlockCount());
- // blocks table for future debugging.
+ // Blocks table for future debugging.
// List blocks
logBlocksIfNonZero(container);
// Log chunks
logBlocksFoundOnDisk(container);
- throw new StorageContainerException("Non-force deletion of " +
- "non-empty container is not allowed.",
+ throw new StorageContainerException("Non-force deletion of non-empty container is not allowed.",
DELETE_ON_NON_EMPTY_CONTAINER);
}
} else {
metrics.incContainersForceDelete();
}
if (container.getContainerData() instanceof KeyValueContainerData) {
- KeyValueContainerData keyValueContainerData =
- (KeyValueContainerData) container.getContainerData();
+ KeyValueContainerData keyValueContainerData = (KeyValueContainerData) container.getContainerData();
HddsVolume hddsVolume = keyValueContainerData.getVolume();
// Steps to delete
- // 1. container marked deleted
- // 2. container is removed from container set
- // 3. container db handler and content removed from db
- // 4. container moved to tmp folder
- // 5. container content deleted from tmp folder
+ // 1. Container marked deleted
+ // 2. Container is removed from container set
+ // 3. Container db handler and content removed from db
+ // 4. Container moved to tmp folder
+ // 5. Container content deleted from tmp folder
try {
container.markContainerForDelete();
long containerId = container.getContainerData().getContainerID();
@@ -1460,27 +1316,18 @@ private void deleteInternal(Container container, boolean force)
ContainerLogger.logDeleted(container.getContainerData(), force);
KeyValueContainerUtil.removeContainer(keyValueContainerData, conf);
} catch (IOException ioe) {
- LOG.error("Failed to move container under " + hddsVolume
- .getDeletedContainerDir());
- String errorMsg =
- "Failed to move container" + container.getContainerData()
- .getContainerID();
- triggerVolumeScanAndThrowException(container, errorMsg,
- CONTAINER_INTERNAL_ERROR);
+ LOG.error("Failed to move container under {}", hddsVolume.getDeletedContainerDir());
+ String errorMsg = "Failed to move container" + container.getContainerData().getContainerID();
+ triggerVolumeScanAndThrowException(container, errorMsg, CONTAINER_INTERNAL_ERROR);
}
}
} catch (StorageContainerException e) {
throw e;
} catch (IOException e) {
- // All other IO Exceptions should be treated as if the container is not
- // empty as a defensive check.
- LOG.error("Could not determine if the container {} is empty",
- container.getContainerData().getContainerID(), e);
- String errorMsg =
- "Failed to read container dir" + container.getContainerData()
- .getContainerID();
- triggerVolumeScanAndThrowException(container, errorMsg,
- CONTAINER_INTERNAL_ERROR);
+ // All other IO Exceptions should be treated as if the container is not empty as a defensive check.
+ LOG.error("Could not determine if the container {} is empty", container.getContainerData().getContainerID(), e);
+ String errorMsg = "Failed to read container dir" + container.getContainerData().getContainerID();
+ triggerVolumeScanAndThrowException(container, errorMsg, CONTAINER_INTERNAL_ERROR);
} finally {
container.writeUnlock();
}
@@ -1489,8 +1336,7 @@ private void deleteInternal(Container container, boolean force)
sendICR(container);
}
- private void triggerVolumeScanAndThrowException(Container container,
- String msg, ContainerProtos.Result result)
+ private void triggerVolumeScanAndThrowException(Container container, String msg, ContainerProtos.Result result)
throws StorageContainerException {
// Trigger a volume scan as exception occurred.
StorageVolumeUtil.onFailure(container.getContainerData().getVolume());
@@ -1520,15 +1366,100 @@ private ContainerCommandResponseProto checkFaultInjector(ContainerCommandRequest
return null;
}
+ private ContainerCommandResponseProto handleVerifyBlock(ContainerCommandRequestProto request,
+ KeyValueContainer kvContainer) {
+ if (!request.hasVerifyBlock()) {
+ LOG.debug("Malformed Verify Block request. trace ID: {}", request.getTraceID());
+ return malformedRequest(request);
+ }
+
+ try {
+ BlockID blockID = BlockID.getFromProtobuf(request.getGetBlock().getBlockID());
+ if (replicaIndexCheckRequired(request)) {
+ BlockUtils.verifyReplicaIdx(kvContainer, blockID);
+ }
+ BlockData block = BlockData.getFromProtoBuf(blockManager.getBlock(kvContainer, blockID).getProtoBufMessage());
+
+ KeyValueContainerData onDiskContainerData;
+
+ String metadataPath = kvContainer.getContainerData().getMetadataPath();
+
+ long containerID = kvContainer.getContainerData().getContainerID();
+
+ File containerFile = KeyValueContainer.getContainerFile(metadataPath, containerID);
+
+ HddsVolume volume = kvContainer.getContainerData().getVolume();
+
+ try {
+ onDiskContainerData = (KeyValueContainerData) ContainerDataYaml.readContainerFile(containerFile);
+ onDiskContainerData.setVolume(volume);
+ } catch (FileNotFoundException ex) {
+ return unhealthy(MISSING_CONTAINER_FILE, containerFile, ex);
+ } catch (IOException ex) {
+ return unhealthy(CORRUPT_CONTAINER_FILE, containerFile, ex);
+ }
+
+ ContainerLayoutVersion layout = onDiskContainerData.getLayoutVersion();
+
+ for (ContainerProtos.ChunkInfo chunk : block.getChunks()) {
+ File chunkFile;
+ try {
+ chunkFile = layout.getChunkFile(onDiskContainerData, block.getBlockID(), chunk.getChunkName());
+ } catch (IOException ex) {
+ return unhealthy(MISSING_CHUNK_FILE, new File(onDiskContainerData.getChunksPath()), ex);
+ }
+
+ if (!chunkFile.exists()) {
+ // In EC, a client may write empty putBlock in padding block nodes.
+ // So, we need to make sure, chunk length > 0, before declaring the missing chunk file.
+ if (!block.getChunks().isEmpty() && block.getChunks().get(0).getLen() > 0) {
+ return unhealthy(MISSING_CHUNK_FILE, chunkFile,
+ new IOException("Missing chunk file " + chunkFile.getAbsolutePath()));
+ }
+ } else if (chunk.getChecksumData().getType() != ContainerProtos.ChecksumType.NONE) {
+ int bytesPerChecksum = chunk.getChecksumData().getBytesPerChecksum();
+ ByteBuffer buffer = BUFFER_POOL.getBuffer(bytesPerChecksum);
+
+ ContainerScannerConfiguration containerScannerConfiguration = conf.getObject(ContainerScannerConfiguration.class);
+
+ DataTransferThrottler throttler = new DataTransferThrottler(containerScannerConfiguration.getBandwidthPerVolume());
+
+ Canceler canceler = new Canceler();
+
+ ScanResult result = verifyChecksum(block, chunk, chunkFile, layout, buffer, throttler, canceler);
+ buffer.clear();
+ BUFFER_POOL.returnBuffer(buffer);
+ if (!result.isHealthy()) {
+ return result;
+ }
+ }
+ }
+ } catch (Exception e) {
+ LOG.error("OOPS", e);
+ }
+
+ return null;
+ }
+
public static Logger getLogger() {
return LOG;
}
+ /**
+ * Returns the instance of {@link FaultInjector} used for testing purposes.
+ *
+ * @return the instance of {@link FaultInjector}.
+ */
@VisibleForTesting
public static FaultInjector getInjector() {
return injector;
}
+ /**
+ * Sets the instance of the {@link FaultInjector} for testing purposes.
+ *
+ * @param instance the instance of {@link FaultInjector} to be set
+ */
@VisibleForTesting
public static void setInjector(FaultInjector instance) {
injector = instance;
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index b7d5b5fa59e..8a3d4534b71 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -49,101 +49,79 @@
import static org.apache.hadoop.ozone.OzoneConsts.SCHEMA_V1;
-/**
- * Class which defines utility methods for KeyValueContainer.
- */
-
+/** Class which defines utility methods for KeyValueContainer. */
public final class KeyValueContainerUtil {
/* Never constructed. */
private KeyValueContainerUtil() {
-
}
- private static final Logger LOG = LoggerFactory.getLogger(
- KeyValueContainerUtil.class);
+ private static final Logger LOG = LoggerFactory.getLogger(KeyValueContainerUtil.class);
/**
- * creates metadata path, chunks path and metadata DB for the specified
- * container.
+ * Creates metadata path, chunk path and metadata DB for the specified container.
*
* @param containerMetaDataPath Path to the container's metadata directory.
- * @param chunksPath Path were chunks for this container should be stored.
+ * @param chunksPath Path was chunks for this container should be stored.
* @param dbFile Path to the container's .db file.
- * @param schemaVersion The schema version of the container. If this method
- * has not been updated after a schema version addition
- * and does not recognize the latest SchemaVersion, an
+ * @param schemaVersion The schema version of the container.
+ * If this method has not been updated after a schema version addition
+ * and does not recognize the latest SchemaVersion, an
* {@link IllegalArgumentException} is thrown.
* @param conf The configuration to use for this container.
- * @throws IOException
*/
- public static void createContainerMetaData(
- File containerMetaDataPath, File chunksPath, File dbFile,
+ public static void createContainerMetaData(File containerMetaDataPath, File chunksPath, File dbFile,
String schemaVersion, ConfigurationSource conf) throws IOException {
Preconditions.checkNotNull(containerMetaDataPath);
Preconditions.checkNotNull(conf);
if (!containerMetaDataPath.mkdirs()) {
- LOG.error("Unable to create directory for metadata storage. Path: {}",
- containerMetaDataPath);
- throw new IOException("Unable to create directory for metadata storage." +
- " Path: " + containerMetaDataPath);
+ LOG.error("Unable to create directory for metadata storage. Path: {}", containerMetaDataPath);
+ throw new IOException("Unable to create directory for metadata storage." + " Path: " + containerMetaDataPath);
}
if (!chunksPath.mkdirs()) {
- LOG.error("Unable to create chunks directory Container {}",
- chunksPath);
- //clean up container metadata path and metadata db
+ LOG.error("Unable to create chunks directory Container {}", chunksPath);
+ // Clean up container metadata path and metadata db
FileUtils.deleteDirectory(containerMetaDataPath);
FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
- throw new IOException("Unable to create directory for data storage." +
- " Path: " + chunksPath);
+ throw new IOException("Unable to create directory for data storage." + " Path: " + chunksPath);
}
DatanodeStore store;
if (isSameSchemaVersion(schemaVersion, OzoneConsts.SCHEMA_V1)) {
- store = new DatanodeStoreSchemaOneImpl(conf, dbFile.getAbsolutePath(),
- false);
+ store = new DatanodeStoreSchemaOneImpl(conf, dbFile.getAbsolutePath(), false);
} else if (isSameSchemaVersion(schemaVersion, OzoneConsts.SCHEMA_V2)) {
- store = new DatanodeStoreSchemaTwoImpl(conf, dbFile.getAbsolutePath(),
- false);
+ store = new DatanodeStoreSchemaTwoImpl(conf, dbFile.getAbsolutePath(), false);
} else if (isSameSchemaVersion(schemaVersion, OzoneConsts.SCHEMA_V3)) {
- // We don't create per-container store for schema v3 containers,
- // they should use per-volume db store.
+ // We don't create per-container store for schema v3 containers, they should use per-volume db store.
return;
} else {
- throw new IllegalArgumentException(
- "Unrecognized schema version for container: " + schemaVersion);
+ throw new IllegalArgumentException("Unrecognized schema version for container: " + schemaVersion);
}
- //add db handler into cache
+ // Add db handler into cache
BlockUtils.addDB(store, dbFile.getAbsolutePath(), conf, schemaVersion);
}
/**
- * remove Container 1. remove db, 2. move to tmp directory.
+ * Remove Container 1. remove db, 2. move to tmp directory.
*
* @param containerData - Data of the container to remove.
- * @throws IOException
*/
- public static void removeContainer(
- KeyValueContainerData containerData, ConfigurationSource conf)
- throws IOException {
+ public static void removeContainer(KeyValueContainerData containerData, ConfigurationSource conf) throws IOException {
Preconditions.checkNotNull(containerData);
KeyValueContainerUtil.removeContainerDB(containerData, conf);
- KeyValueContainerUtil.moveToDeletedContainerDir(containerData,
- containerData.getVolume());
+ KeyValueContainerUtil.moveToDeletedContainerDir(containerData, containerData.getVolume());
}
/**
- * remove Container db, the Level DB file.
+ * Remove Container db, the Level DB file.
*
* @param containerData - Data of the container to remove.
* @param conf - configuration of the cluster.
- * @throws IOException
*/
- public static void removeContainerDB(
- KeyValueContainerData containerData, ConfigurationSource conf)
+ public static void removeContainerDB(KeyValueContainerData containerData, ConfigurationSource conf)
throws IOException {
if (containerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
// DB failure is catastrophic, the disk needs to be replaced.
@@ -151,36 +129,31 @@ public static void removeContainerDB(
try {
BlockUtils.removeContainerFromDB(containerData, conf);
} catch (IOException ex) {
- LOG.error("DB failure, unable to remove container. " +
- "Disk need to be replaced.", ex);
+ LOG.error("DB failure, unable to remove container. Disk need to be replaced.", ex);
throw ex;
}
} else {
- // Close the DB connection and remove the DB handler from cache
+ // Close the DB connection and remove the DB handler from the cache
BlockUtils.removeDB(containerData, conf);
}
}
/**
* Returns if there are no blocks in the container.
+ *
* @param store DBStore
* @param containerData Container to check
- * @param bCheckChunksFilePath Whether to check chunksfilepath has any blocks
+ * @param bCheckChunksFilePath Whether to check chunks filepath has any blocks
* @return true if the directory containing blocks is empty
- * @throws IOException
*/
- public static boolean noBlocksInContainer(DatanodeStore store,
- KeyValueContainerData
- containerData,
- boolean bCheckChunksFilePath)
- throws IOException {
+ public static boolean noBlocksInContainer(DatanodeStore store, KeyValueContainerData containerData,
+ boolean bCheckChunksFilePath) throws IOException {
Preconditions.checkNotNull(store);
Preconditions.checkNotNull(containerData);
if (containerData.isOpen()) {
return false;
}
- try (BlockIterator blockIterator =
- store.getBlockIterator(containerData.getContainerID())) {
+ try (BlockIterator blockIterator = store.getBlockIterator(containerData.getContainerID())) {
if (blockIterator.hasNext()) {
return false;
}
@@ -188,8 +161,7 @@ public static boolean noBlocksInContainer(DatanodeStore store,
if (bCheckChunksFilePath) {
File chunksPath = new File(containerData.getChunksPath());
Preconditions.checkArgument(chunksPath.isDirectory());
- try (DirectoryStream dir
- = Files.newDirectoryStream(chunksPath.toPath())) {
+ try (DirectoryStream dir = Files.newDirectoryStream(chunksPath.toPath())) {
return !dir.iterator().hasNext();
}
}
@@ -197,16 +169,12 @@ public static boolean noBlocksInContainer(DatanodeStore store,
}
/**
- * Parse KeyValueContainerData and verify checksum. Set block related
- * metadata like block commit sequence id, block count, bytes used and
- * pending delete block count and delete transaction id.
- * @param kvContainerData
- * @param config
- * @throws IOException
+ * Parse KeyValueContainerData and verify checksum.
+ * Set block related metadata like a block commit sequence id, block count,
+ * bytes used and pending delete block count and delete transaction id.
*/
- public static void parseKVContainerData(KeyValueContainerData kvContainerData,
- ConfigurationSource config) throws IOException {
-
+ public static void parseKVContainerData(KeyValueContainerData kvContainerData, ConfigurationSource config)
+ throws IOException {
long containerID = kvContainerData.getContainerID();
// Verify Checksum
@@ -218,25 +186,20 @@ public static void parseKVContainerData(KeyValueContainerData kvContainerData,
kvContainerData.setSchemaVersion(OzoneConsts.SCHEMA_V1);
}
- File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
- kvContainerData);
+ File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(kvContainerData);
if (!dbFile.exists()) {
- LOG.error("Container DB file is missing for ContainerID {}. " +
- "Skipping loading of this container.", containerID);
+ LOG.error("Container DB file is missing for ContainerID {}. Skipping loading of this container.", containerID);
// Don't further process this container, as it is missing db file.
- throw new IOException("Container DB file is missing for containerID "
- + containerID);
+ throw new IOException("Container DB file is missing for containerID " + containerID);
}
kvContainerData.setDbFile(dbFile);
- DatanodeConfiguration dnConf =
- config.getObject(DatanodeConfiguration.class);
+ DatanodeConfiguration dnConf = config.getObject(DatanodeConfiguration.class);
boolean bCheckChunksFilePath = dnConf.getCheckEmptyContainerDir();
if (kvContainerData.hasSchema(OzoneConsts.SCHEMA_V3)) {
try (DBHandle db = BlockUtils.getDB(kvContainerData, config)) {
- populateContainerMetadata(kvContainerData,
- db.getStore(), bCheckChunksFilePath);
+ populateContainerMetadata(kvContainerData, db.getStore(), bCheckChunksFilePath);
}
return;
}
@@ -245,94 +208,76 @@ public static void parseKVContainerData(KeyValueContainerData kvContainerData,
DatanodeStore store = null;
try {
try {
- boolean readOnly = ContainerInspectorUtil.isReadOnly(
- ContainerProtos.ContainerType.KeyValueContainer);
- store = BlockUtils.getUncachedDatanodeStore(
- kvContainerData, config, readOnly);
+ boolean readOnly = ContainerInspectorUtil.isReadOnly(ContainerProtos.ContainerType.KeyValueContainer);
+ store = BlockUtils.getUncachedDatanodeStore(kvContainerData, config, readOnly);
} catch (IOException e) {
- // If an exception is thrown, then it may indicate the RocksDB is
- // already open in the container cache. As this code is only executed at
- // DN startup, this should only happen in the tests.
+ // If an exception is thrown, then it may indicate the RocksDB is already open in the container cache.
+ // As this code is only executed at DN startup, this should only happen in the tests.
cachedDB = BlockUtils.getDB(kvContainerData, config);
store = cachedDB.getStore();
- LOG.warn("Attempt to get an uncached RocksDB handle failed and an " +
- "instance was retrieved from the cache. This should only happen " +
- "in tests");
+ LOG.warn("Attempt to get an uncached RocksDB handle failed and an instance was retrieved from the cache." +
+ " This should only happen in tests");
}
populateContainerMetadata(kvContainerData, store, bCheckChunksFilePath);
} finally {
if (cachedDB != null) {
- // If we get a cached instance, calling close simply decrements the
- // reference count.
+ // If we get a cached instance, calling close simply decrements the reference count.
cachedDB.close();
} else if (store != null) {
- // We only stop the store if cacheDB is null, as otherwise we would
- // close the rocksDB handle in the cache and the next reader would fail
+ // We only stop the store if cacheDB is null,
+ // as otherwise we would close the rocksDB handle in the cache and the next reader would fail
try {
store.stop();
} catch (IOException e) {
throw e;
} catch (Exception e) {
- throw new RuntimeException("Unexpected exception closing the " +
- "RocksDB when loading containers", e);
+ throw new RuntimeException("Unexpected exception closing the RocksDB when loading containers", e);
}
}
}
}
- private static void populateContainerMetadata(
- KeyValueContainerData kvContainerData, DatanodeStore store,
- boolean bCheckChunksFilePath)
- throws IOException {
+ private static void populateContainerMetadata(KeyValueContainerData kvContainerData, DatanodeStore store,
+ boolean bCheckChunksFilePath) throws IOException {
boolean isBlockMetadataSet = false;
Table metadataTable = store.getMetadataTable();
// Set pending deleted block count.
- Long pendingDeleteBlockCount =
- metadataTable.get(kvContainerData
- .getPendingDeleteBlockCountKey());
+ Long pendingDeleteBlockCount = metadataTable.get(kvContainerData.getPendingDeleteBlockCountKey());
if (pendingDeleteBlockCount != null) {
- kvContainerData.incrPendingDeletionBlocks(
- pendingDeleteBlockCount);
+ kvContainerData.incrPendingDeletionBlocks(pendingDeleteBlockCount);
} else {
// Set pending deleted block count.
- MetadataKeyFilters.KeyPrefixFilter filter =
- kvContainerData.getDeletingBlockKeyFilter();
+ MetadataKeyFilters.KeyPrefixFilter filter = kvContainerData.getDeletingBlockKeyFilter();
int numPendingDeletionBlocks = store.getBlockDataTable()
.getSequentialRangeKVs(kvContainerData.startKeyEmpty(),
- Integer.MAX_VALUE, kvContainerData.containerPrefix(),
+ Integer.MAX_VALUE,
+ kvContainerData.containerPrefix(),
filter).size();
kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks);
}
// Set delete transaction id.
- Long delTxnId =
- metadataTable.get(kvContainerData.getLatestDeleteTxnKey());
+ Long delTxnId = metadataTable.get(kvContainerData.getLatestDeleteTxnKey());
if (delTxnId != null) {
- kvContainerData
- .updateDeleteTransactionId(delTxnId);
+ kvContainerData.updateDeleteTransactionId(delTxnId);
}
// Set BlockCommitSequenceId.
- Long bcsId = metadataTable.get(
- kvContainerData.getBcsIdKey());
+ Long bcsId = metadataTable.get(kvContainerData.getBcsIdKey());
if (bcsId != null) {
- kvContainerData
- .updateBlockCommitSequenceId(bcsId);
+ kvContainerData.updateBlockCommitSequenceId(bcsId);
}
- // Set bytes used.
- // commitSpace for Open Containers relies on usedBytes
- Long bytesUsed =
- metadataTable.get(kvContainerData.getBytesUsedKey());
+ // Set bytes used. commitSpace for Open Containers relies on usedBytes
+ Long bytesUsed = metadataTable.get(kvContainerData.getBytesUsedKey());
if (bytesUsed != null) {
isBlockMetadataSet = true;
kvContainerData.setBytesUsed(bytesUsed);
}
// Set block count.
- Long blockCount = metadataTable.get(
- kvContainerData.getBlockCountKey());
+ Long blockCount = metadataTable.get(kvContainerData.getBlockCountKey());
if (blockCount != null) {
isBlockMetadataSet = true;
kvContainerData.setBlockCount(blockCount);
@@ -341,8 +286,7 @@ private static void populateContainerMetadata(
initializeUsedBytesAndBlockCount(store, kvContainerData);
}
- // If the container is missing a chunks directory, possibly due to the
- // bug fixed by HDDS-6235, create it here.
+ // If the container is missing a chunks directory, possibly due to the bug fixed by HDDS-6235, create it here.
File chunksDir = new File(kvContainerData.getChunksPath());
if (!chunksDir.exists()) {
Files.createDirectories(chunksDir.toPath());
@@ -352,9 +296,8 @@ private static void populateContainerMetadata(
kvContainerData.markAsEmpty();
}
- // Run advanced container inspection/repair operations if specified on
- // startup. If this method is called but not as a part of startup,
- // The inspectors will be unloaded and this will be a no-op.
+ // Run advanced container inspection/repair operations if specified on startup.
+ // If this method is called but not as a part of startup, the inspectors will be unloaded and this will be a no-op.
ContainerInspectorUtil.process(kvContainerData, store);
// Load finalizeBlockLocalIds for container in memory.
@@ -363,12 +306,11 @@ private static void populateContainerMetadata(
/**
* Loads finalizeBlockLocalIds for container in memory.
+ *
* @param kvContainerData - KeyValueContainerData
* @param store - DatanodeStore
- * @throws IOException
*/
- private static void populateContainerFinalizeBlock(
- KeyValueContainerData kvContainerData, DatanodeStore store)
+ private static void populateContainerFinalizeBlock(KeyValueContainerData kvContainerData, DatanodeStore store)
throws IOException {
if (store.getFinalizeBlocksTable() != null) {
try (BlockIterator iter =
@@ -383,20 +325,15 @@ private static void populateContainerFinalizeBlock(
/**
* Initialize bytes used and block count.
- * @param kvData
- * @throws IOException
*/
private static void initializeUsedBytesAndBlockCount(DatanodeStore store,
KeyValueContainerData kvData) throws IOException {
- final String errorMessage = "Failed to parse block data for" +
- " Container " + kvData.getContainerID();
+ final String errorMessage = "Failed to parse block data for Container " + kvData.getContainerID();
long blockCount = 0;
long usedBytes = 0;
try (BlockIterator blockIter =
- store.getBlockIterator(kvData.getContainerID(),
- kvData.getUnprefixedKeyFilter())) {
-
+ store.getBlockIterator(kvData.getContainerID(), kvData.getUnprefixedKeyFilter())) {
while (blockIter.hasNext()) {
blockCount++;
try {
@@ -409,9 +346,7 @@ private static void initializeUsedBytesAndBlockCount(DatanodeStore store,
// Count all deleting blocks.
try (BlockIterator blockIter =
- store.getBlockIterator(kvData.getContainerID(),
- kvData.getDeletingBlockKeyFilter())) {
-
+ store.getBlockIterator(kvData.getContainerID(), kvData.getDeletingBlockKeyFilter())) {
while (blockIter.hasNext()) {
blockCount++;
try {
@@ -425,12 +360,26 @@ private static void initializeUsedBytesAndBlockCount(DatanodeStore store,
kvData.setBlockCount(blockCount);
}
+ /**
+ * Calculates the total length of all chunks in a given block.
+ *
+ * @param block BlockData object containing chunks whose lengths need to be summed.
+ * @return Total length of all chunks within the block.
+ * @throws IOException if an I/O error occurs while accessing chunk lengths.
+ */
public static long getBlockLength(BlockData block) throws IOException {
return block.getChunks().stream()
.mapToLong(ContainerProtos.ChunkInfo::getLen)
.sum();
}
+ /**
+ * Compares two schema versions to determine if they are the same.
+ *
+ * @param schema the first schema version to compare. If {@code null}, defaults to {@link OzoneConsts#SCHEMA_V1}.
+ * @param other the second schema version to compare. If {@code null}, defaults to {@link OzoneConsts#SCHEMA_V1}.
+ * @return {@code true} if both schema versions are equal, {@code false} otherwise.
+ */
public static boolean isSameSchemaVersion(String schema, String other) {
String effective1 = schema != null ? schema : SCHEMA_V1;
String effective2 = other != null ? other : SCHEMA_V1;
@@ -438,59 +387,58 @@ public static boolean isSameSchemaVersion(String schema, String other) {
}
/**
- * Moves container directory to a new location
- * under "/hdds//tmp/deleted-containers"
- * and updates metadata and chunks path.
- * Containers will be moved under it before getting deleted
- * to avoid, in case of failure, having artifact leftovers
- * on the default container path on the disk.
- *
+ * Moves container directory to a new location under "/hdds//tmp/deleted-containers"
+ * and updates metadata and chunk path.
+ * Containers will be moved under it before getting deleted to avoid, in case of failure,
+ * having artifact leftovers on the default container path on the disk.
+ *
* Delete operation for Schema < V3
- * 1. Container is marked DELETED
- * 2. Container is removed from memory container set
- * 3. Container DB handler from cache is removed and closed
- * 4. Container directory renamed to tmp directory.
- * 5. Container is deleted from tmp directory.
- *
+ *
+ * - Container is marked DELETED
+ *
- Container is removed from memory container set
+ *
- Container DB handler from cache is removed and closed
+ *
- Container directory renamed to tmp directory.
+ *
- Container is deleted from tmp directory.
+ *
* Delete operation for Schema V3
- * 1. Container is marked DELETED
- * 2. Container is removed from memory container set
- * 3. Container from DB is removed
- * 4. Container directory renamed to tmp directory.
- * 5. Container is deleted from tmp directory.
- *
- * @param keyValueContainerData
- * @return true if renaming was successful
+ *
+ * - Container is marked DELETED
+ *
- Container is removed from memory container set
+ *
- Container from DB is removed
+ *
- Container directory renamed to tmp directory.
+ *
- Container is deleted from tmp directory.
+ *
*/
- public static void moveToDeletedContainerDir(
- KeyValueContainerData keyValueContainerData,
- HddsVolume hddsVolume) throws IOException {
+ public static void moveToDeletedContainerDir(KeyValueContainerData keyValueContainerData, HddsVolume hddsVolume)
+ throws IOException {
String containerPath = keyValueContainerData.getContainerPath();
File container = new File(containerPath);
- Path destinationDirPath = getTmpDirectoryPath(keyValueContainerData,
- hddsVolume);
+ Path destinationDirPath = getTmpDirectoryPath(keyValueContainerData, hddsVolume);
File destinationDirFile = destinationDirPath.toFile();
- // If a container by the same name was moved to the delete directory but
- // the final delete failed, clear it out before adding another container
- // with the same name.
+ // If a container by the same name was moved to the delete directory but the final delete failed,
+ // clear it out before adding another container with the same name.
if (destinationDirFile.exists()) {
FileUtils.deleteDirectory(destinationDirFile);
}
Files.move(container.toPath(), destinationDirPath);
LOG.debug("Container {} has been successfully moved under {}",
- container.getName(), hddsVolume.getDeletedContainerDir());
+ container.getName(),
+ hddsVolume.getDeletedContainerDir());
}
- public static Path getTmpDirectoryPath(
- KeyValueContainerData keyValueContainerData,
- HddsVolume hddsVolume) {
+ /**
+ * Retrieves the temporary directory path for a container.
+ *
+ * @param keyValueContainerData the data associated with the KeyValue container
+ * @param hddsVolume the volume where the container resides
+ * @return the path representing the temporary directory for the container
+ */
+ public static Path getTmpDirectoryPath(KeyValueContainerData keyValueContainerData, HddsVolume hddsVolume) {
String containerPath = keyValueContainerData.getContainerPath();
File container = new File(containerPath);
String containerDirName = container.getName();
- Path destinationDirPath = hddsVolume.getDeletedContainerDir().toPath()
- .resolve(Paths.get(containerDirName));
- return destinationDirPath;
+ return hddsVolume.getDeletedContainerDir().toPath().resolve(Paths.get(containerDirName));
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
index 6232b843567..93720694999 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.hdds.utils.db.BatchOperation;
import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.impl.BlockDeletingService;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.interfaces.DBHandle;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
@@ -48,15 +49,13 @@
import org.slf4j.LoggerFactory;
/**
- * This class is for performing block related operations on the KeyValue
- * Container.
+ * This class is for performing block related operations on the KeyValue Container.
*/
public class BlockManagerImpl implements BlockManager {
- public static final Logger LOG =
- LoggerFactory.getLogger(BlockManagerImpl.class);
+ public static final Logger LOG = LoggerFactory.getLogger(BlockManagerImpl.class);
- private ConfigurationSource config;
+ private final ConfigurationSource config;
private static final String DB_NULL_ERR_MSG = "DB cannot be null here";
public static final String FULL_CHUNK = "full";
@@ -91,66 +90,67 @@ public long putBlock(Container container, BlockData data) throws IOException {
}
@Override
- public long putBlock(Container container, BlockData data,
- boolean endOfBlock) throws IOException {
- return persistPutBlock(
- (KeyValueContainer) container,
- data, endOfBlock);
+ public long putBlock(Container container, BlockData data, boolean endOfBlock) throws IOException {
+ return persistPutBlock((KeyValueContainer) container, data, endOfBlock);
}
- public long persistPutBlock(KeyValueContainer container,
- BlockData data, boolean endOfBlock)
- throws IOException {
- Preconditions.checkNotNull(data, "BlockData cannot be null for put " +
- "operation.");
- Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
- "cannot be negative");
+ /**
+ * Persists a block into the key-value container. This method is responsible
+ * for updating the state of the container along with the block metadata. It
+ * ensures that the block data is correctly stored in the associated DB and
+ * handles potential conflicts during the process.
+ *
+ * @param container The key-value container where the block data will be stored.
+ * @param data The block data to be persisted.
+ * @param endOfBlock A flag indicating whether this is the final chunk of the block.
+ * @return The size of the block that was persisted.
+ * @throws IOException If an I/O error occurs during the operation.
+ */
+ public long persistPutBlock(KeyValueContainer container, BlockData data, boolean endOfBlock) throws IOException {
+ Preconditions.checkNotNull(data, "BlockData cannot be null for put operation.");
+ Preconditions.checkState(data.getContainerID() >= 0, "Container Id cannot be negative");
KeyValueContainerData containerData = container.getContainerData();
- // We are not locking the key manager since LevelDb serializes all actions
- // against a single DB. We rely on DB level locking to avoid conflicts.
+ // We are not locking the key manager since LevelDb serializes all actions against a single DB.
+ // We rely on DB level locking to avoid conflicts.
try (DBHandle db = BlockUtils.getDB(containerData, config)) {
- // This is a post condition that acts as a hint to the user.
+ // This is a post-condition that acts as a hint to the user.
// Should never fail.
Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
long bcsId = data.getBlockCommitSequenceId();
long containerBCSId = containerData.getBlockCommitSequenceId();
- // default blockCommitSequenceId for any block is 0. It the putBlock
- // request is not coming via Ratis(for test scenarios), it will be 0.
+ // Default blockCommitSequenceId for any block is 0.
+ // It the putBlock request is not coming via Ratis(for test scenarios), it will be 0.
// In such cases, we should overwrite the block as well
if ((bcsId != 0) && (bcsId <= containerBCSId)) {
- // Since the blockCommitSequenceId stored in the db is greater than
- // equal to blockCommitSequenceId to be updated, it means the putBlock
- // transaction is reapplied in the ContainerStateMachine on restart.
+ // Since the blockCommitSequenceId stored in the db is greater than equal to blockCommitSequenceId
+ // to be updated,
+ // it means the putBlock transaction is reapplied in the ContainerStateMachine on restart.
// It also implies that the given block must already exist in the db.
- // just log and return
- LOG.debug("blockCommitSequenceId {} in the Container Db is greater"
- + " than the supplied value {}. Ignoring it",
- containerBCSId, bcsId);
+ // Log and return
+ LOG.debug("blockCommitSequenceId {} in the Container Db is greater than the supplied value {}. Ignoring it",
+ containerBCSId,
+ bcsId);
return data.getSize();
}
- // Check if the block is present in the pendingPutBlockCache for the
- // container to determine whether the blockCount is already incremented
- // for this block in the DB or not.
+ // Check if the block is present in the pendingPutBlockCache for the container to determine
+ // whether the blockCount is already incremented for this block in the DB or not.
long localID = data.getLocalID();
boolean isBlockInCache = container.isBlockInPendingPutBlockCache(localID);
boolean incrBlockCount = false;
- // update the blockData as well as BlockCommitSequenceId here
- try (BatchOperation batch = db.getStore().getBatchHandler()
- .initBatchOperation()) {
- // If the block does not exist in the pendingPutBlockCache of the
- // container, then check the DB to ascertain if it exists or not.
- // If block exists in cache, blockCount should not be incremented.
+ // Update the blockData as well as BlockCommitSequenceId here
+ try (BatchOperation batch = db.getStore().getBatchHandler().initBatchOperation()) {
+ // If the block does not exist in the pendingPutBlockCache of the container,
+ // then check the DB to figure out if it exists or not.
+ // If a block exists in cache, blockCount should not be incremented.
if (!isBlockInCache) {
- if (db.getStore().getBlockDataTable().get(
- containerData.getBlockKey(localID)) == null) {
- // Block does not exist in DB => blockCount needs to be
- // incremented when the block is added into DB.
+ if (db.getStore().getBlockDataTable().get(containerData.getBlockKey(localID)) == null) {
+ // Block does not exist in DB ⇒ blockCount needs to be incremented when the block is added into DB.
incrBlockCount = true;
}
}
@@ -158,33 +158,31 @@ public long persistPutBlock(KeyValueContainer container,
boolean incrementalEnabled = true;
if (!VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.HBASE_SUPPORT)) {
if (isPartialChunkList(data)) {
- throw new StorageContainerException("DataNode has not finalized " +
- "upgrading to a version that supports incremental chunk list.", UNSUPPORTED_REQUEST);
+ throw new StorageContainerException("DataNode has not finalized upgrading to a version" +
+ " that supports incremental chunk list.", UNSUPPORTED_REQUEST);
}
incrementalEnabled = false;
}
- db.getStore().putBlockByID(batch, incrementalEnabled, localID, data,
- containerData, endOfBlock);
+ db.getStore().putBlockByID(batch, incrementalEnabled, localID, data, containerData, endOfBlock);
if (bcsId != 0) {
- db.getStore().getMetadataTable().putWithBatch(
- batch, containerData.getBcsIdKey(), bcsId);
+ db.getStore().getMetadataTable().putWithBatch(batch, containerData.getBcsIdKey(), bcsId);
}
- // Set Bytes used, this bytes used will be updated for every write and
- // only get committed for every put block. In this way, when datanode
- // is up, for computation of disk space by container only committed
- // block length is used, And also on restart the blocks committed to DB
- // is only used to compute the bytes used. This is done to keep the
- // current behavior and avoid DB write during write chunk operation.
- db.getStore().getMetadataTable().putWithBatch(
- batch, containerData.getBytesUsedKey(),
- containerData.getBytesUsed());
+ // Set Bytes used,
+ // these bytes used will be updated for every writing and only get committed for every put block.
+ // In this way, when datanode is up,
+ // for computation of disk space by container only committed block length is used,
+ // and also on restart the blocks committed to DB is only used to compute the bytes used.
+ // This is done to keep the current behavior and avoid DB write during write chunk operation.
+ db.getStore()
+ .getMetadataTable()
+ .putWithBatch(batch, containerData.getBytesUsedKey(), containerData.getBytesUsed());
// Set Block Count for a container.
if (incrBlockCount) {
- db.getStore().getMetadataTable().putWithBatch(
- batch, containerData.getBlockCountKey(),
- containerData.getBlockCount() + 1);
+ db.getStore()
+ .getMetadataTable()
+ .putWithBatch(batch, containerData.getBlockCountKey(), containerData.getBlockCount() + 1);
}
db.getStore().getBatchHandler().commitBatchOperation(batch);
@@ -194,55 +192,48 @@ public long persistPutBlock(KeyValueContainer container,
container.updateBlockCommitSequenceId(bcsId);
}
- // Increment block count and add block to pendingPutBlockCache
- // in-memory after the DB update.
+ // Increment block count and add block to pendingPutBlockCache in-memory after the DB update.
if (incrBlockCount) {
containerData.incrBlockCount();
}
// If the Block is not in PendingPutBlockCache (and it is not endOfBlock),
- // add it there so that subsequent putBlock calls for this block do not
- // have to read the DB to check for block existence
+ // add it there
+ // so that later putBlock calls for this block do not have to read the DB to check for block existence
if (!isBlockInCache && !endOfBlock) {
container.addToPendingPutBlockCache(localID);
} else if (isBlockInCache && endOfBlock) {
- // Remove the block from the PendingPutBlockCache as there would not
- // be any more writes to this block
+ // Remove the block from the PendingPutBlockCache as there would not be anymore writes to this block
container.removeFromPendingPutBlockCache(localID);
}
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Block " + data.getBlockID() + " successfully committed with bcsId "
- + bcsId + " chunk size " + data.getChunks().size());
- }
+ LOG.debug("Block {} successfully committed with bcsId {} chunk size {}",
+ data.getBlockID(),
+ bcsId,
+ data.getChunks().size());
return data.getSize();
}
}
@Override
- public void finalizeBlock(Container container, BlockID blockId)
- throws IOException {
- Preconditions.checkNotNull(blockId, "blockId cannot " +
- "be null for finalizeBlock operation.");
- Preconditions.checkState(blockId.getContainerID() >= 0,
- "Container Id cannot be negative");
+ public void finalizeBlock(Container container, BlockID blockId) throws IOException {
+ Preconditions.checkNotNull(blockId, "blockId cannot be null for finalizeBlock operation.");
+ Preconditions.checkState(blockId.getContainerID() >= 0, "Container Id cannot be negative");
KeyValueContainer kvContainer = (KeyValueContainer)container;
long localID = blockId.getLocalID();
kvContainer.removeFromPendingPutBlockCache(localID);
- try (DBHandle db = BlockUtils.getDB(kvContainer.getContainerData(),
- config)) {
+ try (DBHandle db = BlockUtils.getDB(kvContainer.getContainerData(), config)) {
// Should never fail.
Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
// persist finalizeBlock
- try (BatchOperation batch = db.getStore().getBatchHandler()
- .initBatchOperation()) {
- db.getStore().getFinalizeBlocksTable().putWithBatch(batch,
- kvContainer.getContainerData().getBlockKey(localID), localID);
+ try (BatchOperation batch = db.getStore().getBatchHandler().initBatchOperation()) {
+ db.getStore()
+ .getFinalizeBlocksTable()
+ .putWithBatch(batch, kvContainer.getContainerData().getBlockKey(localID), localID);
db.getStore().getBatchHandler().commitBatchOperation(batch);
mergeLastChunkForBlockFinalization(blockId, db, kvContainer, batch, localID);
@@ -250,48 +241,41 @@ public void finalizeBlock(Container container, BlockID blockId)
}
}
- private void mergeLastChunkForBlockFinalization(BlockID blockId, DBHandle db,
- KeyValueContainer kvContainer, BatchOperation batch,
- long localID) throws IOException {
- // if the chunk list of the block to be finalized was written incremental,
- // merge the last chunk into block data.
+ private void mergeLastChunkForBlockFinalization(BlockID blockId, DBHandle db, KeyValueContainer kvContainer,
+ BatchOperation batch, long localID) throws IOException {
+ // If the chunk list of the block to be finalized was written incremental, merge the last chunk into block data.
BlockData blockData = getBlockByID(db, blockId, kvContainer.getContainerData());
if (blockData.getMetadata().containsKey(INCREMENTAL_CHUNK_LIST)) {
BlockData emptyBlockData = new BlockData(blockId);
emptyBlockData.addMetadata(INCREMENTAL_CHUNK_LIST, "");
- db.getStore().putBlockByID(batch, true, localID,
- emptyBlockData, kvContainer.getContainerData(), true);
+ db.getStore()
+ .putBlockByID(batch, true, localID, emptyBlockData, kvContainer.getContainerData(), true);
}
}
@Override
public BlockData getBlock(Container container, BlockID blockID) throws IOException {
BlockUtils.verifyBCSId(container, blockID);
- KeyValueContainerData containerData = (KeyValueContainerData) container
- .getContainerData();
+ KeyValueContainerData containerData = (KeyValueContainerData) container.getContainerData();
long bcsId = blockID.getBlockCommitSequenceId();
try (DBHandle db = BlockUtils.getDB(containerData, config)) {
- // This is a post condition that acts as a hint to the user.
- // Should never fail.
+ // This is a post-condition that acts as a hint to the user. Should never fail.
Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
BlockData blockData = getBlockByID(db, blockID, containerData);
long id = blockData.getBlockID().getBlockCommitSequenceId();
if (id < bcsId) {
- throw new StorageContainerException(
- "bcsId " + bcsId + " mismatches with existing block Id "
- + id + " for block " + blockID + ".", BCSID_MISMATCH);
+ throw new StorageContainerException("bcsId " + bcsId + " mismatches with existing block Id " + id
+ + " for block " + blockID + ".", BCSID_MISMATCH);
}
return blockData;
}
}
@Override
- public long getCommittedBlockLength(Container container, BlockID blockID)
- throws IOException {
- KeyValueContainerData containerData = (KeyValueContainerData) container
- .getContainerData();
+ public long getCommittedBlockLength(Container container, BlockID blockID) throws IOException {
+ KeyValueContainerData containerData = (KeyValueContainerData) container.getContainerData();
try (DBHandle db = BlockUtils.getDB(containerData, config)) {
- // This is a post condition that acts as a hint to the user.
+ // This is a post-condition that acts as a hint to the user.
// Should never fail.
Preconditions.checkNotNull(db, DB_NULL_ERR_MSG);
BlockData blockData = getBlockByID(db, blockID, containerData);
@@ -304,6 +288,7 @@ public int getDefaultReadBufferCapacity() {
return defaultReadBufferCapacity;
}
+ @Override
public int getReadMappedBufferThreshold() {
return readMappedBufferThreshold;
}
@@ -315,42 +300,34 @@ public int getReadMappedBufferMaxCount() {
/**
* Deletes an existing block.
- * As Deletion is handled by BlockDeletingService,
- * UnsupportedOperationException is thrown always
+ * As Deletion is handled by {@link BlockDeletingService}, {@link UnsupportedOperationException} is thrown always.
*
* @param container - Container from which block need to be deleted.
* @param blockID - ID of the block.
*/
@Override
- public void deleteBlock(Container container, BlockID blockID) throws
- IOException {
+ public void deleteBlock(Container container, BlockID blockID) {
// Block/ Chunk Deletion is handled by BlockDeletingService.
- // SCM sends Block Deletion commands directly to Datanodes and not
- // through a Pipeline.
+ // SCM sends Block Deletion commands directly to Datanodes and not through a Pipeline.
throw new UnsupportedOperationException();
}
@Override
- public List listBlock(Container container, long startLocalID, int
- count) throws IOException {
+ public List listBlock(Container container, long startLocalID, int count) throws IOException {
Preconditions.checkNotNull(container, "container cannot be null");
- Preconditions.checkState(startLocalID >= 0 || startLocalID == -1,
- "startLocal ID cannot be negative");
- Preconditions.checkArgument(count > 0,
- "Count must be a positive number.");
+ Preconditions.checkState(startLocalID >= 0 || startLocalID == -1, "startLocal ID cannot be negative");
+ Preconditions.checkArgument(count > 0, "Count must be a positive number.");
container.readLock();
try {
- List result = null;
- KeyValueContainerData cData =
- (KeyValueContainerData) container.getContainerData();
+ List result;
+ KeyValueContainerData cData = (KeyValueContainerData) container.getContainerData();
try (DBHandle db = BlockUtils.getDB(cData, config)) {
result = new ArrayList<>();
- String startKey = (startLocalID == -1) ? cData.startKeyEmpty()
+ String startKey = (startLocalID == -1)
+ ? cData.startKeyEmpty()
: cData.getBlockKey(startLocalID);
- List extends Table.KeyValue> range =
- db.getStore().getBlockDataTable()
- .getSequentialRangeKVs(startKey, count,
- cData.containerPrefix(), cData.getUnprefixedKeyFilter());
+ List extends Table.KeyValue> range = db.getStore().getBlockDataTable()
+ .getSequentialRangeKVs(startKey, count, cData.containerPrefix(), cData.getUnprefixedKeyFilter());
for (Table.KeyValue entry : range) {
result.add(db.getStore().getCompleteBlockData(entry.getValue(), null, entry.getKey()));
}
@@ -369,8 +346,7 @@ public void shutdown() {
BlockUtils.shutdownCache(config);
}
- private BlockData getBlockByID(DBHandle db, BlockID blockID,
- KeyValueContainerData containerData) throws IOException {
+ private BlockData getBlockByID(DBHandle db, BlockID blockID, KeyValueContainerData containerData) throws IOException {
String blockKey = containerData.getBlockKey(blockID.getLocalID());
return db.getStore().getBlockByID(blockID, blockKey);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java
index 6a1d5533cf2..0139e286eb1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java
@@ -52,64 +52,57 @@
*/
public class ChunkManagerDispatcher implements ChunkManager {
- private static final Logger LOG =
- LoggerFactory.getLogger(ChunkManagerDispatcher.class);
-
- private final Map handlers
- = new EnumMap<>(ContainerLayoutVersion.class);
-
- ChunkManagerDispatcher(boolean sync, BlockManager manager,
- VolumeSet volSet) {
- handlers.put(FILE_PER_CHUNK,
- new FilePerChunkStrategy(sync, manager, volSet));
- handlers.put(FILE_PER_BLOCK,
- new FilePerBlockStrategy(sync, manager, volSet));
+ private static final Logger LOG = LoggerFactory.getLogger(ChunkManagerDispatcher.class);
+
+ private final Map handlers = new EnumMap<>(ContainerLayoutVersion.class);
+
+ ChunkManagerDispatcher(boolean sync, BlockManager manager, VolumeSet volSet) {
+ handlers.put(FILE_PER_CHUNK, new FilePerChunkStrategy(sync, manager, volSet));
+ handlers.put(FILE_PER_BLOCK, new FilePerBlockStrategy(sync, manager));
}
@Override
- public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
- ChunkBuffer data, DispatcherContext dispatcherContext)
- throws StorageContainerException {
+ public void writeChunk(Container container, BlockID blockID, ChunkInfo info, ChunkBuffer data,
+ DispatcherContext dispatcherContext) throws StorageContainerException {
- selectHandler(container)
- .writeChunk(container, blockID, info, data, dispatcherContext);
+ selectHandler(container).writeChunk(container, blockID, info, data, dispatcherContext);
}
- public String streamInit(Container container, BlockID blockID)
- throws StorageContainerException {
- return selectHandler(container)
- .streamInit(container, blockID);
+ /**
+ * Initializes a stream for a given container and block ID.
+ *
+ * @param container the container in which the stream is being initialized
+ * @param blockID the block ID for which the stream is being initialized
+ * @return a String representing the initialization state of the stream
+ * @throws StorageContainerException if there is an error during stream initialization
+ */
+ @Override
+ public String streamInit(Container container, BlockID blockID) throws StorageContainerException {
+ return selectHandler(container).streamInit(container, blockID);
}
@Override
- public StateMachine.DataChannel getStreamDataChannel(
- Container container, BlockID blockID, ContainerMetrics metrics)
- throws StorageContainerException {
- return selectHandler(container)
- .getStreamDataChannel(container, blockID, metrics);
+ public StateMachine.DataChannel getStreamDataChannel(Container container, BlockID blockID, ContainerMetrics metrics)
+ throws StorageContainerException {
+ return selectHandler(container).getStreamDataChannel(container, blockID, metrics);
}
@Override
- public void finishWriteChunks(KeyValueContainer kvContainer,
- BlockData blockData) throws IOException {
+ public void finishWriteChunks(KeyValueContainer kvContainer, BlockData blockData) throws IOException {
- selectHandler(kvContainer)
- .finishWriteChunks(kvContainer, blockData);
+ selectHandler(kvContainer).finishWriteChunks(kvContainer, blockData);
}
@Override
- public void finalizeWriteChunk(KeyValueContainer kvContainer,
- BlockID blockId) throws IOException {
+ public void finalizeWriteChunk(KeyValueContainer kvContainer, BlockID blockId) throws IOException {
selectHandler(kvContainer).finalizeWriteChunk(kvContainer, blockId);
}
@Override
- public ChunkBuffer readChunk(Container container, BlockID blockID,
- ChunkInfo info, DispatcherContext dispatcherContext)
- throws StorageContainerException {
+ public ChunkBuffer readChunk(Container container, BlockID blockID, ChunkInfo info,
+ DispatcherContext dispatcherContext) throws StorageContainerException {
- ChunkBuffer data = selectHandler(container)
- .readChunk(container, blockID, info, dispatcherContext);
+ ChunkBuffer data = selectHandler(container).readChunk(container, blockID, info, dispatcherContext);
Preconditions.checkState(data != null);
container.getContainerData().updateReadStats(data.remaining());
@@ -118,28 +111,23 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
}
@Override
- public void deleteChunk(Container container, BlockID blockID, ChunkInfo info)
- throws StorageContainerException {
-
+ public void deleteChunk(Container container, BlockID blockID, ChunkInfo info) throws StorageContainerException {
Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
// Delete the chunk from disk.
- // Do not decrement the ContainerData counters (usedBytes) here as it
- // will be updated while deleting the block from the DB
+ // Do not decrement the ContainerData counters (usedBytes)
+ // here as it will be updated while deleting the block from the DB
selectHandler(container).deleteChunk(container, blockID, info);
-
}
@Override
- public void deleteChunks(Container container, BlockData blockData)
- throws StorageContainerException {
-
+ public void deleteChunks(Container container, BlockData blockData) throws StorageContainerException {
Preconditions.checkNotNull(blockData, "Block data cannot be null.");
// Delete the chunks belonging to blockData.
- // Do not decrement the ContainerData counters (usedBytes) here as it
- // will be updated while deleting the block from the DB
+ // Do not decrement the ContainerData counters (usedBytes)
+ // here as it will be updated while deleting the block from the DB
selectHandler(container).deleteChunks(container, blockData);
}
@@ -149,17 +137,12 @@ public void shutdown() {
handlers.values().forEach(ChunkManager::shutdown);
}
- private @Nonnull ChunkManager selectHandler(Container container)
- throws StorageContainerException {
-
- ContainerLayoutVersion layout =
- container.getContainerData().getLayoutVersion();
+ private @Nonnull ChunkManager selectHandler(Container container) throws StorageContainerException {
+ ContainerLayoutVersion layout = container.getContainerData().getLayoutVersion();
return selectVersionHandler(layout);
}
- private @Nonnull ChunkManager selectVersionHandler(
- ContainerLayoutVersion version)
- throws StorageContainerException {
+ private @Nonnull ChunkManager selectVersionHandler(ContainerLayoutVersion version) throws StorageContainerException {
ChunkManager versionHandler = handlers.get(version);
if (versionHandler == null) {
return throwUnknownLayoutVersion(version);
@@ -167,13 +150,12 @@ public void shutdown() {
return versionHandler;
}
- private static ChunkManager throwUnknownLayoutVersion(
- ContainerLayoutVersion version) throws StorageContainerException {
+ private static ChunkManager throwUnknownLayoutVersion(ContainerLayoutVersion version)
+ throws StorageContainerException {
String message = "Unsupported storage container layout: " + version;
LOG.warn(message);
// TODO pick best result code
throw new StorageContainerException(message, UNSUPPORTED_REQUEST);
}
-
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java
index 4ca578d7717..3ac7cafea74 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java
@@ -32,7 +32,6 @@
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@@ -68,8 +67,7 @@
*/
public class FilePerBlockStrategy implements ChunkManager {
- private static final Logger LOG =
- LoggerFactory.getLogger(FilePerBlockStrategy.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FilePerBlockStrategy.class);
private final boolean doSyncWrite;
private final OpenFiles files = new OpenFiles();
@@ -77,19 +75,13 @@ public class FilePerBlockStrategy implements ChunkManager {
private final int readMappedBufferThreshold;
private final int readMappedBufferMaxCount;
private final MappedBufferManager mappedBufferManager;
- private final VolumeSet volumeSet;
- public FilePerBlockStrategy(boolean sync, BlockManager manager,
- VolumeSet volSet) {
+ public FilePerBlockStrategy(boolean sync, BlockManager manager) {
doSyncWrite = sync;
- this.defaultReadBufferCapacity = manager == null ? 0 :
- manager.getDefaultReadBufferCapacity();
- this.readMappedBufferThreshold = manager == null ? 0
- : manager.getReadMappedBufferThreshold();
- this.readMappedBufferMaxCount = manager == null ? 0
- : manager.getReadMappedBufferMaxCount();
+ this.defaultReadBufferCapacity = manager == null ? 0 : manager.getDefaultReadBufferCapacity();
+ this.readMappedBufferThreshold = manager == null ? 0 : manager.getReadMappedBufferThreshold();
+ this.readMappedBufferMaxCount = manager == null ? 0 : manager.getReadMappedBufferMaxCount();
LOG.info("ozone.chunk.read.mapped.buffer.max.count is load with {}", readMappedBufferMaxCount);
- this.volumeSet = volSet;
if (this.readMappedBufferMaxCount > 0) {
mappedBufferManager = new MappedBufferManager(this.readMappedBufferMaxCount);
} else {
@@ -98,32 +90,27 @@ public FilePerBlockStrategy(boolean sync, BlockManager manager,
}
private static void checkLayoutVersion(Container container) {
- Preconditions.checkArgument(
- container.getContainerData().getLayoutVersion() == FILE_PER_BLOCK);
+ Preconditions.checkArgument(container.getContainerData().getLayoutVersion() == FILE_PER_BLOCK);
}
@Override
- public String streamInit(Container container, BlockID blockID)
- throws StorageContainerException {
+ public String streamInit(Container container, BlockID blockID) throws StorageContainerException {
checkLayoutVersion(container);
final File chunkFile = getChunkFile(container, blockID);
return chunkFile.getAbsolutePath();
}
@Override
- public StateMachine.DataChannel getStreamDataChannel(
- Container container, BlockID blockID, ContainerMetrics metrics)
- throws StorageContainerException {
+ public StateMachine.DataChannel getStreamDataChannel(Container container, BlockID blockID, ContainerMetrics metrics)
+ throws StorageContainerException {
checkLayoutVersion(container);
final File chunkFile = getChunkFile(container, blockID);
- return new KeyValueStreamDataChannel(chunkFile,
- container.getContainerData(), metrics);
+ return new KeyValueStreamDataChannel(chunkFile, container.getContainerData(), metrics);
}
@Override
- public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
- ChunkBuffer data, DispatcherContext dispatcherContext)
- throws StorageContainerException {
+ public void writeChunk(Container container, BlockID blockID, ChunkInfo info, ChunkBuffer data,
+ DispatcherContext dispatcherContext) throws StorageContainerException {
checkLayoutVersion(container);
@@ -131,21 +118,16 @@ public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
DispatcherContext.WriteChunkStage stage = dispatcherContext.getStage();
if (info.getLen() <= 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Skip writing empty chunk {} in stage {}", info, stage);
- }
+ LOG.debug("Skip writing empty chunk {} in stage {}", info, stage);
return;
}
if (stage == COMMIT_DATA) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Ignore chunk {} in stage {}", info, stage);
- }
+ LOG.debug("Ignore chunk {} in stage {}", info, stage);
return;
}
- KeyValueContainerData containerData = (KeyValueContainerData) container
- .getContainerData();
+ KeyValueContainerData containerData = (KeyValueContainerData) container.getContainerData();
final File chunkFile = getChunkFile(container, blockID);
long len = info.getLen();
@@ -153,7 +135,7 @@ public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
HddsVolume volume = containerData.getVolume();
- FileChannel channel = null;
+ FileChannel channel;
boolean overwrite;
try {
channel = files.getChannel(chunkFile, doSyncWrite);
@@ -163,26 +145,21 @@ public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
throw e;
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("Writing chunk {} (overwrite: {}) in stage {} to file {}",
- info, overwrite, stage, chunkFile);
- }
+ LOG.debug("Writing chunk {} (overwrite: {}) in stage {} to file {}", info, overwrite, stage, chunkFile);
- // check whether offset matches block file length if its an overwrite
+ // Check whether offset matches block file length if it's overwritten.
if (!overwrite) {
ChunkUtils.validateChunkSize(channel, info, chunkFile.getName());
}
- ChunkUtils
- .writeData(channel, chunkFile.getName(), data, offset, len, volume);
+ ChunkUtils.writeData(channel, chunkFile.getName(), data, offset, len, volume);
containerData.updateWriteStats(len, overwrite);
}
@Override
- public ChunkBuffer readChunk(Container container, BlockID blockID,
- ChunkInfo info, DispatcherContext dispatcherContext)
- throws StorageContainerException {
+ public ChunkBuffer readChunk(Container container, BlockID blockID, ChunkInfo info,
+ DispatcherContext dispatcherContext) throws StorageContainerException {
checkLayoutVersion(container);
@@ -193,8 +170,7 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
limitReadSize(info.getLen());
- KeyValueContainerData containerData = (KeyValueContainerData) container
- .getContainerData();
+ KeyValueContainerData containerData = (KeyValueContainerData) container.getContainerData();
HddsVolume volume = containerData.getVolume();
@@ -202,27 +178,23 @@ public ChunkBuffer readChunk(Container container, BlockID blockID,
final long len = info.getLen();
long offset = info.getOffset();
- int bufferCapacity = ChunkManager.getBufferCapacityForChunkRead(info,
- defaultReadBufferCapacity);
- return ChunkUtils.readData(len, bufferCapacity, chunkFile, offset, volume,
- readMappedBufferThreshold, readMappedBufferMaxCount > 0, mappedBufferManager);
+ int bufferCapacity = ChunkManager.getBufferCapacityForChunkRead(info, defaultReadBufferCapacity);
+ return ChunkUtils.readData(len, bufferCapacity, chunkFile, offset, volume, readMappedBufferThreshold,
+ readMappedBufferMaxCount > 0, mappedBufferManager);
}
@Override
- public void deleteChunk(Container container, BlockID blockID, ChunkInfo info)
- throws StorageContainerException {
+ public void deleteChunk(Container container, BlockID blockID, ChunkInfo info) throws StorageContainerException {
deleteChunk(container, blockID, info, true);
}
@Override
- public void deleteChunks(Container container, BlockData blockData)
- throws StorageContainerException {
+ public void deleteChunks(Container container, BlockData blockData) throws StorageContainerException {
deleteChunk(container, blockData.getBlockID(), null, false);
}
@Override
- public void finishWriteChunks(KeyValueContainer container,
- BlockData blockData) throws IOException {
+ public void finishWriteChunks(KeyValueContainer container, BlockData blockData) throws IOException {
final File chunkFile = getChunkFile(container, blockData.getBlockID());
try {
files.close(chunkFile);
@@ -234,8 +206,7 @@ public void finishWriteChunks(KeyValueContainer container,
}
@Override
- public void finalizeWriteChunk(KeyValueContainer container,
- BlockID blockId) throws IOException {
+ public void finalizeWriteChunk(KeyValueContainer container, BlockID blockId) throws IOException {
synchronized (container) {
File chunkFile = getChunkFile(container, blockId);
try {
@@ -250,8 +221,7 @@ public void finalizeWriteChunk(KeyValueContainer container,
}
}
- private void deleteChunk(Container container, BlockID blockID,
- ChunkInfo info, boolean verifyLength)
+ private void deleteChunk(Container container, BlockID blockID, ChunkInfo info, boolean verifyLength)
throws StorageContainerException {
checkLayoutVersion(container);
@@ -259,17 +229,15 @@ private void deleteChunk(Container container, BlockID blockID,
final File file = getChunkFile(container, blockID);
- // if the chunk file does not exist, it might have already been deleted.
- // The call might be because of reapply of transactions on datanode
- // restart.
+ // If the chunk file does not exist, it might have already been deleted.
+ // The call might be because of reapplied of transactions on datanode restart.
if (!file.exists()) {
LOG.warn("Block file to be deleted does not exist: {}", file);
return;
}
if (verifyLength) {
- Preconditions.checkNotNull(info, "Chunk info cannot be null for single " +
- "chunk delete");
+ Preconditions.checkNotNull(info, "Chunk info cannot be null for single chunk delete");
checkFullDelete(info, file);
}
@@ -281,13 +249,13 @@ private static File getChunkFile(Container container, BlockID blockID) throws St
return FILE_PER_BLOCK.getChunkFile(container.getContainerData(), blockID, null);
}
- private static void checkFullDelete(ChunkInfo info, File chunkFile)
- throws StorageContainerException {
+ private static void checkFullDelete(ChunkInfo info, File chunkFile) throws StorageContainerException {
long fileLength = chunkFile.length();
if ((info.getOffset() > 0) || (info.getLen() != fileLength)) {
- String msg = String.format(
- "Trying to delete partial chunk %s from file %s with length %s",
- info, chunkFile, fileLength);
+ String msg = String.format("Trying to delete partial chunk %s from file %s with length %s",
+ info,
+ chunkFile,
+ fileLength);
LOG.error(msg);
throw new StorageContainerException(msg, UNSUPPORTED_REQUEST);
}
@@ -295,25 +263,21 @@ private static void checkFullDelete(ChunkInfo info, File chunkFile)
private static final class OpenFiles {
- private static final RemovalListener ON_REMOVE =
- event -> close(event.getKey(), event.getValue());
+ private static final RemovalListener ON_REMOVE = event -> close(event.getKey(), event.getValue());
private final Cache