diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 82356bafa595..f540c1df77a3 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -147,7 +147,7 @@ jobs:
distribution: 'temurin'
java-version: ${{ matrix.java }}
- name: Compile Ozone using Java ${{ matrix.java }}
- run: hadoop-ozone/dev-support/checks/build.sh -Dskip.npx -Dskip.installnpx
+ run: hadoop-ozone/dev-support/checks/build.sh -Dskip.npx -Dskip.installnpx -Djavac.version=${{ matrix.java }}
- name: Delete temporary build artifacts before caching
run: |
#Never cache local artifacts
diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml
index c38a0b058d49..c1678f12c8e4 100644
--- a/.github/workflows/post-commit.yml
+++ b/.github/workflows/post-commit.yml
@@ -23,3 +23,4 @@ concurrency:
jobs:
CI:
uses: ./.github/workflows/ci.yml
+ secrets: inherit
diff --git a/.github/workflows/scheduled_ci.yml b/.github/workflows/scheduled_ci.yml
index b89f52a4f189..86c3943999b7 100644
--- a/.github/workflows/scheduled_ci.yml
+++ b/.github/workflows/scheduled_ci.yml
@@ -19,3 +19,4 @@ on:
jobs:
CI:
uses: ./.github/workflows/ci.yml
+ secrets: inherit
diff --git a/LICENSE.txt b/LICENSE.txt
index 76e979f2837c..021266844b82 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -207,8 +207,8 @@ licenses. This section summarizes those components and their licenses.
See licenses/ for text of these licenses.
-Apache Software Foundation License 2.0
---------------------------------------
+Apache License 2.0
+------------------
hadoop-hdds/framework/src/main/resources/webapps/static/nvd3-1.8.5.min.js.map
hadoop-hdds/framework/src/main/resources/webapps/static/nvd3-1.8.5.min.css.map
@@ -221,15 +221,15 @@ BSD 3-Clause
------------
hadoop-hdds/framework/src/main/resources/webapps/static/d3-3.5.17.min.js
-hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/d3-3.5.17.min.js
-hadoop-hdds/docs/themes/ozonedoc/static/fonts/glyphicons-*
-hadoop-hdds/docs/themes/ozonedoc/static/js/bootstrap.min.js
+
MIT License
-----------
hadoop-hdds/framework/src/main/resources/webapps/static/bootstrap-3.4.1
hadoop-hdds/docs/themes/ozonedoc/static/css/bootstrap-*
+hadoop-hdds/docs/themes/ozonedoc/static/js/bootstrap.min.js
+hadoop-hdds/docs/themes/ozonedoc/static/fonts/glyphicons-*
hadoop-hdds/framework/src/main/resources/webapps/static/angular-route-1.8.0.min.js
hadoop-hdds/framework/src/main/resources/webapps/static/angular-nvd3-1.0.9.min.js
diff --git a/NOTICE.txt b/NOTICE.txt
index 3a7cf6effe57..7a1e855f6a33 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -1,5 +1,5 @@
Apache Ozone
-Copyright 2021 The Apache Software Foundation
+Copyright 2022 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
diff --git a/hadoop-hdds/annotations/pom.xml b/hadoop-hdds/annotations/pom.xml
index cbef3d46715d..582e232889d5 100644
--- a/hadoop-hdds/annotations/pom.xml
+++ b/hadoop-hdds/annotations/pom.xml
@@ -20,11 +20,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozonehdds
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOThdds-annotation-processing
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOTApache Ozone annotation processing tools for validating custom
annotations at compile time.
diff --git a/hadoop-hdds/client/pom.xml b/hadoop-hdds/client/pom.xml
index 3e3ff0cb9f8c..fd44ffc2432e 100644
--- a/hadoop-hdds/client/pom.xml
+++ b/hadoop-hdds/client/pom.xml
@@ -20,11 +20,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozonehdds
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOThdds-client
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOTApache Ozone Distributed Data Store Client LibraryApache Ozone HDDS Clientjar
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
index 14a8aacbc015..e2c5471be8e4 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
@@ -66,6 +66,37 @@ public enum ChecksumCombineMode {
tags = ConfigTag.CLIENT)
private int streamBufferSize = 4 * 1024 * 1024;
+ @Config(key = "datastream.buffer.flush.size",
+ defaultValue = "16MB",
+ type = ConfigType.SIZE,
+ description = "The boundary at which putBlock is executed",
+ tags = ConfigTag.CLIENT)
+ private long dataStreamBufferFlushSize = 16 * 1024 * 1024;
+
+ @Config(key = "datastream.min.packet.size",
+ defaultValue = "1MB",
+ type = ConfigType.SIZE,
+ description = "The maximum size of the ByteBuffer "
+ + "(used via ratis streaming)",
+ tags = ConfigTag.CLIENT)
+ private int dataStreamMinPacketSize = 1024 * 1024;
+
+ @Config(key = "datastream.window.size",
+ defaultValue = "64MB",
+ type = ConfigType.SIZE,
+ description = "Maximum size of BufferList(used for retry) size per " +
+ "BlockDataStreamOutput instance",
+ tags = ConfigTag.CLIENT)
+ private long streamWindowSize = 64 * 1024 * 1024;
+
+ @Config(key = "datastream.pipeline.mode",
+ defaultValue = "true",
+ description = "Streaming write support both pipeline mode(datanode1->" +
+ "datanode2->datanode3) and star mode(datanode1->datanode2, " +
+ "datanode1->datanode3). By default we use pipeline mode.",
+ tags = ConfigTag.CLIENT)
+ private boolean datastreamPipelineMode = true;
+
@Config(key = "stream.buffer.increment",
defaultValue = "0B",
type = ConfigType.SIZE,
@@ -143,6 +174,13 @@ public enum ChecksumCombineMode {
tags = ConfigTag.CLIENT)
private int maxECStripeWriteRetries = 10;
+ @Config(key = "ec.stripe.queue.size",
+ defaultValue = "2",
+ description = "The max number of EC stripes can be buffered in client " +
+ " before flushing into datanodes.",
+ tags = ConfigTag.CLIENT)
+ private int ecStripeQueueSize = 2;
+
@Config(key = "exclude.nodes.expiry.time",
defaultValue = "600000",
description = "Time after which an excluded node is reconsidered for" +
@@ -244,6 +282,22 @@ public void setStreamBufferMaxSize(long streamBufferMaxSize) {
this.streamBufferMaxSize = streamBufferMaxSize;
}
+ public int getDataStreamMinPacketSize() {
+ return dataStreamMinPacketSize;
+ }
+
+ public void setDataStreamMinPacketSize(int dataStreamMinPacketSize) {
+ this.dataStreamMinPacketSize = dataStreamMinPacketSize;
+ }
+
+ public long getStreamWindowSize() {
+ return streamWindowSize;
+ }
+
+ public void setStreamWindowSize(long streamWindowSize) {
+ this.streamWindowSize = streamWindowSize;
+ }
+
public int getMaxRetryCount() {
return maxRetryCount;
}
@@ -288,6 +342,10 @@ public int getMaxECStripeWriteRetries() {
return this.maxECStripeWriteRetries;
}
+ public int getEcStripeQueueSize() {
+ return this.ecStripeQueueSize;
+ }
+
public long getExcludeNodesExpiryTime() {
return excludeNodesExpiryTime;
}
@@ -296,6 +354,14 @@ public int getBufferIncrement() {
return bufferIncrement;
}
+ public long getDataStreamBufferFlushSize() {
+ return dataStreamBufferFlushSize;
+ }
+
+ public void setDataStreamBufferFlushSize(long dataStreamBufferFlushSize) {
+ this.dataStreamBufferFlushSize = dataStreamBufferFlushSize;
+ }
+
public ChecksumCombineMode getChecksumCombineMode() {
try {
return ChecksumCombineMode.valueOf(checksumCombineMode);
@@ -325,4 +391,12 @@ public void setFsDefaultBucketLayout(String bucketLayout) {
public String getFsDefaultBucketLayout() {
return fsDefaultBucketLayout;
}
+
+ public boolean isDatastreamPipelineMode() {
+ return datastreamPipelineMode;
+ }
+
+ public void setDatastreamPipelineMode(boolean datastreamPipelineMode) {
+ this.datastreamPipelineMode = datastreamPipelineMode;
+ }
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index cab7bee7801c..1fee58c3b45e 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -198,7 +198,7 @@ protected NettyChannelBuilder createChannel(DatanodeDetails dn, int port)
NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
.intercept(new GrpcClientInterceptor());
- if (secConfig.isGrpcTlsEnabled()) {
+ if (secConfig.isSecurityEnabled() && secConfig.isGrpcTlsEnabled()) {
SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient();
if (caCerts != null) {
sslContextBuilder.trustManager(caCerts);
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
index d0fd0db12950..3ea269b08b08 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
@@ -53,6 +53,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.api.DataStreamApi;
import org.apache.ratis.grpc.GrpcTlsConfig;
import org.apache.ratis.proto.RaftProtos;
import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
@@ -135,7 +136,7 @@ private long updateCommitInfosMap(RaftClientReply reply) {
.orElse(0L);
}
- private long updateCommitInfosMap(
+ public long updateCommitInfosMap(
Collection commitInfoProtos) {
// if the commitInfo map is empty, just update the commit indexes for each
// of the servers
@@ -382,4 +383,8 @@ public XceiverClientReply sendCommandAsync(
throw new UnsupportedOperationException(
"Operation Not supported for ratis client");
}
+
+ public DataStreamApi getDataStreamApi() {
+ return this.getClient().getDataStreamApi();
+ }
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractDataStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractDataStreamOutput.java
new file mode 100644
index 000000000000..cad1d0479249
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractDataStreamOutput.java
@@ -0,0 +1,131 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
+import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * This class is used for error handling methods.
+ */
+public abstract class AbstractDataStreamOutput
+ implements ByteBufferStreamOutput {
+
+ private final Map, RetryPolicy> retryPolicyMap;
+ private int retryCount;
+ private boolean isException;
+
+ protected AbstractDataStreamOutput(
+ Map, RetryPolicy> retryPolicyMap) {
+ this.retryPolicyMap = retryPolicyMap;
+ this.isException = false;
+ this.retryCount = 0;
+ }
+
+ @VisibleForTesting
+ public int getRetryCount() {
+ return retryCount;
+ }
+
+ protected void resetRetryCount() {
+ retryCount = 0;
+ }
+
+ protected boolean isException() {
+ return isException;
+ }
+
+ /**
+ * Checks if the provided exception signifies retry failure in ratis client.
+ * In case of retry failure, ratis client throws RaftRetryFailureException
+ * and all succeeding operations are failed with AlreadyClosedException.
+ */
+ protected boolean checkForRetryFailure(Throwable t) {
+ return t instanceof RaftRetryFailureException
+ || t instanceof AlreadyClosedException;
+ }
+
+ // Every container specific exception from datatnode will be seen as
+ // StorageContainerException
+ protected boolean checkIfContainerToExclude(Throwable t) {
+ return t instanceof StorageContainerException;
+ }
+
+ protected void setExceptionAndThrow(IOException ioe) throws IOException {
+ isException = true;
+ throw ioe;
+ }
+
+ protected void handleRetry(IOException exception) throws IOException {
+ RetryPolicy retryPolicy = retryPolicyMap
+ .get(HddsClientUtils.checkForException(exception).getClass());
+ if (retryPolicy == null) {
+ retryPolicy = retryPolicyMap.get(Exception.class);
+ }
+ handleRetry(exception, retryPolicy);
+ }
+
+ protected void handleRetry(IOException exception, RetryPolicy retryPolicy)
+ throws IOException {
+ RetryPolicy.RetryAction action = null;
+ try {
+ action = retryPolicy.shouldRetry(exception, retryCount, 0, true);
+ } catch (Exception e) {
+ setExceptionAndThrow(new IOException(e));
+ }
+ if (action != null &&
+ action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+ String msg = "";
+ if (action.reason != null) {
+ msg = "Retry request failed. " + action.reason;
+ }
+ setExceptionAndThrow(new IOException(msg, exception));
+ }
+
+ // Throw the exception if the thread is interrupted
+ if (Thread.currentThread().isInterrupted()) {
+ setExceptionAndThrow(exception);
+ }
+ Objects.requireNonNull(action);
+ Preconditions.checkArgument(
+ action.action == RetryPolicy.RetryAction.RetryDecision.RETRY);
+ if (action.delayMillis > 0) {
+ try {
+ Thread.sleep(action.delayMillis);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ IOException ioe = (IOException) new InterruptedIOException(
+ "Interrupted: action=" + action + ", retry policy=" + retryPolicy)
+ .initCause(e);
+ setExceptionAndThrow(ioe);
+ }
+ }
+ retryCount++;
+ }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
new file mode 100644
index 000000000000..d19f2aea1300
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
@@ -0,0 +1,733 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+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.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.ratis.RatisHelper;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.XceiverClientMetrics;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link ByteBufferStreamOutput} used by the REST service in combination
+ * with the SCMClient to write the value of a key to a sequence
+ * of container chunks. Writes are buffered locally and periodically written to
+ * the container as a new chunk. In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier. This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data. The list of chunks is updated all at once. Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput implements ByteBufferStreamOutput {
+ public static final Logger LOG =
+ LoggerFactory.getLogger(BlockDataStreamOutput.class);
+
+ public static final int PUT_BLOCK_REQUEST_LENGTH_MAX = 1 << 20; // 1MB
+
+ public static final String EXCEPTION_MSG =
+ "Unexpected Storage Container Exception: ";
+ private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+ private AtomicReference blockID;
+
+ private final BlockData.Builder containerBlockData;
+ private XceiverClientFactory xceiverClientFactory;
+ private XceiverClientRatis xceiverClient;
+ private OzoneClientConfig config;
+
+ private int chunkIndex;
+ private final AtomicLong chunkOffset = new AtomicLong();
+
+ // Similar to 'BufferPool' but this list maintains only references
+ // to the ByteBuffers.
+ private List bufferList;
+
+ // The IOException will be set by response handling thread in case there is an
+ // exception received in the response. If the exception is set, the next
+ // request will fail upfront.
+ private final AtomicReference ioException;
+ private final ExecutorService responseExecutor;
+
+ // the effective length of data flushed so far
+ private long totalDataFlushedLength;
+
+ // effective data write attempted so far for the block
+ private long writtenDataLength;
+
+ // This object will maintain the commitIndexes and byteBufferList in order
+ // Also, corresponding to the logIndex, the corresponding list of buffers will
+ // be released from the buffer pool.
+ private final StreamCommitWatcher commitWatcher;
+
+ private Queue>
+ putBlockFutures = new LinkedList<>();
+
+ private final List failedServers;
+ private final Checksum checksum;
+
+ //number of buffers used before doing a flush/putBlock.
+ private int flushPeriod;
+ private final Token extends TokenIdentifier> token;
+ private final DataStreamOutput out;
+ private CompletableFuture dataStreamCloseReply;
+ private List> futures = new ArrayList<>();
+ private final long syncSize = 0; // TODO: disk sync is disabled for now
+ private long syncPosition = 0;
+ private StreamBuffer currentBuffer;
+ private XceiverClientMetrics metrics;
+ // buffers for which putBlock is yet to be executed
+ private List buffersForPutBlock;
+ private boolean isDatastreamPipelineMode;
+ /**
+ * Creates a new BlockDataStreamOutput.
+ *
+ * @param blockID block ID
+ * @param xceiverClientManager client manager that controls client
+ * @param pipeline pipeline where block will be written
+ */
+ public BlockDataStreamOutput(
+ BlockID blockID,
+ XceiverClientFactory xceiverClientManager,
+ Pipeline pipeline,
+ OzoneClientConfig config,
+ Token extends TokenIdentifier> token,
+ List bufferList
+ ) throws IOException {
+ this.xceiverClientFactory = xceiverClientManager;
+ this.config = config;
+ this.isDatastreamPipelineMode = config.isDatastreamPipelineMode();
+ this.blockID = new AtomicReference<>(blockID);
+ KeyValue keyValue =
+ KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+ this.containerBlockData =
+ BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+ .addMetadata(keyValue);
+ this.xceiverClient =
+ (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+ // Alternatively, stream setup can be delayed till the first chunk write.
+ this.out = setupStream(pipeline);
+ this.token = token;
+ this.bufferList = bufferList;
+ flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+ .getStreamBufferSize());
+
+ Preconditions
+ .checkArgument(
+ (long) flushPeriod * config.getStreamBufferSize() == config
+ .getStreamBufferFlushSize());
+
+ // A single thread executor handle the responses of async requests
+ responseExecutor = Executors.newSingleThreadExecutor();
+ commitWatcher = new StreamCommitWatcher(xceiverClient, bufferList);
+ totalDataFlushedLength = 0;
+ writtenDataLength = 0;
+ failedServers = new ArrayList<>(0);
+ ioException = new AtomicReference<>(null);
+ checksum = new Checksum(config.getChecksumType(),
+ config.getBytesPerChecksum());
+ metrics = XceiverClientManager.getXceiverClientMetrics();
+ }
+
+ private DataStreamOutput setupStream(Pipeline pipeline) throws IOException {
+ // Execute a dummy WriteChunk request to get the path of the target file,
+ // but does NOT write any data to it.
+ ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+ ContainerProtos.WriteChunkRequestProto.newBuilder()
+ .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+ String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+ ContainerProtos.ContainerCommandRequestProto.Builder builder =
+ ContainerProtos.ContainerCommandRequestProto.newBuilder()
+ .setCmdType(ContainerProtos.Type.StreamInit)
+ .setContainerID(blockID.get().getContainerID())
+ .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+ ContainerCommandRequestMessage message =
+ ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+ if (isDatastreamPipelineMode) {
+ return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+ .stream(message.getContent().asReadOnlyByteBuffer(),
+ RatisHelper.getRoutingTable(pipeline));
+ } else {
+ return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+ .stream(message.getContent().asReadOnlyByteBuffer());
+ }
+ }
+
+ public BlockID getBlockID() {
+ return blockID.get();
+ }
+
+ public long getWrittenDataLength() {
+ return writtenDataLength;
+ }
+
+ public List getFailedServers() {
+ return failedServers;
+ }
+
+ @VisibleForTesting
+ public XceiverClientRatis getXceiverClient() {
+ return xceiverClient;
+ }
+
+ public IOException getIoException() {
+ return ioException.get();
+ }
+
+ @Override
+ public void write(ByteBuffer b, int off, int len) throws IOException {
+ checkOpen();
+ if (b == null) {
+ throw new NullPointerException();
+ }
+ if (len == 0) {
+ return;
+ }
+ while (len > 0) {
+ allocateNewBufferIfNeeded();
+ int writeLen = Math.min(len, currentBuffer.length());
+ final StreamBuffer buf = new StreamBuffer(b, off, writeLen);
+ currentBuffer.put(buf);
+ writeChunkIfNeeded();
+ off += writeLen;
+ writtenDataLength += writeLen;
+ len -= writeLen;
+ doFlushIfNeeded();
+ }
+ }
+
+ private void writeChunkIfNeeded() throws IOException {
+ if (currentBuffer.length() == 0) {
+ writeChunk(currentBuffer);
+ currentBuffer = null;
+ }
+ }
+
+ private void writeChunk(StreamBuffer sb) throws IOException {
+ bufferList.add(sb);
+ if (buffersForPutBlock == null) {
+ buffersForPutBlock = new ArrayList<>();
+ }
+ buffersForPutBlock.add(sb);
+ ByteBuffer dup = sb.duplicate();
+ dup.position(0);
+ dup.limit(sb.position());
+ writeChunkToContainer(dup);
+ }
+
+ private void allocateNewBufferIfNeeded() {
+ if (currentBuffer == null) {
+ currentBuffer =
+ StreamBuffer.allocate(config.getDataStreamMinPacketSize());
+ }
+ }
+
+ private void doFlushIfNeeded() throws IOException {
+ long boundary = config.getDataStreamBufferFlushSize() / config
+ .getDataStreamMinPacketSize();
+ // streamWindow is the maximum number of buffers that
+ // are allowed to exist in the bufferList. If buffers in
+ // the list exceed this limit , client will till it gets
+ // one putBlockResponse (first index) . This is similar to
+ // the bufferFull condition in async write path.
+ long streamWindow = config.getStreamWindowSize() / config
+ .getDataStreamMinPacketSize();
+ if (!bufferList.isEmpty() && bufferList.size() % boundary == 0 &&
+ buffersForPutBlock != null && !buffersForPutBlock.isEmpty()) {
+ updateFlushLength();
+ executePutBlock(false, false);
+ }
+ if (bufferList.size() == streamWindow) {
+ try {
+ checkOpen();
+ if (!putBlockFutures.isEmpty()) {
+ putBlockFutures.remove().get();
+ }
+ } catch (ExecutionException e) {
+ handleExecutionException(e);
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ handleInterruptedException(ex, true);
+ }
+ watchForCommit(true);
+ }
+ }
+
+ private void updateFlushLength() {
+ totalDataFlushedLength = writtenDataLength;
+ }
+
+ @VisibleForTesting
+ public long getTotalDataFlushedLength() {
+ return totalDataFlushedLength;
+ }
+ /**
+ * Will be called on the retryPath in case closedContainerException/
+ * TimeoutException.
+ * @param len length of data to write
+ * @throws IOException if error occurred
+ */
+
+ public void writeOnRetry(long len) throws IOException {
+ if (len == 0) {
+ return;
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+ }
+ int count = 0;
+ while (len > 0) {
+ final StreamBuffer buf = bufferList.get(count);
+ final long writeLen = Math.min(buf.position(), len);
+ if (buffersForPutBlock == null) {
+ buffersForPutBlock = new ArrayList<>();
+ }
+ buffersForPutBlock.add(buf);
+ final ByteBuffer duplicated = buf.duplicate();
+ duplicated.position(0);
+ duplicated.limit(buf.position());
+ writeChunkToContainer(duplicated);
+ len -= writeLen;
+ count++;
+ writtenDataLength += writeLen;
+ }
+
+
+ }
+
+ /**
+ * calls watchForCommit API of the Ratis Client. For Standalone client,
+ * it is a no op.
+ * @param bufferFull flag indicating whether bufferFull condition is hit or
+ * its called as part flush/close
+ * @return minimum commit index replicated to all nodes
+ * @throws IOException IOException in case watch gets timed out
+ */
+ private void watchForCommit(boolean bufferFull) throws IOException {
+ checkOpen();
+ try {
+ XceiverClientReply reply = bufferFull ?
+ commitWatcher.streamWatchOnFirstIndex() :
+ commitWatcher.streamWatchOnLastIndex();
+ if (reply != null) {
+ List dnList = reply.getDatanodes();
+ if (!dnList.isEmpty()) {
+ Pipeline pipe = xceiverClient.getPipeline();
+
+ LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+ blockID, pipe, dnList);
+ failedServers.addAll(dnList);
+ }
+ }
+ } catch (IOException ioe) {
+ setIoException(ioe);
+ throw getIoException();
+ }
+
+ }
+
+ /**
+ * @param close whether putBlock is happening as part of closing the stream
+ * @param force true if no data was written since most recent putBlock and
+ * stream is being closed
+ */
+ private void executePutBlock(boolean close,
+ boolean force) throws IOException {
+ checkOpen();
+ long flushPos = totalDataFlushedLength;
+ final List byteBufferList;
+ if (!force) {
+ Preconditions.checkNotNull(bufferList);
+ byteBufferList = buffersForPutBlock;
+ buffersForPutBlock = null;
+ Preconditions.checkNotNull(byteBufferList);
+ } else {
+ byteBufferList = null;
+ }
+ waitFuturesComplete();
+ final BlockData blockData = containerBlockData.build();
+ if (close) {
+ final ContainerCommandRequestProto putBlockRequest
+ = ContainerProtocolCalls.getPutBlockRequest(
+ xceiverClient.getPipeline(), blockData, true, token);
+ dataStreamCloseReply = executePutBlockClose(putBlockRequest,
+ PUT_BLOCK_REQUEST_LENGTH_MAX, out);
+ dataStreamCloseReply.whenComplete((reply, e) -> {
+ if (e != null || reply == null || !reply.isSuccess()) {
+ LOG.warn("Failed executePutBlockClose, reply=" + reply, e);
+ try {
+ executePutBlock(true, false);
+ } catch (IOException ex) {
+ throw new CompletionException(ex);
+ }
+ }
+ });
+ }
+
+ try {
+ XceiverClientReply asyncReply =
+ putBlockAsync(xceiverClient, blockData, close, token);
+ final CompletableFuture flushFuture
+ = asyncReply.getResponse().thenApplyAsync(e -> {
+ try {
+ validateResponse(e);
+ } catch (IOException sce) {
+ throw new CompletionException(sce);
+ }
+ // if the ioException is not set, putBlock is successful
+ if (getIoException() == null && !force) {
+ BlockID responseBlockID = BlockID.getFromProtobuf(
+ e.getPutBlock().getCommittedBlockLength().getBlockID());
+ Preconditions.checkState(blockID.get().getContainerBlockID()
+ .equals(responseBlockID.getContainerBlockID()));
+ // updates the bcsId of the block
+ blockID.set(responseBlockID);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Adding index " + asyncReply.getLogIndex() +
+ " commitMap size "
+ + commitWatcher.getCommitInfoMapSize() + " flushLength "
+ + flushPos + " blockID " + blockID);
+ }
+ // for standalone protocol, logIndex will always be 0.
+ commitWatcher
+ .updateCommitInfoMap(asyncReply.getLogIndex(),
+ byteBufferList);
+ }
+ return e;
+ }, responseExecutor).exceptionally(e -> {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("putBlock failed for blockID {} with exception {}",
+ blockID, e.getLocalizedMessage());
+ }
+ CompletionException ce = new CompletionException(e);
+ setIoException(ce);
+ throw ce;
+ });
+ putBlockFutures.add(flushFuture);
+ } catch (IOException | ExecutionException e) {
+ throw new IOException(EXCEPTION_MSG + e.toString(), e);
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ handleInterruptedException(ex, false);
+ }
+ }
+
+ public static CompletableFuture executePutBlockClose(
+ ContainerCommandRequestProto putBlockRequest, int max,
+ DataStreamOutput out) {
+ final ByteBuffer putBlock = ContainerCommandRequestMessage.toMessage(
+ putBlockRequest, null).getContent().asReadOnlyByteBuffer();
+ final ByteBuffer protoLength = getProtoLength(putBlock, max);
+ RatisHelper.debug(putBlock, "putBlock", LOG);
+ out.writeAsync(putBlock);
+ RatisHelper.debug(protoLength, "protoLength", LOG);
+ return out.writeAsync(protoLength, StandardWriteOption.CLOSE);
+ }
+
+ public static ByteBuffer getProtoLength(ByteBuffer putBlock, int max) {
+ final int protoLength = putBlock.remaining();
+ Preconditions.checkState(protoLength <= max,
+ "protoLength== %s > max = %s", protoLength, max);
+ final ByteBuffer buffer = ByteBuffer.allocate(4);
+ buffer.putInt(protoLength);
+ buffer.flip();
+ LOG.debug("protoLength = {}", protoLength);
+ Preconditions.checkState(buffer.remaining() == 4);
+ return buffer.asReadOnlyBuffer();
+ }
+
+ @Override
+ public void flush() throws IOException {
+ if (xceiverClientFactory != null && xceiverClient != null
+ && !config.isStreamBufferFlushDelay()) {
+ waitFuturesComplete();
+ }
+ }
+
+ public void waitFuturesComplete() throws IOException {
+ try {
+ CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+ futures.clear();
+ } catch (Exception e) {
+ LOG.warn("Failed to write all chunks through stream: " + e);
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * @param close whether the flush is happening as part of closing the stream
+ */
+ private void handleFlush(boolean close)
+ throws IOException, InterruptedException, ExecutionException {
+ checkOpen();
+ // flush the last chunk data residing on the currentBuffer
+ if (totalDataFlushedLength < writtenDataLength) {
+ // This can be a partially filled chunk. Since we are flushing the buffer
+ // here, we just limit this buffer to the current position. So that next
+ // write will happen in new buffer
+
+ if (currentBuffer != null) {
+ writeChunk(currentBuffer);
+ currentBuffer = null;
+ }
+ updateFlushLength();
+ executePutBlock(close, false);
+ } else if (close) {
+ // forcing an "empty" putBlock if stream is being closed without new
+ // data since latest flush - we need to send the "EOF" flag
+ executePutBlock(true, true);
+ }
+ CompletableFuture.allOf(putBlockFutures.toArray(EMPTY_FUTURE_ARRAY)).get();
+ watchForCommit(false);
+ // just check again if the exception is hit while waiting for the
+ // futures to ensure flush has indeed succeeded
+
+ // irrespective of whether the commitIndex2flushedDataMap is empty
+ // or not, ensure there is no exception set
+ checkOpen();
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (xceiverClientFactory != null && xceiverClient != null) {
+ try {
+ handleFlush(true);
+ dataStreamCloseReply.get();
+ } catch (ExecutionException e) {
+ handleExecutionException(e);
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ handleInterruptedException(ex, true);
+ } finally {
+ cleanup(false);
+ }
+
+ }
+ }
+
+ private void validateResponse(
+ ContainerProtos.ContainerCommandResponseProto responseProto)
+ throws IOException {
+ try {
+ // if the ioException is already set, it means a prev request has failed
+ // just throw the exception. The current operation will fail with the
+ // original error
+ IOException exception = getIoException();
+ if (exception != null) {
+ throw exception;
+ }
+ ContainerProtocolCalls.validateContainerResponse(responseProto);
+ } catch (StorageContainerException sce) {
+ setIoException(sce);
+ throw sce;
+ }
+ }
+
+
+ private void setIoException(Throwable e) {
+ IOException ioe = getIoException();
+ if (ioe == null) {
+ IOException exception = new IOException(EXCEPTION_MSG + e.toString(), e);
+ ioException.compareAndSet(null, exception);
+ } else {
+ LOG.debug("Previous request had already failed with " + ioe.toString()
+ + " so subsequent request also encounters"
+ + " Storage Container Exception ", e);
+ }
+ }
+
+ public void cleanup(boolean invalidateClient) {
+ if (xceiverClientFactory != null) {
+ xceiverClientFactory.releaseClient(xceiverClient, invalidateClient);
+ }
+ xceiverClientFactory = null;
+ xceiverClient = null;
+ commitWatcher.cleanup();
+ responseExecutor.shutdown();
+ }
+
+ /**
+ * Checks if the stream is open or exception has occurred.
+ * If not, throws an exception.
+ *
+ * @throws IOException if stream is closed
+ */
+ private void checkOpen() throws IOException {
+ if (isClosed()) {
+ throw new IOException("BlockDataStreamOutput has been closed.");
+ } else if (getIoException() != null) {
+ throw getIoException();
+ }
+ }
+
+ public boolean isClosed() {
+ return xceiverClient == null;
+ }
+
+ private boolean needSync(long position) {
+ if (syncSize > 0) {
+ // TODO: or position >= fileLength
+ if (position - syncPosition >= syncSize) {
+ syncPosition = position;
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Writes buffered data as a new chunk to the container and saves chunk
+ * information to be used later in putKey call.
+ *
+ * @param buf chunk data to write, from position to limit
+ * @throws IOException if there is an I/O error while performing the call
+ * @throws OzoneChecksumException if there is an error while computing
+ * checksum
+ */
+ private void writeChunkToContainer(ByteBuffer buf)
+ throws IOException {
+ final int effectiveChunkSize = buf.remaining();
+ final long offset = chunkOffset.getAndAdd(effectiveChunkSize);
+ ChecksumData checksumData = checksum.computeChecksum(
+ buf.asReadOnlyBuffer());
+ ChunkInfo chunkInfo = ChunkInfo.newBuilder()
+ .setChunkName(blockID.get().getLocalID() + "_chunk_" + ++chunkIndex)
+ .setOffset(offset)
+ .setLen(effectiveChunkSize)
+ .setChecksumData(checksumData.getProtoBufMessage())
+ .build();
+ metrics.incrPendingContainerOpsMetrics(ContainerProtos.Type.WriteChunk);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Writing chunk {} length {} at offset {}",
+ chunkInfo.getChunkName(), effectiveChunkSize, offset);
+ }
+
+ CompletableFuture future =
+ (needSync(offset + effectiveChunkSize) ?
+ out.writeAsync(buf, StandardWriteOption.SYNC) :
+ out.writeAsync(buf))
+ .whenCompleteAsync((r, e) -> {
+ if (e != null || !r.isSuccess()) {
+ if (e == null) {
+ e = new IOException("result is not success");
+ }
+ String msg =
+ "Failed to write chunk " + chunkInfo.getChunkName() +
+ " " + "into block " + blockID;
+ LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage());
+ CompletionException ce = new CompletionException(msg, e);
+ setIoException(ce);
+ throw ce;
+ } else if (r.isSuccess()) {
+ xceiverClient.updateCommitInfosMap(r.getCommitInfos());
+ }
+ }, responseExecutor);
+
+ futures.add(future);
+ containerBlockData.addChunks(chunkInfo);
+ }
+
+ @VisibleForTesting
+ public void setXceiverClient(XceiverClientRatis xceiverClient) {
+ this.xceiverClient = xceiverClient;
+ }
+
+ /**
+ * Handles InterruptedExecution.
+ *
+ * @param ex
+ * @param processExecutionException is optional, if passed as TRUE, then
+ * handle ExecutionException else skip it.
+ * @throws IOException
+ */
+ private void handleInterruptedException(Exception ex,
+ boolean processExecutionException)
+ throws IOException {
+ LOG.error("Command execution was interrupted.");
+ if (processExecutionException) {
+ handleExecutionException(ex);
+ } else {
+ throw new IOException(EXCEPTION_MSG + ex.toString(), ex);
+ }
+ }
+
+ /**
+ * Handles ExecutionException by adjusting buffers.
+ * @param ex
+ * @throws IOException
+ */
+ private void handleExecutionException(Exception ex) throws IOException {
+ setIoException(ex);
+ throw getIoException();
+ }
+
+ public long getTotalAckDataLength() {
+ return commitWatcher.getTotalAckDataLength();
+ }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
index 5be2b078a7e8..5f46059b66f0 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
@@ -23,12 +23,19 @@
* Abstract class used as an interface for input streams related to Ozone
* blocks.
*/
-public abstract class BlockExtendedInputStream extends ExtendedInputStream {
+public abstract class BlockExtendedInputStream extends ExtendedInputStream
+ implements PartInputStream {
public abstract BlockID getBlockID();
- public abstract long getRemaining();
+ @Override
+ public long getRemaining() {
+ return getLength() - getPos();
+ }
+ @Override
public abstract long getLength();
+ @Override
+ public abstract long getPos();
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
index 4626f580f5e4..e89c7f9ee8c0 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
@@ -275,11 +275,6 @@ protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) {
xceiverClientFactory, () -> pipeline, verifyChecksum, token);
}
- @Override
- public synchronized long getRemaining() {
- return length - getPos();
- }
-
@Override
protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
throws IOException {
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
index de75e47781c0..5a478294bbd7 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
@@ -372,6 +372,10 @@ public void writeOnRetry(long len) throws IOException {
* @throws IOException
*/
private void handleFullBuffer() throws IOException {
+ waitForFlushAndCommit(true);
+ }
+
+ void waitForFlushAndCommit(boolean bufferFull) throws IOException {
try {
checkOpen();
waitOnFlushFutures();
@@ -381,7 +385,7 @@ private void handleFullBuffer() throws IOException {
Thread.currentThread().interrupt();
handleInterruptedException(ex, true);
}
- watchForCommit(true);
+ watchForCommit(bufferFull);
}
void releaseBuffersOnException() {
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java
new file mode 100644
index 000000000000..0650a685b634
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java
@@ -0,0 +1,57 @@
+/*
+ * 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 java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+* This interface is for writing an output stream of ByteBuffers.
+* An ByteBufferStreamOutput accepts nio ByteBuffer and sends them to some sink.
+*/
+public interface ByteBufferStreamOutput extends Closeable {
+ /**
+ * Try to write all the bytes in ByteBuf b to DataStream.
+ *
+ * @param b the data.
+ * @exception IOException if an I/O error occurs.
+ */
+ default void write(ByteBuffer b) throws IOException {
+ write(b, b.position(), b.remaining());
+ }
+
+ /**
+ * Try to write the [off:off + len) slice in ByteBuf b to DataStream.
+ *
+ * @param b the data.
+ * @param off the start offset in the data.
+ * @param len the number of bytes to write.
+ * @exception IOException if an I/O error occurs.
+ */
+ void write(ByteBuffer b, int off, int len) throws IOException;
+
+ /**
+ * Flushes this DataStream output and forces any buffered output bytes
+ * to be written out.
+ *
+ * @exception IOException if an I/O error occurs.
+ */
+ void flush() throws IOException;
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/MultipartCryptoKeyInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/MultipartInputStream.java
similarity index 63%
rename from hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/MultipartCryptoKeyInputStream.java
rename to hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/MultipartInputStream.java
index c7fc21cbb242..4bc144f3bd75 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/MultipartCryptoKeyInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/MultipartInputStream.java
@@ -16,38 +16,28 @@
* limitations under the License.
*/
-package org.apache.hadoop.ozone.client.io;
+package org.apache.hadoop.hdds.scm.storage;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
-import org.apache.hadoop.crypto.CryptoInputStream;
-import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.Seekable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.io.EOFException;
import java.io.IOException;
+import java.io.InputStream;
import java.util.Arrays;
import java.util.List;
/**
- * {@link OzoneInputStream} for accessing MPU keys in encrypted buckets.
+ * A stream for accessing multipart streams.
*/
-public class MultipartCryptoKeyInputStream extends OzoneInputStream
- implements Seekable, CanUnbuffer {
+public class MultipartInputStream extends ExtendedInputStream {
- private static final Logger LOG =
- LoggerFactory.getLogger(MultipartCryptoKeyInputStream.class);
+ private final String key;
+ private final long length;
- private static final int EOF = -1;
-
- private String key;
- private long length = 0L;
- private boolean closed = false;
-
- // List of OzoneCryptoInputStream, one for each part of the key
- private List partStreams;
+ // List of PartInputStream, one for each part of the key
+ private final List extends PartInputStream> partStreams;
// partOffsets[i] stores the index of the first data byte in
// partStream w.r.t the whole key data.
@@ -55,18 +45,19 @@ public class MultipartCryptoKeyInputStream extends OzoneInputStream
// data from indices 0 - 199, part[1] from indices 200 - 399 and so on.
// Then, partOffsets[0] = 0 (the offset of the first byte of data in
// part[0]), partOffsets[1] = 200 and so on.
- private long[] partOffsets;
+ private final long[] partOffsets;
+ private boolean closed;
// Index of the partStream corresponding to the current position of the
// MultipartCryptoKeyInputStream.
- private int partIndex = 0;
+ private int partIndex;
// Tracks the partIndex corresponding to the last seeked position so that it
// can be reset if a new position is seeked.
- private int prevPartIndex = 0;
+ private int prevPartIndex;
- public MultipartCryptoKeyInputStream(String keyName,
- List inputStreams) {
+ public MultipartInputStream(String keyName,
+ List extends PartInputStream> inputStreams) {
Preconditions.checkNotNull(inputStreams);
@@ -76,69 +67,59 @@ public MultipartCryptoKeyInputStream(String keyName,
// Calculate and update the partOffsets
this.partOffsets = new long[inputStreams.size()];
int i = 0;
- for (OzoneCryptoInputStream ozoneCryptoInputStream : inputStreams) {
- this.partOffsets[i++] = length;
- length += ozoneCryptoInputStream.getLength();
+ long streamLength = 0L;
+ for (PartInputStream partInputStream : inputStreams) {
+ this.partOffsets[i++] = streamLength;
+ streamLength += partInputStream.getLength();
}
+ this.length = streamLength;
}
- /**
- * {@inheritDoc}
- */
- @Override
- public int read() throws IOException {
- byte[] buf = new byte[1];
- if (read(buf, 0, 1) == EOF) {
- return EOF;
- }
- return Byte.toUnsignedInt(buf[0]);
- }
-
- /**
- * {@inheritDoc}
- */
@Override
- public int read(byte[] b, int off, int len) throws IOException {
+ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+ throws IOException {
+ Preconditions.checkArgument(strategy != null);
checkOpen();
- if (b == null) {
- throw new NullPointerException();
- }
- if (off < 0 || len < 0 || len > b.length - off) {
- throw new IndexOutOfBoundsException();
- }
- if (len == 0) {
- return 0;
- }
+
int totalReadLen = 0;
- while (len > 0) {
+ while (strategy.getTargetLength() > 0) {
if (partStreams.size() == 0 ||
- (partStreams.size() - 1 <= partIndex &&
- partStreams.get(partIndex).getRemaining() == 0)) {
+ partStreams.size() - 1 <= partIndex &&
+ partStreams.get(partIndex).getRemaining() == 0) {
return totalReadLen == 0 ? EOF : totalReadLen;
}
// Get the current partStream and read data from it
- OzoneCryptoInputStream current = partStreams.get(partIndex);
- int numBytesRead = current.read(b, off, len);
+ PartInputStream current = partStreams.get(partIndex);
+ int numBytesToRead = getNumBytesToRead(strategy, current);
+ int numBytesRead = strategy
+ .readFromBlock((InputStream) current, numBytesToRead);
+ checkPartBytesRead(numBytesToRead, numBytesRead, current);
totalReadLen += numBytesRead;
- off += numBytesRead;
- len -= numBytesRead;
if (current.getRemaining() <= 0 &&
- ((partIndex + 1) < partStreams.size())) {
+ partIndex + 1 < partStreams.size()) {
partIndex += 1;
}
-
}
return totalReadLen;
}
+ protected int getNumBytesToRead(ByteReaderStrategy strategy,
+ PartInputStream current) throws IOException {
+ return strategy.getTargetLength();
+ }
+
+ protected void checkPartBytesRead(int numBytesToRead, int numBytesRead,
+ PartInputStream stream) throws IOException {
+ }
+
/**
* Seeks the InputStream to the specified position. This involves 2 steps:
- * 1. Updating the partIndex to the partStream corresponding to the
- * seeked position.
- * 2. Seeking the corresponding partStream to the adjusted position.
- *
+ * 1. Updating the partIndex to the partStream corresponding to the
+ * seeked position.
+ * 2. Seeking the corresponding partStream to the adjusted position.
+ *
* For example, let’s say the part sizes are 200 bytes and part[0] stores
* data from indices 0 - 199, part[1] from indices 200 - 399 and so on.
* Let’s say we seek to position 240. In the first step, the partIndex
@@ -147,14 +128,16 @@ public int read(byte[] b, int off, int len) throws IOException {
* 240 - blockOffset[1] (= 200)).
*/
@Override
- public void seek(long pos) throws IOException {
+ public synchronized void seek(long pos) throws IOException {
+ checkOpen();
if (pos == 0 && length == 0) {
// It is possible for length and pos to be zero in which case
// seek should return instead of throwing exception
return;
}
if (pos < 0 || pos > length) {
- throw new EOFException("EOF encountered at pos: " + pos);
+ throw new EOFException(
+ "EOF encountered at pos: " + pos + " for key: " + key);
}
// 1. Update the partIndex
@@ -192,32 +175,26 @@ public void seek(long pos) throws IOException {
@Override
public synchronized long getPos() throws IOException {
- checkOpen();
- return length == 0 ? 0 : partOffsets[partIndex] +
- partStreams.get(partIndex).getPos();
+ return length == 0 ? 0 :
+ partOffsets[partIndex] + partStreams.get(partIndex).getPos();
}
@Override
- public boolean seekToNewSource(long targetPos) throws IOException {
- return false;
- }
-
- @Override
- public int available() throws IOException {
+ public synchronized int available() throws IOException {
checkOpen();
long remaining = length - getPos();
return remaining <= Integer.MAX_VALUE ? (int) remaining : Integer.MAX_VALUE;
}
@Override
- public void unbuffer() {
- for (CryptoInputStream cryptoInputStream : partStreams) {
- cryptoInputStream.unbuffer();
+ public synchronized void unbuffer() {
+ for (PartInputStream stream : partStreams) {
+ stream.unbuffer();
}
}
@Override
- public long skip(long n) throws IOException {
+ public synchronized long skip(long n) throws IOException {
if (n <= 0) {
return 0;
}
@@ -230,14 +207,15 @@ public long skip(long n) throws IOException {
@Override
public synchronized void close() throws IOException {
closed = true;
- for (OzoneCryptoInputStream partStream : partStreams) {
- partStream.close();
+ for (PartInputStream stream : partStreams) {
+ stream.close();
}
}
/**
* Verify that the input stream is open. Non blocking; this gives
* the last state of the volatile {@link #closed} field.
+ *
* @throws IOException if the connection is closed.
*/
private void checkOpen() throws IOException {
@@ -246,4 +224,23 @@ private void checkOpen() throws IOException {
": " + FSExceptionMessages.STREAM_IS_CLOSED + " Key: " + key);
}
}
+
+ public long getLength() {
+ return length;
+ }
+
+ @VisibleForTesting
+ public synchronized int getCurrentStreamIndex() {
+ return partIndex;
+ }
+
+ @VisibleForTesting
+ public long getRemainingOfIndex(int index) throws IOException {
+ return partStreams.get(index).getRemaining();
+ }
+
+ @VisibleForTesting
+ public List extends PartInputStream> getPartStreams() {
+ return partStreams;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMEchoRPCRequest.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/PartInputStream.java
similarity index 54%
rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMEchoRPCRequest.java
rename to hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/PartInputStream.java
index 1d52bf92256a..ab16dded403b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMEchoRPCRequest.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/PartInputStream.java
@@ -1,4 +1,4 @@
-/*
+/**
* 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
@@ -15,27 +15,24 @@
* 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.fs.CanUnbuffer;
+import org.apache.hadoop.fs.Seekable;
-package org.apache.hadoop.ozone.om.request;
+import java.io.IOException;
-import org.apache.hadoop.ozone.om.OzoneManager;
-import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
-import org.apache.hadoop.ozone.om.response.OMClientResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
/**
- * Handles EchoRPCRequest.
+ * A stream that can be a part of a {@link MultipartInputStream}.
*/
-public class OMEchoRPCRequest extends OMClientRequest {
- public OMEchoRPCRequest(OMRequest omRequest) {
- super(omRequest);
- }
+public interface PartInputStream
+ extends CanUnbuffer, Seekable {
+ long getLength();
- @Override
- public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
- long transactionLogIndex,
- OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
- return null;
+ default long getRemaining() throws IOException {
+ return getLength() - getPos();
}
+ void close() throws IOException;
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java
index 7c2d87d9bd0b..35a70e7bd79e 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.storage;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.scm.ContainerClientMetrics;
@@ -54,7 +55,8 @@
* This class encapsulates all state management for buffering and writing
* through to the container.
*/
-public class RatisBlockOutputStream extends BlockOutputStream {
+public class RatisBlockOutputStream extends BlockOutputStream
+ implements Syncable {
public static final Logger LOG = LoggerFactory.getLogger(
RatisBlockOutputStream.class);
@@ -126,4 +128,16 @@ void waitOnFlushFutures() throws InterruptedException, ExecutionException {
void cleanup() {
commitWatcher.cleanup();
}
+
+ @Override
+ public void hflush() throws IOException {
+ hsync();
+ }
+
+ @Override
+ public void hsync() throws IOException {
+ if (!isClosed()) {
+ waitForFlushAndCommit(false);
+ }
+ }
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBuffer.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBuffer.java
new file mode 100644
index 000000000000..d34e4dca9483
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBuffer.java
@@ -0,0 +1,59 @@
+/*
+ * 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 java.nio.ByteBuffer;
+
+/**
+ * Used for streaming write.
+ */
+public class StreamBuffer {
+ private final ByteBuffer buffer;
+
+ public StreamBuffer(ByteBuffer buffer) {
+ this.buffer = buffer;
+ }
+
+ public StreamBuffer(ByteBuffer buffer, int offset, int length) {
+ this((ByteBuffer) buffer.asReadOnlyBuffer().position(offset)
+ .limit(offset + length));
+ }
+
+ public ByteBuffer duplicate() {
+ return buffer.duplicate();
+ }
+
+ public int length() {
+ return buffer.limit() - buffer.position();
+ }
+
+ public int position() {
+ return buffer.position();
+ }
+
+
+ public void put(StreamBuffer sb) {
+ buffer.put(sb.buffer);
+ }
+
+ public static StreamBuffer allocate(int size) {
+ return new StreamBuffer(ByteBuffer.allocate(size));
+ }
+
+}
\ No newline at end of file
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamCommitWatcher.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamCommitWatcher.java
new file mode 100644
index 000000000000..8ca70de81684
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamCommitWatcher.java
@@ -0,0 +1,208 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.MemoizedSupplier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * This class executes watchForCommit on ratis pipeline and releases
+ * buffers once data successfully gets replicated.
+ */
+public class StreamCommitWatcher {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(StreamCommitWatcher.class);
+
+ private Map> commitIndexMap;
+ private final List bufferList;
+
+ // total data which has been successfully flushed and acknowledged
+ // by all servers
+ private long totalAckDataLength;
+ private final ConcurrentMap>
+ replies = new ConcurrentHashMap<>();
+
+ private final XceiverClientSpi xceiverClient;
+
+ public StreamCommitWatcher(XceiverClientSpi xceiverClient,
+ List bufferList) {
+ this.xceiverClient = xceiverClient;
+ commitIndexMap = new ConcurrentSkipListMap<>();
+ this.bufferList = bufferList;
+ totalAckDataLength = 0;
+ }
+
+ public void updateCommitInfoMap(long index, List buffers) {
+ commitIndexMap.computeIfAbsent(index, k -> new LinkedList<>())
+ .addAll(buffers);
+ }
+
+ int getCommitInfoMapSize() {
+ return commitIndexMap.size();
+ }
+
+ /**
+ * Calls watch for commit for the first index in commitIndex2flushedDataMap to
+ * the Ratis client.
+ * @return {@link XceiverClientReply} reply from raft client
+ * @throws IOException in case watchForCommit fails
+ */
+ public XceiverClientReply streamWatchOnFirstIndex() throws IOException {
+ if (!commitIndexMap.isEmpty()) {
+ // wait for the first commit index in the commitIndex2flushedDataMap
+ // to get committed to all or majority of nodes in case timeout
+ // happens.
+ long index =
+ commitIndexMap.keySet().stream().mapToLong(v -> v).min()
+ .getAsLong();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("waiting for first index {} to catch up", index);
+ }
+ return streamWatchForCommit(index);
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Calls watch for commit for the last index in commitIndex2flushedDataMap to
+ * the Ratis client.
+ * @return {@link XceiverClientReply} reply from raft client
+ * @throws IOException in case watchForCommit fails
+ */
+ public XceiverClientReply streamWatchOnLastIndex()
+ throws IOException {
+ if (!commitIndexMap.isEmpty()) {
+ // wait for the commit index in the commitIndex2flushedDataMap
+ // to get committed to all or majority of nodes in case timeout
+ // happens.
+ long index =
+ commitIndexMap.keySet().stream().mapToLong(v -> v).max()
+ .getAsLong();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("waiting for last flush Index {} to catch up", index);
+ }
+ return streamWatchForCommit(index);
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * calls watchForCommit API of the Ratis Client. This method is for streaming
+ * and no longer requires releaseBuffers
+ * @param commitIndex log index to watch for
+ * @return minimum commit index replicated to all nodes
+ * @throws IOException IOException in case watch gets timed out
+ */
+ public XceiverClientReply streamWatchForCommit(long commitIndex)
+ throws IOException {
+ final MemoizedSupplier> supplier
+ = JavaUtils.memoize(CompletableFuture::new);
+ final CompletableFuture f = replies.compute(commitIndex,
+ (key, value) -> value != null ? value : supplier.get());
+ if (!supplier.isInitialized()) {
+ // future already exists
+ return f.join();
+ }
+
+ try {
+ XceiverClientReply reply =
+ xceiverClient.watchForCommit(commitIndex);
+ f.complete(reply);
+ final CompletableFuture removed
+ = replies.remove(commitIndex);
+ Preconditions.checkState(removed == f);
+
+ adjustBuffers(reply.getLogIndex());
+ return reply;
+ } catch (InterruptedException e) {
+ // Re-interrupt the thread while catching InterruptedException
+ Thread.currentThread().interrupt();
+ throw getIOExceptionForWatchForCommit(commitIndex, e);
+ } catch (TimeoutException | ExecutionException e) {
+ throw getIOExceptionForWatchForCommit(commitIndex, e);
+ }
+ }
+
+ void releaseBuffersOnException() {
+ adjustBuffers(xceiverClient.getReplicatedMinCommitIndex());
+ }
+
+ private void adjustBuffers(long commitIndex) {
+ List keyList = commitIndexMap.keySet().stream()
+ .filter(p -> p <= commitIndex).collect(Collectors.toList());
+ if (!keyList.isEmpty()) {
+ releaseBuffers(keyList);
+ }
+ }
+
+ private long releaseBuffers(List indexes) {
+ Preconditions.checkArgument(!commitIndexMap.isEmpty());
+ for (long index : indexes) {
+ Preconditions.checkState(commitIndexMap.containsKey(index));
+ final List buffers = commitIndexMap.remove(index);
+ final long length =
+ buffers.stream().mapToLong(StreamBuffer::position).sum();
+ totalAckDataLength += length;
+ for (StreamBuffer byteBuffer : buffers) {
+ bufferList.remove(byteBuffer);
+ }
+ }
+ return totalAckDataLength;
+ }
+
+ public long getTotalAckDataLength() {
+ return totalAckDataLength;
+ }
+
+ private IOException getIOExceptionForWatchForCommit(long commitIndex,
+ Exception e) {
+ LOG.warn("watchForCommit failed for index {}", commitIndex, e);
+ IOException ioException = new IOException(
+ "Unexpected Storage Container Exception: " + e.toString(), e);
+ releaseBuffersOnException();
+ return ioException;
+ }
+
+ public void cleanup() {
+ if (commitIndexMap != null) {
+ commitIndexMap.clear();
+ }
+ commitIndexMap = null;
+ }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
index 5734d4dc4bd3..ed57ea5a1ed3 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
@@ -308,11 +308,6 @@ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
return totalRead;
}
- @Override
- public synchronized long getRemaining() {
- return blockInfo.getLength() - position;
- }
-
@Override
public synchronized long getLength() {
return blockInfo.getLength();
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
index 7a8b0d3e8eea..c1356230b2bd 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
@@ -220,7 +220,7 @@ public synchronized void unbuffer() {
}
@Override
- public synchronized long getPos() throws IOException {
+ public synchronized long getPos() {
return blockReader != null ? blockReader.getPos() : 0;
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
index 96aaa3692047..5edb5c399860 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
@@ -149,7 +149,7 @@ public synchronized void unbuffer() {
}
@Override
- public synchronized long getPos() throws IOException {
+ public synchronized long getPos() {
return position;
}
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/ECStreamTestUtil.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/ECStreamTestUtil.java
index 9994d3d4e559..0e15d834a26d 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/ECStreamTestUtil.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/ECStreamTestUtil.java
@@ -337,11 +337,6 @@ public long getLength() {
return length;
}
- @Override
- public long getRemaining() {
- return getLength() - getPos();
- }
-
@Override
public int read(byte[] b, int off, int len)
throws IOException {
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStream.java
index 5197517925fc..e354fc71793b 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStream.java
@@ -496,11 +496,6 @@ public long getLength() {
return length;
}
- @Override
- public long getRemaining() {
- return getLength() - position;
- }
-
@Override
public int read(byte[] b, int off, int len)
throws IOException {
diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml
index 6a129f9df929..d63515b3b73d 100644
--- a/hadoop-hdds/common/pom.xml
+++ b/hadoop-hdds/common/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozonehdds
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOThdds-common
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOTApache Ozone Distributed Data Store CommonApache Ozone HDDS Commonjar
@@ -212,7 +212,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.reflectionsreflections
- 0.9.11org.mockito
@@ -235,7 +234,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
com.codahale.metricsmetrics-core
- 3.0.2test
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 945ca91a4088..cb258dfa74dc 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -193,6 +193,28 @@ public final class HddsConfigKeys {
"hdds.x509.renew.grace.duration";
public static final String HDDS_X509_RENEW_GRACE_DURATION_DEFAULT = "P28D";
+ public static final String HDDS_NEW_KEY_CERT_DIR_NAME_SUFFIX = "-next";
+ public static final String HDDS_BACKUP_KEY_CERT_DIR_NAME_SUFFIX = "-previous";
+
+ public static final String HDDS_CONTAINER_REPLICATION_COMPRESSION =
+ "hdds.container.replication.compression";
+ public static final String HDDS_X509_ROOTCA_CERTIFICATE_FILE =
+ "hdds.x509.rootca.certificate.file";
+
+ public static final String HDDS_X509_ROOTCA_CERTIFICATE_FILE_DEFAULT =
+ "";
+
+ public static final String HDDS_X509_ROOTCA_PUBLIC_KEY_FILE =
+ "hdds.x509.rootca.public.key.file";
+
+ public static final String HDDS_X509_ROOTCA_PUBLIC_KEY_FILE_DEFAULT =
+ "";
+
+ public static final String HDDS_X509_ROOTCA_PRIVATE_KEY_FILE =
+ "hdds.x509.rootca.private.key.file";
+
+ public static final String HDDS_X509_ROOTCA_PRIVATE_KEY_FILE_DEFAULT =
+ "";
/**
* Do not instantiate.
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
index 1a979f1eb2a1..3b39817bab51 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
@@ -16,6 +16,7 @@
*/
package org.apache.hadoop.hdds.client;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -84,6 +85,7 @@ public void appendTo(StringBuilder sb) {
sb.append(" bcsId: ").append(blockCommitSequenceId);
}
+ @JsonIgnore
public ContainerProtos.DatanodeBlockID getDatanodeBlockIDProtobuf() {
return ContainerProtos.DatanodeBlockID.newBuilder().
setContainerID(containerBlockID.getContainerID())
@@ -91,18 +93,21 @@ public ContainerProtos.DatanodeBlockID getDatanodeBlockIDProtobuf() {
.setBlockCommitSequenceId(blockCommitSequenceId).build();
}
+ @JsonIgnore
public static BlockID getFromProtobuf(
ContainerProtos.DatanodeBlockID blockID) {
return new BlockID(blockID.getContainerID(),
blockID.getLocalID(), blockID.getBlockCommitSequenceId());
}
+ @JsonIgnore
public HddsProtos.BlockID getProtobuf() {
return HddsProtos.BlockID.newBuilder()
.setContainerBlockID(containerBlockID.getProtobuf())
.setBlockCommitSequenceId(blockCommitSequenceId).build();
}
+ @JsonIgnore
public static BlockID getFromProtobuf(HddsProtos.BlockID blockID) {
return new BlockID(
ContainerBlockID.getFromProtobuf(blockID.getContainerBlockID()),
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ContainerBlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ContainerBlockID.java
index 94a1c87d4154..57980363850f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ContainerBlockID.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ContainerBlockID.java
@@ -16,6 +16,7 @@
*/
package org.apache.hadoop.hdds.client;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import java.util.Objects;
@@ -52,11 +53,13 @@ public void appendTo(StringBuilder sb) {
.append(" locID: ").append(localID);
}
+ @JsonIgnore
public HddsProtos.ContainerBlockID getProtobuf() {
return HddsProtos.ContainerBlockID.newBuilder().
setContainerID(containerID).setLocalID(localID).build();
}
+ @JsonIgnore
public static ContainerBlockID getFromProtobuf(
HddsProtos.ContainerBlockID containerBlockID) {
return new ContainerBlockID(containerBlockID.getContainerID(),
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
index 610419527a4b..7542409679b0 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
@@ -76,6 +76,17 @@ static ReplicationConfig getDefault(ConfigurationSource config) {
return parse(null, replication, config);
}
+ static ReplicationConfig resolve(ReplicationConfig replicationConfig,
+ ReplicationConfig bucketReplicationConfig, ConfigurationSource conf) {
+ if (replicationConfig == null) {
+ replicationConfig = bucketReplicationConfig;
+ }
+ if (replicationConfig == null) {
+ replicationConfig = getDefault(conf);
+ }
+ return replicationConfig;
+ }
+
/**
* Helper method to serialize from proto.
*
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
index f35073c4728b..91415c34ecce 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
@@ -46,6 +46,7 @@
import org.apache.ratis.server.RaftServerConfigKeys;
import static org.apache.hadoop.hdds.ratis.RatisHelper.HDDS_DATANODE_RATIS_PREFIX_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CONTAINER_COPY_WORKDIR;
/**
* Configuration for ozone.
@@ -308,7 +309,9 @@ private static void addDeprecatedKeys() {
new DeprecationDelta("dfs.datanode.keytab.file",
DFSConfigKeysLegacy.DFS_DATANODE_KERBEROS_KEYTAB_FILE_KEY),
new DeprecationDelta("ozone.scm.chunk.layout",
- ScmConfigKeys.OZONE_SCM_CONTAINER_LAYOUT_KEY)
+ ScmConfigKeys.OZONE_SCM_CONTAINER_LAYOUT_KEY),
+ new DeprecationDelta("hdds.datanode.replication.work.dir",
+ OZONE_CONTAINER_COPY_WORKDIR)
});
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
index 32358ef40a44..fc3f8ae6fb82 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
@@ -284,8 +284,10 @@ public synchronized Port getPort(Port.Name name) {
return port;
}
}
- // if no separate admin/server port, return single Ratis one for compat
- if (name == Name.RATIS_ADMIN || name == Name.RATIS_SERVER) {
+ // if no separate admin/server/datastream port, return single Ratis one for
+ // compat
+ if (name == Name.RATIS_ADMIN || name == Name.RATIS_SERVER ||
+ name == Name.RATIS_DATASTREAM) {
return getPort(Name.RATIS);
}
return null;
@@ -492,6 +494,10 @@ public void setCurrentVersion(int currentVersion) {
@Override
public String toString() {
+ return uuidString + "(" + hostName + "/" + ipAddress + ")";
+ }
+
+ public String toDebugString() {
return uuid.toString() + "{" +
"ip: " +
ipAddress +
@@ -795,7 +801,8 @@ public static final class Port {
* Ports that are supported in DataNode.
*/
public enum Name {
- STANDALONE, RATIS, REST, REPLICATION, RATIS_ADMIN, RATIS_SERVER;
+ STANDALONE, RATIS, REST, REPLICATION, RATIS_ADMIN, RATIS_SERVER,
+ RATIS_DATASTREAM;
public static final Set ALL_PORTS = ImmutableSet.copyOf(
Name.values());
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
index be6076a9183b..5b7ecb0c6b97 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.ratis;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Collection;
@@ -46,17 +47,21 @@
import org.apache.ratis.client.RaftClientConfigKeys;
import org.apache.ratis.conf.Parameters;
import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
import org.apache.ratis.grpc.GrpcConfigKeys;
import org.apache.ratis.grpc.GrpcTlsConfig;
+import org.apache.ratis.netty.NettyConfigKeys;
import org.apache.ratis.proto.RaftProtos;
import org.apache.ratis.protocol.RaftGroup;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.RoutingTable;
import org.apache.ratis.retry.RetryPolicy;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -134,7 +139,9 @@ private static RaftPeer.Builder raftPeerBuilderFor(DatanodeDetails dn) {
.setId(toRaftPeerId(dn))
.setAddress(toRaftPeerAddress(dn, Port.Name.RATIS_SERVER))
.setAdminAddress(toRaftPeerAddress(dn, Port.Name.RATIS_ADMIN))
- .setClientAddress(toRaftPeerAddress(dn, Port.Name.RATIS));
+ .setClientAddress(toRaftPeerAddress(dn, Port.Name.RATIS))
+ .setDataStreamAddress(
+ toRaftPeerAddress(dn, Port.Name.RATIS_DATASTREAM));
}
private static List toRaftPeers(Pipeline pipeline) {
@@ -188,6 +195,7 @@ public static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline,
ConfigurationSource ozoneConfiguration) throws IOException {
return newRaftClient(rpcType,
toRaftPeerId(pipeline.getLeaderNode()),
+ toRaftPeer(pipeline.getFirstNode()),
newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()),
pipeline.getNodes()), retryPolicy, tlsConfig, ozoneConfiguration);
}
@@ -207,7 +215,7 @@ public static BiFunction newRaftClient(
public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
RetryPolicy retryPolicy, GrpcTlsConfig tlsConfig,
ConfigurationSource configuration) {
- return newRaftClient(rpcType, leader.getId(),
+ return newRaftClient(rpcType, leader.getId(), leader,
newRaftGroup(Collections.singletonList(leader)), retryPolicy,
tlsConfig, configuration);
}
@@ -215,14 +223,14 @@ public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
RetryPolicy retryPolicy,
ConfigurationSource ozoneConfiguration) {
- return newRaftClient(rpcType, leader.getId(),
+ return newRaftClient(rpcType, leader.getId(), leader,
newRaftGroup(Collections.singletonList(leader)), retryPolicy, null,
ozoneConfiguration);
}
@SuppressWarnings("checkstyle:ParameterNumber")
private static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
- RaftGroup group, RetryPolicy retryPolicy,
+ RaftPeer primary, RaftGroup group, RetryPolicy retryPolicy,
GrpcTlsConfig tlsConfig, ConfigurationSource ozoneConfiguration) {
if (LOG.isTraceEnabled()) {
LOG.trace("newRaftClient: {}, leader={}, group={}",
@@ -236,6 +244,7 @@ private static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
return RaftClient.newBuilder()
.setRaftGroup(group)
.setLeaderId(leader)
+ .setPrimaryDataStreamServer(primary)
.setProperties(properties)
.setParameters(setClientTlsConf(rpcType, tlsConfig))
.setRetryPolicy(retryPolicy)
@@ -265,6 +274,7 @@ private static void setClientTlsConf(Parameters parameters,
GrpcTlsConfig tlsConfig) {
if (tlsConfig != null) {
GrpcConfigKeys.Client.setTlsConf(parameters, tlsConfig);
+ NettyConfigKeys.DataStream.Client.setTlsConf(parameters, tlsConfig);
}
}
@@ -275,6 +285,8 @@ public static Parameters setServerTlsConf(
GrpcConfigKeys.Server.setTlsConf(parameters, serverConf);
GrpcConfigKeys.TLS.setConf(parameters, serverConf);
setAdminTlsConf(parameters, serverConf);
+
+ NettyConfigKeys.DataStream.Server.setTlsConf(parameters, serverConf);
}
setClientTlsConf(parameters, clientConf);
return parameters;
@@ -293,6 +305,8 @@ public static RaftProperties newRaftProperties(RpcType rpcType) {
public static RaftProperties setRpcType(RaftProperties properties,
RpcType rpcType) {
RaftConfigKeys.Rpc.setType(properties, rpcType);
+ RaftConfigKeys.DataStream.setType(properties,
+ SupportedDataStreamType.NETTY);
return properties;
}
@@ -310,7 +324,8 @@ public static void createRaftClientProperties(ConfigurationSource ozoneConf,
Map ratisClientConf =
getDatanodeRatisPrefixProps(ozoneConf);
ratisClientConf.forEach((key, val) -> {
- if (isClientConfig(key) || isGrpcClientConfig(key)) {
+ if (isClientConfig(key) || isGrpcClientConfig(key)
+ || isNettyStreamConfig(key)) {
raftProperties.set(key, val);
}
});
@@ -326,6 +341,15 @@ private static boolean isGrpcClientConfig(String key) {
!key.startsWith(GrpcConfigKeys.Admin.PREFIX) &&
!key.startsWith(GrpcConfigKeys.Server.PREFIX);
}
+
+ private static boolean isNettyStreamConfig(String key) {
+ return key.startsWith(NettyConfigKeys.DataStream.PREFIX);
+ }
+
+ private static boolean isStreamClientConfig(String key) {
+ return key.startsWith(RaftClientConfigKeys.DataStream.PREFIX);
+ }
+
/**
* Set all server properties matching with prefix
* {@link RatisHelper#HDDS_DATANODE_RATIS_PREFIX_KEY} in
@@ -340,7 +364,8 @@ public static void createRaftServerProperties(ConfigurationSource ozoneConf,
getDatanodeRatisPrefixProps(ozoneConf);
ratisServerConf.forEach((key, val) -> {
// Exclude ratis client configuration.
- if (!isClientConfig(key)) {
+ if (isNettyStreamConfig(key) || isStreamClientConfig(key) ||
+ !isClientConfig(key)) {
raftProperties.set(key, val);
}
});
@@ -403,4 +428,59 @@ private static Class extends U> getClass(String name,
throw new RuntimeException(e);
}
}
+
+ public static RoutingTable getRoutingTable(Pipeline pipeline) {
+ RaftPeerId primaryId = null;
+ List raftPeers = new ArrayList<>();
+
+ for (DatanodeDetails dn : pipeline.getNodes()) {
+ final RaftPeerId raftPeerId = RaftPeerId.valueOf(dn.getUuidString());
+ try {
+ if (dn == pipeline.getFirstNode()) {
+ primaryId = raftPeerId;
+ }
+ } catch (IOException e) {
+ LOG.error("Can not get FirstNode from the pipeline: {} with " +
+ "exception: {}", pipeline.toString(), e.getLocalizedMessage());
+ return null;
+ }
+ raftPeers.add(raftPeerId);
+ }
+
+ RoutingTable.Builder builder = RoutingTable.newBuilder();
+ RaftPeerId previousId = primaryId;
+ for (RaftPeerId peerId : raftPeers) {
+ if (peerId.equals(primaryId)) {
+ continue;
+ }
+ builder.addSuccessor(previousId, peerId);
+ previousId = peerId;
+ }
+
+ return builder.build();
+ }
+
+ public static void debug(ByteBuffer buffer, String name, Logger log) {
+ if (!log.isDebugEnabled()) {
+ return;
+ }
+ buffer = buffer.duplicate();
+ final StringBuilder builder = new StringBuilder();
+ for (int i = 1; buffer.remaining() > 0; i++) {
+ builder.append(buffer.get()).append(i % 20 == 0 ? "\n " : ", ");
+ }
+ log.debug("{}: {}\n {}", name, buffer, builder);
+ }
+
+ public static void debug(ByteBuf buf, String name, Logger log) {
+ if (!log.isDebugEnabled()) {
+ return;
+ }
+ buf = buf.duplicate();
+ final StringBuilder builder = new StringBuilder();
+ for (int i = 1; buf.readableBytes() > 0; i++) {
+ builder.append(buf.readByte()).append(i % 20 == 0 ? "\n " : ", ");
+ }
+ log.debug("{}: {}\n {}", name, buf, builder);
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java
index a1823b355d1c..c1f36fb6defe 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java
@@ -68,4 +68,7 @@ private ReconConfigKeys() {
public static final String OZONE_RECON_ADMINISTRATORS_GROUPS =
"ozone.recon.administrators.groups";
+
+ public static final String OZONE_RECON_TASK_SAFEMODE_WAIT_THRESHOLD
+ = "ozone.recon.task.safemode.wait.threshold";
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PlacementPolicy.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PlacementPolicy.java
index b240e5c3b789..248e7d715b22 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PlacementPolicy.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PlacementPolicy.java
@@ -22,12 +22,14 @@
import java.io.IOException;
import java.util.Collections;
import java.util.List;
+import java.util.Map;
+import java.util.Set;
/**
* A PlacementPolicy support choosing datanodes to build
* pipelines or containers with specified constraints.
*/
-public interface PlacementPolicy {
+public interface PlacementPolicy {
default List chooseDatanodes(
List excludedNodes,
@@ -60,9 +62,29 @@ List chooseDatanodes(List usedNodes,
* Given a list of datanode and the number of replicas required, return
* a PlacementPolicyStatus object indicating if the container meets the
* placement policy - ie is it on the correct number of racks, etc.
- * @param dns List of datanodes holding a replica of the container
+ * @param dns List of replica holding a replica of the container
* @param replicas The expected number of replicas
*/
ContainerPlacementStatus validateContainerPlacement(
- List dns, int replicas);
+ List dns, int replicas);
+
+ /**
+ * Given a set of replicas of a container which are
+ * neither over underreplicated nor overreplicated,
+ * return a set of replicas to copy to another node to fix misreplication.
+ * @param replicas: Map of replicas with value signifying if
+ * replica can be copied
+ */
+ Set replicasToCopyToFixMisreplication(
+ Map replicas);
+
+ /**
+ * Given a set of replicas of a container which are overreplicated,
+ * return a set of replicas to delete to fix overreplication.
+ * @param replicas: Set of existing replicas of the container
+ * @param expectedCountPerUniqueReplica: Replication factor of each
+ * unique replica
+ */
+ Set replicasToRemoveToFixOverreplication(
+ Set replicas, int expectedCountPerUniqueReplica);
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index 1c4e09ead9e7..c324c3e8415a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -344,6 +344,8 @@ public final class ScmConfigKeys {
public static final String OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY =
"ozone.scm.container.placement.impl";
+ public static final String OZONE_SCM_PIPELINE_PLACEMENT_IMPL_KEY =
+ "ozone.scm.pipeline.placement.impl";
public static final String OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY =
"ozone.scm.container.placement.ec.impl";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
index 026b31304013..0ffa8d68e89f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
@@ -21,8 +21,8 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
-import org.apache.hadoop.ozone.common.MonotonicClock;
+import java.time.Clock;
import java.time.ZoneOffset;
import java.util.Collection;
import java.util.HashSet;
@@ -49,7 +49,7 @@ public ExcludeList() {
datanodes = new ConcurrentHashMap<>();
containerIds = new HashSet<>();
pipelineIds = new HashSet<>();
- clock = new MonotonicClock(ZoneOffset.UTC);
+ clock = Clock.system(ZoneOffset.UTC);
}
public ExcludeList(long autoExpiryTime, java.time.Clock clock) {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index 34bd2748f688..f06066b2aa09 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -426,4 +426,11 @@ StatusAndMessages queryUpgradeFinalizationProgress(
Token> getContainerToken(ContainerID containerID) throws IOException;
long getContainerCount() throws IOException;
+
+ long getContainerCount(HddsProtos.LifeCycleState state)
+ throws IOException;
+
+ List getListOfContainers(
+ long startContainerID, int count, HddsProtos.LifeCycleState state)
+ throws IOException;
}
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 e024d79b9a7d..d0999268be75 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
@@ -56,6 +56,7 @@
import org.apache.hadoop.hdds.scm.container.common.helpers.BlockNotCommittedException;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
import org.apache.hadoop.ozone.common.Checksum;
import org.apache.hadoop.ozone.common.ChecksumData;
@@ -233,11 +234,19 @@ public static XceiverClientReply putBlockAsync(XceiverClientSpi xceiverClient,
BlockData containerBlockData, boolean eof,
Token extends TokenIdentifier> token)
throws IOException, InterruptedException, ExecutionException {
+ final ContainerCommandRequestProto request = getPutBlockRequest(
+ xceiverClient.getPipeline(), containerBlockData, eof, token);
+ return xceiverClient.sendCommandAsync(request);
+ }
+
+ public static ContainerCommandRequestProto getPutBlockRequest(
+ Pipeline pipeline, BlockData containerBlockData, boolean eof,
+ Token extends TokenIdentifier> token) throws IOException {
PutBlockRequestProto.Builder createBlockRequest =
PutBlockRequestProto.newBuilder()
.setBlockData(containerBlockData)
.setEof(eof);
- String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+ final String id = pipeline.getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder builder =
ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutBlock)
.setContainerID(containerBlockData.getBlockID().getContainerID())
@@ -246,8 +255,7 @@ public static XceiverClientReply putBlockAsync(XceiverClientSpi xceiverClient,
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
- ContainerCommandRequestProto request = builder.build();
- return xceiverClient.sendCommandAsync(request);
+ return builder.build();
}
/**
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
index 94401e5e3223..1fe22a45c9a9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;
import com.google.common.base.Preconditions;
+
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED_DEFAULT;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_TOKEN_ENABLED;
@@ -37,6 +38,12 @@
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_ALGORITHM;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_LEN;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_SECURITY_PROVIDER;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_ROOTCA_CERTIFICATE_FILE;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_ROOTCA_CERTIFICATE_FILE_DEFAULT;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_ROOTCA_PRIVATE_KEY_FILE;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_ROOTCA_PRIVATE_KEY_FILE_DEFAULT;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_ROOTCA_PUBLIC_KEY_FILE;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_ROOTCA_PUBLIC_KEY_FILE_DEFAULT;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_GRPC_TLS_ENABLED;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_GRPC_TLS_ENABLED_DEFAULT;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_GRPC_TLS_PROVIDER;
@@ -74,6 +81,7 @@
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SECURITY_SSL_TRUSTSTORE_RELOAD_INTERVAL_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
+
import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider;
import org.bouncycastle.jce.provider.BouncyCastleProvider;
import org.slf4j.Logger;
@@ -111,6 +119,9 @@ public class SecurityConfig {
private boolean grpcTlsUseTestCert;
private final long keystoreReloadInterval;
private final long truststoreReloadInterval;
+ private final String externalRootCaPublicKeyPath;
+ private final String externalRootCaPrivateKeyPath;
+ private final String externalRootCaCert;
/**
* Constructs a SecurityConfig.
@@ -175,15 +186,20 @@ public SecurityConfig(ConfigurationSource configuration) {
HDDS_X509_RENEW_GRACE_DURATION_DEFAULT);
renewalGracePeriod = Duration.parse(renewalGraceDurationString);
- if (maxCertDuration.compareTo(defaultCertDuration) < 0) {
- LOG.error("Certificate duration {} should not be greater than Maximum " +
- "Certificate duration {}", maxCertDuration, defaultCertDuration);
- throw new IllegalArgumentException("Certificate duration should not be " +
- "greater than maximum Certificate duration");
- }
+ validateCertificateValidityConfig();
+
+ this.externalRootCaCert = this.configuration.get(
+ HDDS_X509_ROOTCA_CERTIFICATE_FILE,
+ HDDS_X509_ROOTCA_CERTIFICATE_FILE_DEFAULT);
+ this.externalRootCaPublicKeyPath = this.configuration.get(
+ HDDS_X509_ROOTCA_PUBLIC_KEY_FILE,
+ HDDS_X509_ROOTCA_PUBLIC_KEY_FILE_DEFAULT);
+ this.externalRootCaPrivateKeyPath = this.configuration.get(
+ HDDS_X509_ROOTCA_PRIVATE_KEY_FILE,
+ HDDS_X509_ROOTCA_PRIVATE_KEY_FILE_DEFAULT);
this.crlName = this.configuration.get(HDDS_X509_CRL_NAME,
- HDDS_X509_CRL_NAME_DEFAULT);
+ HDDS_X509_CRL_NAME_DEFAULT);
// First Startup -- if the provider is null, check for the provider.
if (SecurityConfig.provider == null) {
@@ -207,6 +223,44 @@ public SecurityConfig(ConfigurationSource configuration) {
TimeUnit.MILLISECONDS);
}
+ /**
+ * Check for certificate validity configuration.
+ */
+ private void validateCertificateValidityConfig() {
+ if (maxCertDuration.isNegative() || maxCertDuration.isZero()) {
+ String msg = "Property " + HDDS_X509_MAX_DURATION +
+ " should not be zero or negative";
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+ if (defaultCertDuration.isNegative() || defaultCertDuration.isZero()) {
+ String msg = "Property " + HDDS_X509_DEFAULT_DURATION +
+ " should not be zero or negative";
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+ if (renewalGracePeriod.isNegative() || renewalGracePeriod.isZero()) {
+ String msg = "Property " + HDDS_X509_RENEW_GRACE_DURATION +
+ " should not be zero or negative";
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+
+ if (maxCertDuration.compareTo(defaultCertDuration) < 0) {
+ String msg = "Property " + HDDS_X509_DEFAULT_DURATION +
+ " should not be greater than Property " + HDDS_X509_MAX_DURATION;
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+ if (defaultCertDuration.compareTo(renewalGracePeriod) < 0) {
+ String msg = "Property " + HDDS_X509_RENEW_GRACE_DURATION +
+ " should not be greater than Property "
+ + HDDS_X509_DEFAULT_DURATION;
+ LOG.error(msg);
+ throw new IllegalArgumentException(msg);
+ }
+ }
+
/**
* Returns the CRL Name.
*
@@ -399,6 +453,18 @@ public SslProvider getGrpcSslProvider() {
HDDS_GRPC_TLS_PROVIDER_DEFAULT));
}
+ public String getExternalRootCaPrivateKeyPath() {
+ return externalRootCaPrivateKeyPath;
+ }
+
+ public String getExternalRootCaPublicKeyPath() {
+ return externalRootCaPublicKeyPath;
+ }
+
+ public String getExternalRootCaCert() {
+ return externalRootCaCert;
+ }
+
/**
* Return true if using test certificates with authority as localhost. This
* should be used only for unit test where certificates are generated by
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateCodec.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateCodec.java
index 03e4c53da826..6e590df04898 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateCodec.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateCodec.java
@@ -79,6 +79,11 @@ public CertificateCodec(SecurityConfig config, String component) {
this.location = securityConfig.getCertificateLocation(component);
}
+ public CertificateCodec(SecurityConfig config, Path certPath) {
+ this.securityConfig = config;
+ this.location = certPath;
+ }
+
/**
* Returns a X509 Certificate from the Certificate Holder.
*
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index 2a1c84aedbe9..70c22eebfcd5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -57,6 +57,12 @@ public final class OzoneConfigKeys {
public static final boolean DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT =
false;
+ public static final String DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT =
+ "dfs.container.ratis.datastream.random.port";
+ public static final boolean
+ DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT_DEFAULT =
+ false;
+
public static final String DFS_CONTAINER_CHUNK_WRITE_SYNC_KEY =
"dfs.container.chunk.write.sync";
public static final boolean DFS_CONTAINER_CHUNK_WRITE_SYNC_DEFAULT = false;
@@ -79,6 +85,26 @@ public final class OzoneConfigKeys {
"dfs.container.ratis.server.port";
public static final int DFS_CONTAINER_RATIS_SERVER_PORT_DEFAULT = 9856;
+ /**
+ * Ratis Port where containers listen to datastream requests.
+ */
+ public static final String DFS_CONTAINER_RATIS_DATASTREAM_ENABLED
+ = "dfs.container.ratis.datastream.enabled";
+ public static final boolean DFS_CONTAINER_RATIS_DATASTREAM_ENABLED_DEFAULT
+ = false;
+ public static final String DFS_CONTAINER_RATIS_DATASTREAM_PORT
+ = "dfs.container.ratis.datastream.port";
+ public static final int DFS_CONTAINER_RATIS_DATASTREAM_PORT_DEFAULT
+ = 9855;
+
+ /**
+ * Flag to enable ratis streaming on filesystem writes.
+ */
+ public static final String OZONE_FS_DATASTREAM_ENABLED
+ = "ozone.fs.datastream.enabled";
+ public static final boolean OZONE_FS_DATASTREAM_ENABLED_DEFAULT
+ = false;
+
/**
* When set to true, allocate a random free port for ozone container, so that
* a mini cluster is able to launch multiple containers on a node.
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java
index 1c87f2bdebad..73aff9ac830c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java
@@ -38,7 +38,8 @@ public enum DNAction implements AuditAction {
PUT_SMALL_FILE,
GET_SMALL_FILE,
CLOSE_CONTAINER,
- GET_COMMITTED_BLOCK_LENGTH;
+ GET_COMMITTED_BLOCK_LENGTH,
+ STREAM_INIT;
@Override
public String getAction() {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java
index bb4b5e3cedbd..939527a5e231 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java
@@ -140,6 +140,11 @@ public ChecksumData computeChecksum(byte[] data)
*/
public ChecksumData computeChecksum(ByteBuffer data)
throws OzoneChecksumException {
+ // If type is set to NONE, we do not need to compute the checksums. We also
+ // need to avoid unnecessary conversions.
+ if (checksumType == ChecksumType.NONE) {
+ return new ChecksumData(checksumType, bytesPerChecksum);
+ }
if (!data.isReadOnly()) {
data = data.asReadOnlyBuffer();
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java
index a13f164eec62..4d7f0f37c4eb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java
@@ -187,6 +187,7 @@ public static DNAction getAuditAction(Type cmdType) {
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;
default :
LOG.debug("Invalid command type - {}", cmdType);
return null;
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index fe11a3b72e18..0bdb5fe91aeb 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -53,6 +53,27 @@
OZONE, CONTAINER, MANAGEMENTThe ipc port number of container.
+
+
+ dfs.container.ratis.datastream.enabled
+ false
+ OZONE, CONTAINER, RATIS, DATASTREAM
+ It specifies whether to enable data stream of container.
+
+
+ dfs.container.ratis.datastream.port
+ 9855
+ OZONE, CONTAINER, RATIS, DATASTREAM
+ The datastream port number of container.
+
+
+ dfs.container.ratis.datastream.random.port
+ false
+ OZONE, CONTAINER, RATIS, DATASTREAM
+ Allocates a random free port for ozone container datastream.
+ This is used only while running unit tests.
+
+ dfs.container.ipc.random.portfalse
@@ -1558,9 +1579,9 @@
hdds.datanode.replication.work.dirDATANODE
- Temporary which is used during the container replication
- betweeen datanodes. Should have enough space to store multiple container
- (in compressed format), but doesn't require fast io access such as SSD.
+ This configuration is deprecated. Temporary sub directory under
+ each hdds.datanode.dir will be used during the container replication between
+ datanodes to save the downloaded container(in compressed format).
@@ -2087,7 +2108,8 @@
Max time for which certificate issued by SCM CA are valid.
This duration is used for self-signed root cert and scm sub-ca certs
issued by root ca. The formats accepted are based on the ISO-8601
- duration format PnDTnHnMn.nS
+ duration format PnDTnHnMn.nS
+
hdds.x509.signature.algorithm
@@ -2095,6 +2117,49 @@
OZONE, HDDS, SECURITYX509 signature certificate.
+
+ hdds.container.replication.compression
+ NO_COMPRESSION
+ OZONE, HDDS, DATANODE
+ Compression algorithm used for closed container replication.
+ Possible chooices include NO_COMPRESSION, GZIP, SNAPPY, LZ4, ZSTD
+
+
+
+ hdds.x509.rootca.certificate.file
+
+ Path to an external CA certificate. The file format is expected
+ to be pem. This certificate is used when initializing SCM to create
+ a root certificate authority. By default, a self-signed certificate is
+ generated instead. Note that this certificate is only used for Ozone's
+ internal communication, and it does not affect the certificates used for
+ HTTPS protocol at WebUIs as they can be configured separately.
+
+
+
+ hdds.x509.rootca.private.key.file
+
+ Path to an external private key. The file format is expected
+ to be pem. This private key is later used when initializing SCM to sign
+ certificates as the root certificate authority. When not specified a
+ private and public key is generated instead.
+ These keys are only used for Ozone's internal communication, and it does
+ not affect the HTTPS protocol at WebUIs as they can be configured
+ separately.
+
+
+
+ hdds.x509.rootca.public.key.file
+
+ Path to an external public key. The file format is expected
+ to be pem. This public key is later used when initializing SCM to sign
+ certificates as the root certificate authority.
+ When only the private key is specified the public key is read from the
+ external certificate. Note that this is only used for Ozone's internal
+ communication, and it does not affect the HTTPS protocol at WebUIs as
+ they can be configured separately.
+
+ ozone.scm.security.handler.count.key2
@@ -3401,6 +3466,16 @@
+
+ ozone.om.enable.ofs.shared.tmp.dir
+ false
+ OZONE, OM
+
+ Enable shared ofs tmp directory ofs://tmp. Allows a root tmp
+ directory with sticky-bit behaviour.
+
+
+
ozone.fs.listing.page.size1024
@@ -3433,4 +3508,29 @@
+
+ ozone.fs.datastream.enabled
+ false
+ OZONE, DATANODE
+
+ To enable/disable filesystem write via ratis streaming.
+
+
+
+
+ ozone.recon.scm.snapshot.task.initial.delay
+ 1m
+ OZONE, MANAGEMENT, RECON
+
+ Initial delay in MINUTES by Recon to request SCM DB Snapshot.
+
+
+
+ ozone.recon.scm.snapshot.task.interval.delay
+ 24h
+ OZONE, MANAGEMENT, RECON
+
+ Interval in MINUTES by Recon to request SCM DB Snapshot.
+
+
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/AuditLogTestUtils.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/AuditLogTestUtils.java
new file mode 100644
index 000000000000..d619c91f0de0
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/AuditLogTestUtils.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.audit;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+/**
+ * Utility class to read audit logs.
+ */
+public final class AuditLogTestUtils {
+ private static final String AUDITLOG_FILENAME = "audit.log";
+
+ private AuditLogTestUtils() {
+ }
+
+ /**
+ * Enables audit logging for the mini ozone cluster. Must be called in static
+ * block of the test class before starting the cluster.
+ */
+ public static void enableAuditLog() {
+ System.setProperty("log4j.configurationFile", "auditlog.properties");
+ }
+
+ /**
+ * Searches for the given action in the audit log file.
+ */
+ public static void verifyAuditLog(AuditAction action,
+ AuditEventStatus eventStatus) {
+ Path file = Paths.get(AUDITLOG_FILENAME);
+ try (BufferedReader br = Files.newBufferedReader(file,
+ StandardCharsets.UTF_8)) {
+ String line;
+ while ((line = br.readLine()) != null) {
+ if (line.contains(action.getAction()) &&
+ line.contains(eventStatus.getStatus())) {
+ return;
+ }
+ }
+ } catch (Exception e) {
+ throw new AssertionError(e);
+ } finally {
+ truncateAuditLogFile();
+ }
+ throw new AssertionError("Audit log file doesn't contain " +
+ "the message with params event=" + action.getAction() +
+ " result=" + eventStatus.getStatus());
+ }
+
+ private static void truncateAuditLogFile() {
+ File auditLogFile = new File(AUDITLOG_FILENAME);
+ try {
+ new FileOutputStream(auditLogFile).getChannel().truncate(0).close();
+ } catch (IOException e) {
+ System.out.println("Failed to truncate file: " + AUDITLOG_FILENAME);
+ }
+ }
+}
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 09fff2371eac..3e5c237edc67 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
@@ -470,19 +470,29 @@ public static Builder newGetCommittedBlockLengthBuilder(Pipeline pipeline,
* Returns a close container request.
* @param pipeline - pipeline
* @param containerID - ID of the container.
+ * @param token - container token
* @return ContainerCommandRequestProto.
*/
public static ContainerCommandRequestProto getCloseContainer(
- Pipeline pipeline, long containerID) throws IOException {
- return ContainerCommandRequestProto.newBuilder()
+ Pipeline pipeline, long containerID, Token> token) throws IOException {
+ Builder builder = ContainerCommandRequestProto.newBuilder()
.setCmdType(ContainerProtos.Type.CloseContainer)
.setContainerID(containerID)
.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
- .setDatanodeUuid(pipeline.getFirstNode().getUuidString())
- .build();
+ .setDatanodeUuid(pipeline.getFirstNode().getUuidString());
+
+ if (token != null) {
+ builder.setEncodedToken(token.encodeToUrlString());
+ }
+
+ return builder.build();
}
+ public static ContainerCommandRequestProto getCloseContainer(
+ Pipeline pipeline, long containerID) throws IOException {
+ return getCloseContainer(pipeline, containerID, null);
+ }
/**
* Returns a simple request without traceId.
* @param pipeline - pipeline
@@ -546,6 +556,18 @@ public static String getFixedLengthString(String string, int length) {
return String.format("%1$" + length + "s", string);
}
+ public static byte[] generateData(int length, boolean random) {
+ final byte[] data = new byte[length];
+ if (random) {
+ ThreadLocalRandom.current().nextBytes(data);
+ } else {
+ for (int i = 0; i < length; i++) {
+ data[i] = (byte) i;
+ }
+ }
+ return data;
+ }
+
/**
* Construct fake protobuf messages for various types of requests.
* This is tedious, however necessary to test. Protobuf classes are final
diff --git a/hadoop-hdds/config/pom.xml b/hadoop-hdds/config/pom.xml
index cb106986c10f..52837149f85e 100644
--- a/hadoop-hdds/config/pom.xml
+++ b/hadoop-hdds/config/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozonehdds
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOThdds-config
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOTApache Ozone Distributed Data Store Config ToolsApache Ozone HDDS Configjar
diff --git a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java
index 8cf584d75f61..3728a0b1f590 100644
--- a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java
+++ b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java
@@ -46,5 +46,6 @@ public enum ConfigTag {
DELETION,
HA,
BALANCER,
- UPGRADE
+ UPGRADE,
+ DATASTREAM
}
diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml
index 9bfd9433bbb2..21d4b755d92e 100644
--- a/hadoop-hdds/container-service/pom.xml
+++ b/hadoop-hdds/container-service/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozonehdds
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOThdds-container-service
- 1.3.0-SNAPSHOT
+ 1.4.0-SNAPSHOTApache Ozone Distributed Data Store Container ServiceApache Ozone HDDS Container Servicejar
@@ -42,6 +42,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.commonscommons-compress
+
+ com.github.luben
+ zstd-jni
+ org.apache.ozonehdds-common
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index 45b453f4a767..62302d04ce9f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -21,8 +21,6 @@
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
-import java.security.KeyPair;
-import java.security.cert.CertificateException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@@ -38,13 +36,10 @@
import org.apache.hadoop.hdds.StringUtils;
import org.apache.hadoop.hdds.cli.GenericCli;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.datanode.metadata.DatanodeCRLStore;
import org.apache.hadoop.hdds.datanode.metadata.DatanodeCRLStoreImpl;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
-import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.hdds.security.x509.certificate.client.DNCertificateClient;
@@ -71,16 +66,12 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
-import com.sun.jmx.mbeanserver.Introspector;
-import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec.getX509Certificate;
-import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
import static org.apache.hadoop.ozone.conf.OzoneServiceConfig.DEFAULT_SHUTDOWN_HOOK_PRIORITY;
import static org.apache.hadoop.ozone.common.Storage.StorageState.INITIALIZED;
import static org.apache.hadoop.util.ExitUtil.terminate;
-import org.bouncycastle.pkcs.PKCS10CertificationRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import picocli.CommandLine.Command;
@@ -99,6 +90,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
HddsDatanodeService.class);
private OzoneConfiguration conf;
+ private SecurityConfig secConf;
private DatanodeDetails datanodeDetails;
private DatanodeStateMachine datanodeStateMachine;
private List plugins;
@@ -110,6 +102,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
private final AtomicBoolean isStopped = new AtomicBoolean(false);
private final Map ratisMetricsMap =
new ConcurrentHashMap<>();
+ private List ratisReporterList = null;
private DNMXBeanImpl serviceRuntimeInfo =
new DNMXBeanImpl(HddsVersionInfo.HDDS_VERSION_INFO) { };
private ObjectName dnInfoBeanName;
@@ -154,7 +147,6 @@ public static void main(String[] args) {
try {
OzoneNetUtils.disableJvmNetworkAddressCacheIfRequired(
new OzoneConfiguration());
- Introspector.checkCompliance(DNMXBeanImpl.class);
HddsDatanodeService hddsDatanodeService =
createHddsDatanodeService(args, true);
hddsDatanodeService.run(args);
@@ -212,8 +204,8 @@ public void start(OzoneConfiguration configuration) {
public void start() {
serviceRuntimeInfo.setStartTime();
- RatisDropwizardExports.
- registerRatisMetricReporters(ratisMetricsMap, () -> isStopped.get());
+ ratisReporterList = RatisDropwizardExports
+ .registerRatisMetricReporters(ratisMetricsMap, () -> isStopped.get());
OzoneConfiguration.activate();
HddsServerUtil.initializeMetrics(conf, "HddsDatanode");
@@ -238,8 +230,10 @@ public void start() {
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
component = "dn-" + datanodeDetails.getUuidString();
- dnCertClient = new DNCertificateClient(new SecurityConfig(conf),
- datanodeDetails.getCertSerialId());
+ secConf = new SecurityConfig(conf);
+ dnCertClient = new DNCertificateClient(secConf, datanodeDetails,
+ datanodeDetails.getCertSerialId(), this::saveNewCertId,
+ this::terminateDatanode);
if (SecurityUtil.getAuthenticationMethod(conf).equals(
UserGroupInformation.AuthenticationMethod.KERBEROS)) {
@@ -274,7 +268,7 @@ public void start() {
dnCRLStore = new DatanodeCRLStoreImpl(conf);
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
- initializeCertificateClient(conf);
+ dnCertClient = initializeCertificateClient(dnCertClient);
}
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf,
dnCertClient, this::terminateDatanode, dnCRLStore);
@@ -334,15 +328,16 @@ private void startRatisForTest() throws IOException {
* Initializes secure Datanode.
* */
@VisibleForTesting
- public void initializeCertificateClient(OzoneConfiguration config)
- throws IOException {
+ public CertificateClient initializeCertificateClient(
+ CertificateClient certClient) throws IOException {
LOG.info("Initializing secure Datanode.");
- CertificateClient.InitResponse response = dnCertClient.init();
+ CertificateClient.InitResponse response = certClient.init();
if (response.equals(CertificateClient.InitResponse.REINIT)) {
LOG.info("Re-initialize certificate client.");
- dnCertClient = new DNCertificateClient(new SecurityConfig(conf));
- response = dnCertClient.init();
+ certClient = new DNCertificateClient(secConf, datanodeDetails, null,
+ this::saveNewCertId, this::terminateDatanode);
+ response = certClient.init();
}
LOG.info("Init response: {}", response);
switch (response) {
@@ -350,7 +345,14 @@ public void initializeCertificateClient(OzoneConfiguration config)
LOG.info("Initialization successful, case:{}.", response);
break;
case GETCERT:
- getSCMSignedCert(config);
+ CertificateSignRequest.Builder csrBuilder = certClient.getCSRBuilder();
+ String dnCertSerialId =
+ certClient.signAndStoreCertificate(csrBuilder.build());
+ // persist cert ID to VERSION file
+ datanodeDetails.setCertSerialId(dnCertSerialId);
+ persistDatanodeDetails(datanodeDetails);
+ // set new certificate ID
+ certClient.setCertificateId(dnCertSerialId);
LOG.info("Successfully stored SCM signed certificate, case:{}.",
response);
break;
@@ -366,51 +368,8 @@ public void initializeCertificateClient(OzoneConfiguration config)
response);
throw new RuntimeException("DN security initialization failed.");
}
- }
- /**
- * Get SCM signed certificate and store it using certificate client.
- * @param config
- * */
- private void getSCMSignedCert(OzoneConfiguration config) {
- try {
- PKCS10CertificationRequest csr = getCSR(config);
- // TODO: For SCM CA we should fetch certificate from multiple SCMs.
- SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
- HddsServerUtil.getScmSecurityClientWithMaxRetry(config);
- SCMGetCertResponseProto response = secureScmClient.
- getDataNodeCertificateChain(
- datanodeDetails.getProtoBufMessage(),
- getEncodedString(csr));
- // Persist certificates.
- if (response.hasX509CACertificate()) {
- String pemEncodedCert = response.getX509Certificate();
- dnCertClient.storeCertificate(pemEncodedCert, true);
- dnCertClient.storeCertificate(response.getX509CACertificate(), true,
- true);
-
- // Store Root CA certificate.
- if (response.hasX509RootCACertificate()) {
- dnCertClient.storeRootCACertificate(
- response.getX509RootCACertificate(), true);
- }
- String dnCertSerialId = getX509Certificate(pemEncodedCert).
- getSerialNumber().toString();
- datanodeDetails.setCertSerialId(dnCertSerialId);
- persistDatanodeDetails(datanodeDetails);
- // Rebuild dnCertClient with the new CSR result so that the default
- // certSerialId and the x509Certificate can be updated.
- dnCertClient = new DNCertificateClient(
- new SecurityConfig(config), dnCertSerialId);
-
- } else {
- throw new RuntimeException("Unable to retrieve datanode certificate " +
- "chain");
- }
- } catch (IOException | CertificateException e) {
- LOG.error("Error while storing SCM signed certificate.", e);
- throw new RuntimeException(e);
- }
+ return certClient;
}
private void registerMXBean() {
@@ -428,30 +387,6 @@ private void unregisterMXBean() {
}
}
- /**
- * Creates CSR for DN.
- * @param config
- * */
- @VisibleForTesting
- public PKCS10CertificationRequest getCSR(ConfigurationSource config)
- throws IOException {
- CertificateSignRequest.Builder builder = dnCertClient.getCSRBuilder();
- KeyPair keyPair = new KeyPair(dnCertClient.getPublicKey(),
- dnCertClient.getPrivateKey());
-
- String hostname = InetAddress.getLocalHost().getCanonicalHostName();
- String subject = UserGroupInformation.getCurrentUser()
- .getShortUserName() + "@" + hostname;
-
- builder.setCA(false)
- .setKey(keyPair)
- .setConfiguration(config)
- .setSubject(subject);
-
- LOG.info("Creating csr for DN-> subject:{}", subject);
- return builder.build();
- }
-
/**
* Returns DatanodeDetails or null in case of Error.
*
@@ -585,11 +520,13 @@ public void stop() {
unregisterMXBean();
// stop dn crl store
try {
- dnCRLStore.stop();
+ if (dnCRLStore != null) {
+ dnCRLStore.stop();
+ }
} catch (Exception ex) {
LOG.error("Datanode CRL store stop failed", ex);
}
- RatisDropwizardExports.clear(ratisMetricsMap);
+ RatisDropwizardExports.clear(ratisMetricsMap, ratisReporterList);
}
}
@@ -604,6 +541,14 @@ public void close() {
}
}
}
+
+ if (dnCertClient != null) {
+ try {
+ dnCertClient.close();
+ } catch (IOException e) {
+ LOG.warn("Certificate client could not be closed", e);
+ }
+ }
}
@VisibleForTesting
@@ -624,4 +569,18 @@ public void setCertificateClient(CertificateClient client) {
public void printError(Throwable error) {
LOG.error("Exception in HddsDatanodeService.", error);
}
+
+ public void saveNewCertId(String newCertId) {
+ // save new certificate Id to VERSION file
+ datanodeDetails.setCertSerialId(newCertId);
+ try {
+ persistDatanodeDetails(datanodeDetails);
+ } catch (IOException ex) {
+ // New cert ID cannot be persisted into VERSION file.
+ String msg = "Failed to persist new cert ID " + newCertId +
+ "to VERSION file. Terminating datanode...";
+ LOG.error(msg, ex);
+ terminateDatanode();
+ }
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index e0480735e5b3..81a6935098af 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -33,6 +33,8 @@
import java.nio.file.Paths;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -278,4 +280,22 @@ public static long getContainerID(File containerBaseDir) {
return Long.parseLong(containerBaseDir.getName());
}
+ public static String getContainerTarGzName(long containerId) {
+ return "container-" + containerId + ".tar.gz";
+ }
+
+ public static long retrieveContainerIdFromTarGzName(String tarGzName)
+ throws IOException {
+ assert tarGzName != null;
+ Pattern pattern = Pattern.compile("container-(\\d+).tar.gz");
+ // Now create matcher object.
+ Matcher m = pattern.matcher(tarGzName);
+
+ if (m.find()) {
+ return Long.parseLong(m.group(1));
+ } else {
+ throw new IOException("Illegal container tar gz file " +
+ tarGzName);
+ }
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index d3caa319fafb..55427b87b096 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -52,7 +52,7 @@
import org.slf4j.LoggerFactory;
import org.yaml.snakeyaml.Yaml;
import org.yaml.snakeyaml.constructor.AbstractConstruct;
-import org.yaml.snakeyaml.constructor.Constructor;
+import org.yaml.snakeyaml.constructor.SafeConstructor;
import org.yaml.snakeyaml.introspector.BeanAccess;
import org.yaml.snakeyaml.introspector.Property;
import org.yaml.snakeyaml.introspector.PropertyUtils;
@@ -160,7 +160,7 @@ public static ContainerData readContainer(InputStream input)
KeyValueContainerData.getYamlFields());
representer.setPropertyUtils(propertyUtils);
- Constructor containerDataConstructor = new ContainerDataConstructor();
+ SafeConstructor containerDataConstructor = new ContainerDataConstructor();
Yaml yaml = new Yaml(containerDataConstructor, representer);
yaml.setBeanAccess(BeanAccess.FIELD);
@@ -200,7 +200,7 @@ public static Yaml getYamlForContainerType(ContainerType containerType,
KeyValueContainerData.class,
KEYVALUE_YAML_TAG);
- Constructor keyValueDataConstructor = new ContainerDataConstructor();
+ SafeConstructor keyValueDataConstructor = new ContainerDataConstructor();
return new Yaml(keyValueDataConstructor, representer);
}
@@ -255,7 +255,7 @@ protected NodeTuple representJavaBeanProperty(
/**
* Constructor class for KeyValueData, which will be used by Yaml.
*/
- private static class ContainerDataConstructor extends Constructor {
+ private static class ContainerDataConstructor extends SafeConstructor {
ContainerDataConstructor() {
//Adding our own specific constructors for tags.
// When a new Container type is added, we need to add yamlConstructor
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index 3e81333d34a3..0c5ae6aeebc7 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -24,7 +24,6 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.ozone.common.MonotonicClock;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.slf4j.Logger;
@@ -63,7 +62,7 @@ public class ContainerSet {
private long recoveringTimeout;
public ContainerSet(long recoveringTimeout) {
- this.clock = new MonotonicClock(ZoneOffset.UTC);
+ this.clock = Clock.system(ZoneOffset.UTC);
this.recoveringTimeout = recoveringTimeout;
}
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 5059a6429135..fc711b5a3717 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,6 +59,7 @@
import org.apache.hadoop.ozone.container.ozoneimpl.OnDemandContainerScanner;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
+import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.thirdparty.com.google.protobuf.ProtocolMessageEnum;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -200,7 +201,8 @@ private ContainerCommandResponseProto dispatchRequest(
boolean isWriteStage =
(cmdType == Type.WriteChunk && dispatcherContext != null
&& dispatcherContext.getStage()
- == DispatcherContext.WriteChunkStage.WRITE_DATA);
+ == DispatcherContext.WriteChunkStage.WRITE_DATA)
+ || (cmdType == Type.StreamInit);
boolean isWriteCommitStage =
(cmdType == Type.WriteChunk && dispatcherContext != null
&& dispatcherContext.getStage()
@@ -699,4 +701,21 @@ private boolean isAllowed(String action) {
default: return false;
}
}
+
+ @Override
+ public StateMachine.DataChannel getStreamDataChannel(
+ ContainerCommandRequestProto msg)
+ throws StorageContainerException {
+ long containerID = msg.getContainerID();
+ Container container = getContainer(containerID);
+ if (container != null) {
+ Handler handler = getHandler(getContainerType(container));
+ return handler.getStreamDataChannel(container, msg);
+ } else {
+ throw new StorageContainerException(
+ "ContainerID " + containerID + " does not exist",
+ ContainerProtos.Result.CONTAINER_NOT_FOUND);
+ }
+ }
+
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
index a2e397d54615..d02bae0a35ad 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
@@ -25,6 +25,7 @@
.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
+import org.apache.ratis.statemachine.StateMachine;
import java.util.Map;
@@ -84,4 +85,13 @@ void validateContainerCommand(
* @param clusterId
*/
void setClusterId(String clusterId);
+
+ /**
+ * When uploading using stream, get StreamDataChannel.
+ */
+ default StateMachine.DataChannel getStreamDataChannel(
+ ContainerCommandRequestProto msg) throws StorageContainerException {
+ throw new UnsupportedOperationException(
+ "getStreamDataChannel not supported.");
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
index 8308c23866b8..a7c7f5ad20fd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.nio.file.Path;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -39,7 +40,7 @@ public interface ContainerPacker {
* file but returned).
*/
byte[] unpackContainerData(Container container,
- InputStream inputStream)
+ InputStream inputStream, Path tmpDir, Path destContainerDir)
throws IOException;
/**
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 0fd7e14699e8..62418f2bbe33 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
@@ -37,6 +37,7 @@
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
+import org.apache.ratis.statemachine.StateMachine;
/**
* Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
@@ -81,6 +82,10 @@ public static Handler getHandlerForContainerType(
}
}
+ public abstract StateMachine.DataChannel getStreamDataChannel(
+ Container container, ContainerCommandRequestProto msg)
+ throws StorageContainerException;
+
/**
* Returns the Id of this datanode.
*
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index 4d72bb317f3e..4b8f0be1e3e4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -18,6 +18,8 @@
import java.io.Closeable;
import java.io.IOException;
+import java.time.Clock;
+import java.time.ZoneId;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -139,6 +141,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
this.conf = conf;
this.datanodeDetails = datanodeDetails;
+ Clock clock = Clock.system(ZoneId.systemDefault());
// Expected to be initialized already.
layoutStorage = new DatanodeLayoutStorage(conf,
datanodeDetails.getUuidString());
@@ -169,10 +172,10 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
nextHB = new AtomicLong(Time.monotonicNow());
ContainerReplicator replicator =
- new DownloadAndImportReplicator(container.getContainerSet(),
+ new DownloadAndImportReplicator(conf, container.getContainerSet(),
container.getController(),
new SimpleContainerDownloader(conf, dnCertClient),
- new TarContainerPacker());
+ new TarContainerPacker(), container.getVolumeSet());
replicatorMetrics = new MeasuredReplicator(replicator);
@@ -180,7 +183,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
conf.getObject(ReplicationConfig.class);
supervisor =
new ReplicationSupervisor(container.getContainerSet(), context,
- replicatorMetrics, replicationConfig);
+ replicatorMetrics, replicationConfig, clock);
replicationSupervisorMetrics =
ReplicationSupervisorMetrics.create(supervisor);
@@ -188,12 +191,12 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
ecReconstructionMetrics = ECReconstructionMetrics.create();
ECReconstructionCoordinator ecReconstructionCoordinator =
- new ECReconstructionCoordinator(conf, certClient,
+ new ECReconstructionCoordinator(conf, certClient, context,
ecReconstructionMetrics);
ecReconstructionSupervisor =
new ECReconstructionSupervisor(container.getContainerSet(), context,
replicationConfig.getReplicationMaxStreams(),
- ecReconstructionCoordinator);
+ ecReconstructionCoordinator, clock);
// When we add new handlers just adding a new handler here should do the
@@ -207,7 +210,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
.addHandler(new ReconstructECContainersCommandHandler(conf,
ecReconstructionSupervisor))
.addHandler(new DeleteContainerCommandHandler(
- dnConf.getContainerDeleteThreads()))
+ dnConf.getContainerDeleteThreads(), clock))
.addHandler(new ClosePipelineCommandHandler())
.addHandler(new CreatePipelineCommandHandler(conf))
.addHandler(new SetNodeOperationalStateCommandHandler(conf))
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
index eb44b3b8b630..c4d3428a8018 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
@@ -24,7 +24,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import java.util.Optional;
+import java.util.OptionalLong;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@@ -142,7 +142,7 @@ public class StateContext {
*
* For non-HA mode, term of SCMCommand will be 0.
*/
- private Optional termOfLeaderSCM = Optional.empty();
+ private OptionalLong termOfLeaderSCM = OptionalLong.empty();
/**
* Starting with a 2 sec heartbeat frequency which will be updated to the
@@ -720,10 +720,9 @@ private void initTermOfLeaderSCM() {
// if commandQueue is not empty, init termOfLeaderSCM
// with the largest term found in commandQueue
- commandQueue.stream()
+ termOfLeaderSCM = commandQueue.stream()
.mapToLong(SCMCommand::getTerm)
- .max()
- .ifPresent(term -> termOfLeaderSCM = Optional.of(term));
+ .max();
}
/**
@@ -731,12 +730,27 @@ private void initTermOfLeaderSCM() {
* Always record the latest term that has seen.
*/
private void updateTermOfLeaderSCM(SCMCommand> command) {
+ updateTermOfLeaderSCM(command.getTerm());
+ }
+
+ public void updateTermOfLeaderSCM(final long newTerm) {
if (!termOfLeaderSCM.isPresent()) {
- LOG.error("should init termOfLeaderSCM before update it.");
return;
}
- termOfLeaderSCM = Optional.of(
- Long.max(termOfLeaderSCM.get(), command.getTerm()));
+
+ final long currentTerm = termOfLeaderSCM.getAsLong();
+ if (currentTerm < newTerm) {
+ setTermOfLeaderSCM(newTerm);
+ }
+ }
+
+ @VisibleForTesting
+ public void setTermOfLeaderSCM(long term) {
+ termOfLeaderSCM = OptionalLong.of(term);
+ }
+
+ public OptionalLong getTermOfLeaderSCM() {
+ return termOfLeaderSCM;
}
/**
@@ -759,13 +773,14 @@ public SCMCommand getNextCommand() {
}
updateTermOfLeaderSCM(command);
- if (command.getTerm() == termOfLeaderSCM.get()) {
+ final long currentTerm = termOfLeaderSCM.getAsLong();
+ if (command.getTerm() == currentTerm) {
return command;
}
LOG.warn("Detect and drop a SCMCommand {} from stale leader SCM," +
" stale term {}, latest term {}.",
- command, command.getTerm(), termOfLeaderSCM.get());
+ command, command.getTerm(), currentTerm);
}
} finally {
lock.unlock();
@@ -780,6 +795,7 @@ public SCMCommand getNextCommand() {
public void addCommand(SCMCommand command) {
lock.lock();
try {
+ updateTermOfLeaderSCM(command);
commandQueue.add(command);
} finally {
lock.unlock();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index c5912a2ffdc3..3e63af05f24f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -333,7 +333,8 @@ private void markBlocksForDeletionSchemaV3(
DeletionMarker schemaV3Marker = (table, batch, tid, txn) -> {
Table delTxTable =
(Table) table;
- delTxTable.putWithBatch(batch, containerData.deleteTxnKey(tid), txn);
+ delTxTable.putWithBatch(batch, containerData.getDeleteTxnKey(tid),
+ txn);
};
markBlocksForDeletionTransaction(containerData, delTX, newDeletionBlocks,
@@ -403,10 +404,10 @@ private void markBlocksForDeletionSchemaV1(
try (BatchOperation batch = containerDB.getStore().getBatchHandler()
.initBatchOperation()) {
for (Long blkLong : delTX.getLocalIDList()) {
- String blk = containerData.blockKey(blkLong);
+ String blk = containerData.getBlockKey(blkLong);
BlockData blkInfo = blockDataTable.get(blk);
if (blkInfo != null) {
- String deletingKey = containerData.deletingBlockKey(blkLong);
+ String deletingKey = containerData.getDeletingBlockKey(blkLong);
if (blockDataTable.get(deletingKey) != null
|| deletedBlocksTable.get(blk) != null) {
if (LOG.isDebugEnabled()) {
@@ -463,15 +464,15 @@ private void updateMetaData(KeyValueContainerData containerData,
if (delTX.getTxID() > containerData.getDeleteTransactionId()) {
// Update in DB pending delete key count and delete transaction ID.
metadataTable
- .putWithBatch(batchOperation, containerData.latestDeleteTxnKey(),
- delTX.getTxID());
+ .putWithBatch(batchOperation,
+ containerData.getLatestDeleteTxnKey(), delTX.getTxID());
}
long pendingDeleteBlocks =
containerData.getNumPendingDeletionBlocks() + newDeletionBlocks;
metadataTable
.putWithBatch(batchOperation,
- containerData.pendingDeleteBlockCountKey(),
+ containerData.getPendingDeleteBlockCountKey(),
pendingDeleteBlocks);
// update pending deletion blocks count and delete transaction ID in
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
index 58ad2d18e4ff..767c00c04497 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
@@ -32,6 +32,8 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.time.Clock;
+import java.util.OptionalLong;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadPoolExecutor;
@@ -48,15 +50,22 @@ public class DeleteContainerCommandHandler implements CommandHandler {
LoggerFactory.getLogger(DeleteContainerCommandHandler.class);
private final AtomicInteger invocationCount = new AtomicInteger(0);
+ private final AtomicInteger timeoutCount = new AtomicInteger(0);
private final AtomicLong totalTime = new AtomicLong(0);
private final ExecutorService executor;
+ private final Clock clock;
- public DeleteContainerCommandHandler(int threadPoolSize) {
- this.executor = Executors.newFixedThreadPool(
+ public DeleteContainerCommandHandler(int threadPoolSize, Clock clock) {
+ this(clock, Executors.newFixedThreadPool(
threadPoolSize, new ThreadFactoryBuilder()
- .setNameFormat("DeleteContainerThread-%d").build());
+ .setNameFormat("DeleteContainerThread-%d").build()));
}
+ protected DeleteContainerCommandHandler(Clock clock,
+ ExecutorService executor) {
+ this.executor = executor;
+ this.clock = clock;
+ }
@Override
public void handle(final SCMCommand command,
final OzoneContainer ozoneContainer,
@@ -65,18 +74,44 @@ public void handle(final SCMCommand command,
final DeleteContainerCommand deleteContainerCommand =
(DeleteContainerCommand) command;
final ContainerController controller = ozoneContainer.getController();
- executor.execute(() -> {
- final long startTime = Time.monotonicNow();
- invocationCount.incrementAndGet();
- try {
- controller.deleteContainer(deleteContainerCommand.getContainerID(),
- deleteContainerCommand.isForce());
- } catch (IOException e) {
- LOG.error("Exception occurred while deleting the container.", e);
- } finally {
- totalTime.getAndAdd(Time.monotonicNow() - startTime);
+ executor.execute(() ->
+ handleInternal(command, context, deleteContainerCommand, controller));
+ }
+
+ private void handleInternal(SCMCommand command, StateContext context,
+ DeleteContainerCommand deleteContainerCommand,
+ ContainerController controller) {
+ final long startTime = Time.monotonicNow();
+ invocationCount.incrementAndGet();
+ try {
+ if (command.hasExpired(clock.millis())) {
+ LOG.info("Not processing the delete container command for " +
+ "container {} as the current time {}ms is after the command " +
+ "deadline {}ms", deleteContainerCommand.getContainerID(),
+ clock.millis(), command.getDeadline());
+ timeoutCount.incrementAndGet();
+ return;
+ }
+
+ if (context != null) {
+ final OptionalLong currentTerm = context.getTermOfLeaderSCM();
+ final long cmdTerm = command.getTerm();
+ if (currentTerm.isPresent() && cmdTerm < currentTerm.getAsLong()) {
+ LOG.info("Ignoring delete container command for container {} since " +
+ "SCM leader has new term ({} < {})",
+ deleteContainerCommand.getContainerID(),
+ cmdTerm, currentTerm.getAsLong());
+ return;
+ }
}
- });
+
+ controller.deleteContainer(deleteContainerCommand.getContainerID(),
+ deleteContainerCommand.isForce());
+ } catch (IOException e) {
+ LOG.error("Exception occurred while deleting the container.", e);
+ } finally {
+ totalTime.getAndAdd(Time.monotonicNow() - startTime);
+ }
}
@Override
@@ -94,6 +129,10 @@ public int getInvocationCount() {
return this.invocationCount.get();
}
+ public int getTimeoutCount() {
+ return this.timeoutCount.get();
+ }
+
@Override
public long getAverageRunTime() {
final int invocations = invocationCount.get();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java
index 57d4d16f8ab1..c6abfc27c33a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java
@@ -47,11 +47,7 @@ public void handle(SCMCommand command, OzoneContainer container,
ReconstructECContainersCommand ecContainersCommand =
(ReconstructECContainersCommand) command;
ECReconstructionCommandInfo reconstructionCommandInfo =
- new ECReconstructionCommandInfo(ecContainersCommand.getContainerID(),
- ecContainersCommand.getEcReplicationConfig(),
- ecContainersCommand.getMissingContainerIndexes(),
- ecContainersCommand.getSources(),
- ecContainersCommand.getTargetDatanodes());
+ new ECReconstructionCommandInfo(ecContainersCommand);
this.supervisor.addTask(reconstructionCommandInfo);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java
index 44c783846ad1..df589e287d87 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java
@@ -71,7 +71,8 @@ public void handle(SCMCommand command, OzoneContainer container,
"Replication command is received for container %s "
+ "without source datanodes.", containerID);
- supervisor.addTask(new ReplicationTask(containerID, sourceDatanodes));
+ ReplicationTask task = new ReplicationTask(replicateCommand);
+ supervisor.addTask(task);
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
index ccb0e8b7d7d9..3a1bd8ffb31d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
@@ -307,6 +307,9 @@ private void processResponse(SCMHeartbeatResponseProto response,
Preconditions.checkState(response.getDatanodeUUID()
.equalsIgnoreCase(datanodeDetails.getUuid()),
"Unexpected datanode ID in the response.");
+ if (response.hasTerm()) {
+ context.updateTermOfLeaderSCM(response.getTerm());
+ }
// Verify the response is indeed for this datanode.
for (SCMCommandProto commandResponseProto : response.getCommandsList()) {
switch (commandResponseProto.getCommandType()) {
@@ -426,6 +429,7 @@ private void processResponse(SCMHeartbeatResponseProto response,
* Common processing for SCM commands.
* - set term
* - set encoded token
+ * - any deadline which is relevant to the command
* - add to context's queue
*/
private void processCommonCommand(
@@ -436,6 +440,9 @@ private void processCommonCommand(
if (response.hasEncodedToken()) {
cmd.setEncodedToken(response.getEncodedToken());
}
+ if (response.hasDeadlineMsSinceEpoch()) {
+ cmd.setDeadline(response.getDeadlineMsSinceEpoch());
+ }
context.addCommand(cmd);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
index c1fc95079221..528f4b8bd7e3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
@@ -140,10 +140,10 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails,
new GrpcXceiverService(dispatcher), new GrpcServerInterceptor()));
SecurityConfig secConf = new SecurityConfig(conf);
- if (secConf.isGrpcTlsEnabled()) {
+ if (secConf.isSecurityEnabled() && secConf.isGrpcTlsEnabled()) {
try {
SslContextBuilder sslClientContextBuilder = SslContextBuilder.forServer(
- caClient.getPrivateKey(), caClient.getCertificate());
+ caClient.getServerKeyStoresFactory().getKeyManagers()[0]);
SslContextBuilder sslContextBuilder = GrpcSslContexts.configure(
sslClientContextBuilder, secConf.getGrpcSslProvider());
nettyServerBuilder.sslContext(sslContextBuilder.build());
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 02c0a8d2b152..f6f5a99927ca 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
@@ -29,6 +29,7 @@
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -61,6 +62,7 @@
import org.apache.hadoop.ozone.common.utils.BufferUtils;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.util.Time;
@@ -80,6 +82,7 @@
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.raftlog.RaftLog;
import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.statemachine.StateMachineStorage;
import org.apache.ratis.statemachine.TransactionContext;
import org.apache.ratis.statemachine.impl.BaseStateMachine;
@@ -90,6 +93,7 @@
import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat;
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;
@@ -423,6 +427,20 @@ private ContainerCommandResponseProto runCommand(
return dispatchCommand(requestProto, context);
}
+ private CompletableFuture runCommandAsync(
+ ContainerCommandRequestProto requestProto, LogEntryProto entry) {
+ return CompletableFuture.supplyAsync(() -> {
+ final DispatcherContext context = new DispatcherContext.Builder()
+ .setTerm(entry.getTerm())
+ .setLogIndex(entry.getIndex())
+ .setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA)
+ .setContainer2BCSIDMap(container2BCSIDMap)
+ .build();
+
+ return runCommand(requestProto, context);
+ }, executor);
+ }
+
private CompletableFuture handleWriteChunk(
ContainerCommandRequestProto requestProto, long entryIndex, long term,
long startTime) {
@@ -510,6 +528,64 @@ private CompletableFuture handleWriteChunk(
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());
+ }
+ runCommand(requestProto, context); // stream init
+ return dispatcher.getStreamDataChannel(requestProto);
+ }
+
+ @Override
+ public CompletableFuture stream(RaftClientRequest request) {
+ return CompletableFuture.supplyAsync(() -> {
+ try {
+ ContainerCommandRequestProto requestProto =
+ message2ContainerCommandRequestProto(request.getMessage());
+ DispatcherContext context =
+ new DispatcherContext.Builder()
+ .setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
+ .setContainer2BCSIDMap(container2BCSIDMap)
+ .build();
+ DataChannel channel = getStreamDataChannel(requestProto, context);
+ 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);
+ }
+ }, executor);
+ }
+
+ @Override
+ public CompletableFuture> link(DataStream stream, LogEntryProto entry) {
+ if (stream == null) {
+ return JavaUtils.completeExceptionally(new IllegalStateException(
+ "DataStream is null"));
+ }
+ final DataChannel dataChannel = stream.getDataChannel();
+ if (dataChannel.isOpen()) {
+ return JavaUtils.completeExceptionally(new IllegalStateException(
+ "DataStream: " + stream + " is not closed properly"));
+ }
+
+ final ContainerCommandRequestProto request;
+ if (dataChannel instanceof KeyValueStreamDataChannel) {
+ request = ((KeyValueStreamDataChannel) dataChannel).getPutBlockRequest();
+ } else {
+ return JavaUtils.completeExceptionally(new IllegalStateException(
+ "Unexpected DataChannel " + dataChannel.getClass()));
+ }
+ return runCommandAsync(request, entry).whenComplete(
+ (res, e) -> LOG.debug("link {}, entry: {}, request: {}",
+ res.getResult(), entry, request));
+ }
+
private ExecutorService getChunkExecutor(WriteChunkRequestProto req) {
int i = (int)(req.getBlockID().getLocalID() % chunkExecutors.size());
return chunkExecutors.get(i);
@@ -803,7 +879,8 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
builder.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA);
}
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile
- || cmdType == Type.PutBlock || cmdType == Type.CreateContainer) {
+ || cmdType == Type.PutBlock || cmdType == Type.CreateContainer
+ || cmdType == Type.StreamInit) {
builder.setContainer2BCSIDMap(container2BCSIDMap);
}
CompletableFuture applyTransactionFuture =
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java
new file mode 100644
index 000000000000..780f8743988a
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.transport.server.ratis;
+
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+
+class LocalStream implements StateMachine.DataStream {
+ private final StateMachine.DataChannel dataChannel;
+ private final Executor executor;
+
+ LocalStream(StateMachine.DataChannel dataChannel, Executor executor) {
+ this.dataChannel = dataChannel;
+ this.executor = executor;
+ }
+
+ @Override
+ public StateMachine.DataChannel getDataChannel() {
+ return dataChannel;
+ }
+
+ @Override
+ public CompletableFuture> cleanUp() {
+ return CompletableFuture.supplyAsync(() -> {
+ try {
+ dataChannel.close();
+ return true;
+ } catch (IOException e) {
+ throw new CompletionException("Failed to close data channel", e);
+ }
+ });
+ }
+
+ @Override
+ public Executor getExecutor() {
+ return executor;
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
index c8d715cc60d2..89e5047b85b6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -98,6 +98,7 @@
import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.server.DataStreamServerRpc;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.RaftServerConfigKeys;
import org.apache.ratis.server.RaftServerRpc;
@@ -129,6 +130,7 @@ private static long nextCallId() {
private int serverPort;
private int adminPort;
private int clientPort;
+ private int dataStreamPort;
private final RaftServer server;
private final List chunkExecutors;
private final ContainerDispatcher dispatcher;
@@ -148,6 +150,7 @@ private static long nextCallId() {
// Timeout used while calling submitRequest directly.
private long requestTimeout;
private boolean shouldDeleteRatisLogDirectory;
+ private boolean streamEnable;
private XceiverServerRatis(DatanodeDetails dd,
ContainerDispatcher dispatcher, ContainerController containerController,
@@ -157,6 +160,9 @@ private XceiverServerRatis(DatanodeDetails dd,
Objects.requireNonNull(dd, "id == null");
datanodeDetails = dd;
assignPorts();
+ this.streamEnable = conf.getBoolean(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_ENABLED,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_ENABLED_DEFAULT);
RaftProperties serverProperties = newRaftProperties();
this.context = context;
this.dispatcher = dispatcher;
@@ -213,6 +219,32 @@ private ContainerStateMachine getStateMachine(RaftGroupId gid) {
chunkExecutors, this, conf);
}
+ private void setUpRatisStream(RaftProperties properties) {
+ // set the datastream config
+ if (conf.getBoolean(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT,
+ OzoneConfigKeys.
+ DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT_DEFAULT)) {
+ dataStreamPort = 0;
+ } else {
+ dataStreamPort = conf.getInt(
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_PORT,
+ OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_PORT_DEFAULT);
+ }
+ NettyConfigKeys.DataStream.setPort(properties, dataStreamPort);
+ int dataStreamAsyncRequestThreadPoolSize =
+ conf.getObject(DatanodeRatisServerConfig.class)
+ .getStreamRequestThreads();
+ RaftServerConfigKeys.DataStream.setAsyncRequestThreadPoolSize(properties,
+ dataStreamAsyncRequestThreadPoolSize);
+ int dataStreamClientPoolSize =
+ conf.getObject(DatanodeRatisServerConfig.class)
+ .getClientPoolSize();
+ RaftServerConfigKeys.DataStream.setClientPoolSize(properties,
+ dataStreamClientPoolSize);
+ }
+
+ @SuppressWarnings("checkstyle:methodlength")
private RaftProperties newRaftProperties() {
final RaftProperties properties = new RaftProperties();
@@ -231,6 +263,10 @@ private RaftProperties newRaftProperties() {
// set the configs enable and set the stateMachineData sync timeout
RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true);
+ if (streamEnable) {
+ setUpRatisStream(properties);
+ }
+
timeUnit = OzoneConfigKeys.
DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT.getUnit();
duration = conf.getTimeDuration(
@@ -491,7 +527,12 @@ public void start() throws IOException {
Port.Name.RATIS_ADMIN);
serverPort = getRealPort(serverRpc.getInetSocketAddress(),
Port.Name.RATIS_SERVER);
-
+ if (streamEnable) {
+ DataStreamServerRpc dataStreamServerRpc =
+ server.getDataStreamServerRpc();
+ dataStreamPort = getRealPort(dataStreamServerRpc.getInetSocketAddress(),
+ Port.Name.RATIS_DATASTREAM);
+ }
isStarted = true;
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java
index 561708b852cd..a5cddc175eaf 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java
@@ -368,6 +368,10 @@ public StorageType getStorageType() {
}
}
+ public String getVolumeRootDir() {
+ return volumeInfo != null ? volumeInfo.getRootDir() : null;
+ }
+
public long getCapacity() {
return volumeInfo != null ? volumeInfo.getCapacity() : 0;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java
index c95f9646f859..c053a9ae9248 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java
@@ -23,60 +23,79 @@
import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand.DatanodeDetailsAndReplicaIndex;
import java.util.Arrays;
-import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.stream.IntStream;
+
+import static java.util.Collections.unmodifiableSortedMap;
+import static java.util.stream.Collectors.toMap;
/**
* This class is to keep the required EC reconstruction info.
*/
public class ECReconstructionCommandInfo {
- private long containerID;
- private ECReplicationConfig ecReplicationConfig;
- private byte[] missingContainerIndexes;
- private List
- sources;
- private List targetDatanodes;
+ private final SortedMap sourceNodeMap;
+ private final SortedMap targetNodeMap;
+ private final long containerID;
+ private final ECReplicationConfig ecReplicationConfig;
+ private final byte[] missingContainerIndexes;
+ private final long deadlineMsSinceEpoch;
+ private final long term;
- public ECReconstructionCommandInfo(long containerID,
- ECReplicationConfig ecReplicationConfig, byte[] missingContainerIndexes,
- List sources,
- List targetDatanodes) {
- this.containerID = containerID;
- this.ecReplicationConfig = ecReplicationConfig;
+ public ECReconstructionCommandInfo(ReconstructECContainersCommand cmd) {
+ this.containerID = cmd.getContainerID();
+ this.ecReplicationConfig = cmd.getEcReplicationConfig();
this.missingContainerIndexes =
- Arrays.copyOf(missingContainerIndexes, missingContainerIndexes.length);
- this.sources = sources;
- this.targetDatanodes = targetDatanodes;
+ Arrays.copyOf(cmd.getMissingContainerIndexes(),
+ cmd.getMissingContainerIndexes().length);
+ this.deadlineMsSinceEpoch = cmd.getDeadline();
+ this.term = cmd.getTerm();
+
+ sourceNodeMap = cmd.getSources().stream()
+ .collect(toMap(
+ DatanodeDetailsAndReplicaIndex::getReplicaIndex,
+ DatanodeDetailsAndReplicaIndex::getDnDetails,
+ (v1, v2) -> v1, TreeMap::new));
+ targetNodeMap = IntStream.range(0, cmd.getTargetDatanodes().size())
+ .boxed()
+ .collect(toMap(
+ i -> (int) missingContainerIndexes[i],
+ i -> cmd.getTargetDatanodes().get(i),
+ (v1, v2) -> v1, TreeMap::new));
}
- public long getContainerID() {
- return containerID;
+ public long getDeadline() {
+ return deadlineMsSinceEpoch;
}
- public byte[] getMissingContainerIndexes() {
- return Arrays
- .copyOf(missingContainerIndexes, missingContainerIndexes.length);
+ public long getContainerID() {
+ return containerID;
}
public ECReplicationConfig getEcReplicationConfig() {
return ecReplicationConfig;
}
- public List getSources() {
- return sources;
+ SortedMap getSourceNodeMap() {
+ return unmodifiableSortedMap(sourceNodeMap);
}
- public List getTargetDatanodes() {
- return targetDatanodes;
+ SortedMap getTargetNodeMap() {
+ return unmodifiableSortedMap(targetNodeMap);
}
@Override
public String toString() {
- return "ECReconstructionCommandInfo{"
+ return "ECReconstructionCommand{"
+ "containerID=" + containerID
- + ", ecReplicationConfig=" + ecReplicationConfig
- + ", missingContainerIndexes=" + Arrays
- .toString(missingContainerIndexes)
- + ", sources=" + sources
- + ", targetDatanodes=" + targetDatanodes + '}';
+ + ", replication=" + ecReplicationConfig.getReplication()
+ + ", missingIndexes=" + Arrays.toString(missingContainerIndexes)
+ + ", sources=" + sourceNodeMap
+ + ", targets=" + targetNodeMap + "}";
+ }
+
+ public long getTerm() {
+ return term;
}
+
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
index d15157956223..4820fbcecb7a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
@@ -46,6 +46,7 @@
import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.security.token.Token;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -58,6 +59,8 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
@@ -105,10 +108,13 @@ public class ECReconstructionCoordinator implements Closeable {
private final TokenHelper tokenHelper;
private final ContainerClientMetrics clientMetrics;
private final ECReconstructionMetrics metrics;
+ private final StateContext context;
public ECReconstructionCoordinator(ConfigurationSource conf,
CertificateClient certificateClient,
+ StateContext context,
ECReconstructionMetrics metrics) throws IOException {
+ this.context = context;
this.containerOperationClient = new ECContainerOperationClient(conf,
certificateClient);
this.byteBufferPool = new ElasticByteBufferPool();
@@ -474,4 +480,10 @@ private long calcEffectiveBlockGroupLen(BlockData[] blockGroup,
public ECReconstructionMetrics getECReconstructionMetrics() {
return this.metrics;
}
+
+ OptionalLong getTermOfLeaderSCM() {
+ return Optional.ofNullable(context)
+ .map(StateContext::getTermOfLeaderSCM)
+ .orElse(OptionalLong.empty());
+ }
}
\ No newline at end of file
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 e0aa14419a4f..03d771da0489 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
@@ -17,36 +17,36 @@
*/
package org.apache.hadoop.ozone.container.ec.reconstruction;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand.DatanodeDetailsAndReplicaIndex;
+import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import java.time.Clock;
+import java.util.OptionalLong;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
/**
* This is the actual EC reconstruction coordination task.
*/
public class ECReconstructionCoordinatorTask implements Runnable {
- static final Logger LOG =
+ private static final Logger LOG =
LoggerFactory.getLogger(ECReconstructionCoordinatorTask.class);
private final ConcurrentHashMap.KeySetView