refreshFunction,
BlockInputStreamFactory streamFactory,
ByteBufferPool byteBufferPool,
- ExecutorService ecReconstructExecutor) {
- super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
- refreshFunction, streamFactory);
+ ExecutorService ecReconstructExecutor,
+ OzoneClientConfig config) {
+ super(repConfig, blockInfo, xceiverClientFactory,
+ refreshFunction, streamFactory, config);
this.byteBufferPool = byteBufferPool;
this.executor = ecReconstructExecutor;
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/TestOzoneClientConfig.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/TestOzoneClientConfig.java
new file mode 100644
index 000000000000..0dd29cb50a45
--- /dev/null
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/TestOzoneClientConfig.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+class TestOzoneClientConfig {
+
+ @Test
+ void missingSizeSuffix() {
+ final int bytes = 1024;
+
+ OzoneConfiguration conf = new OzoneConfiguration();
+ conf.setInt("ozone.client.bytes.per.checksum", bytes);
+
+ OzoneClientConfig subject = conf.getObject(OzoneClientConfig.class);
+
+ assertEquals(OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE, subject.getBytesPerChecksum());
+ }
+}
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java
index ca3199d8acfb..2987a9b6136f 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java
@@ -25,6 +25,7 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
@@ -45,14 +46,14 @@ class DummyBlockInputStream extends BlockInputStream {
long blockLen,
Pipeline pipeline,
Token token,
- boolean verifyChecksum,
XceiverClientFactory xceiverClientManager,
Function refreshFunction,
List chunkList,
- Map chunks) {
+ Map chunks,
+ OzoneClientConfig config) {
super(new BlockLocationInfo(new BlockLocationInfo.Builder().setBlockID(blockId).setLength(blockLen)),
- pipeline, token, verifyChecksum,
- xceiverClientManager, refreshFunction);
+ pipeline, token,
+ xceiverClientManager, refreshFunction, config);
this.chunkDataMap = chunks;
this.chunks = chunkList;
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java
index d66c76dcddcb..172e62887bdd 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java
@@ -25,6 +25,7 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
@@ -52,12 +53,12 @@ final class DummyBlockInputStreamWithRetry
long blockLen,
Pipeline pipeline,
Token token,
- boolean verifyChecksum,
XceiverClientFactory xceiverClientManager,
List chunkList,
Map chunkMap,
- AtomicBoolean isRerfreshed, IOException ioException) {
- super(blockId, blockLen, pipeline, token, verifyChecksum,
+ AtomicBoolean isRerfreshed, IOException ioException,
+ OzoneClientConfig config) {
+ super(blockId, blockLen, pipeline, token,
xceiverClientManager, blockID -> {
isRerfreshed.set(true);
try {
@@ -69,7 +70,7 @@ final class DummyBlockInputStreamWithRetry
throw new RuntimeException(e);
}
- }, chunkList, chunkMap);
+ }, chunkList, chunkMap, config);
this.ioException = ioException;
}
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
index 4db569b7c07a..0012d691f92d 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
@@ -22,9 +22,11 @@
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
@@ -34,6 +36,7 @@
import org.apache.hadoop.ozone.common.Checksum;
import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.ozone.test.GenericTestUtils;
import org.apache.ratis.thirdparty.io.grpc.Status;
import org.apache.ratis.thirdparty.io.grpc.StatusException;
import org.junit.jupiter.api.BeforeEach;
@@ -42,6 +45,7 @@
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.mockito.stubbing.OngoingStubbing;
+import org.slf4j.event.Level;
import java.io.EOFException;
import java.io.IOException;
@@ -58,6 +62,7 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND;
import static org.apache.hadoop.hdds.scm.storage.TestChunkInputStream.generateRandomData;
+import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -86,6 +91,8 @@ public class TestBlockInputStream {
private Function refreshFunction;
+ private OzoneConfiguration conf = new OzoneConfiguration();
+
@BeforeEach
@SuppressWarnings("unchecked")
public void setup() throws Exception {
@@ -93,10 +100,12 @@ public void setup() throws Exception {
BlockID blockID = new BlockID(new ContainerBlockID(1, 1));
checksum = new Checksum(ChecksumType.NONE, CHUNK_SIZE);
createChunkList(5);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(false);
Pipeline pipeline = MockPipeline.createSingleNodePipeline();
blockStream = new DummyBlockInputStream(blockID, blockSize, pipeline, null,
- false, null, refreshFunction, chunks, chunkDataMap);
+ null, refreshFunction, chunks, chunkDataMap, clientConfig);
}
/**
@@ -257,18 +266,25 @@ public void testSeekAndRead() throws Exception {
@Test
public void testRefreshPipelineFunction() throws Exception {
+ GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
+ .captureLogs(BlockInputStream.LOG);
+ GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG);
BlockID blockID = new BlockID(new ContainerBlockID(1, 1));
AtomicBoolean isRefreshed = new AtomicBoolean();
createChunkList(5);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(false);
try (BlockInputStream blockInputStreamWithRetry =
new DummyBlockInputStreamWithRetry(blockID, blockSize,
MockPipeline.createSingleNodePipeline(), null,
- false, null, chunks, chunkDataMap, isRefreshed, null)) {
+ null, chunks, chunkDataMap, isRefreshed, null,
+ clientConfig)) {
assertFalse(isRefreshed.get());
seekAndVerify(50);
byte[] b = new byte[200];
blockInputStreamWithRetry.read(b, 0, 200);
+ assertThat(logCapturer.getOutput()).contains("Retry read after");
assertTrue(isRefreshed.get());
}
}
@@ -348,8 +364,10 @@ private static ChunkInputStream throwingChunkInputStream(IOException ex,
private BlockInputStream createSubject(BlockID blockID, Pipeline pipeline,
ChunkInputStream stream) {
- return new DummyBlockInputStream(blockID, blockSize, pipeline, null, false,
- null, refreshFunction, chunks, null) {
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(false);
+ return new DummyBlockInputStream(blockID, blockSize, pipeline, null,
+ null, refreshFunction, chunks, null, clientConfig) {
@Override
protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) {
return stream;
@@ -401,9 +419,12 @@ public void testRefreshOnReadFailureAfterUnbuffer(IOException ex)
.thenReturn(blockLocationInfo);
when(blockLocationInfo.getPipeline()).thenReturn(newPipeline);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(false);
BlockInputStream subject = new BlockInputStream(
new BlockLocationInfo(new BlockLocationInfo.Builder().setBlockID(blockID).setLength(blockSize)),
- pipeline, null, false, clientFactory, refreshFunction) {
+ pipeline, null, clientFactory, refreshFunction,
+ clientConfig) {
@Override
protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) {
return stream;
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java
index 9b061f5392d3..bf4830c6fcb5 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java
@@ -23,9 +23,13 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
@@ -42,11 +46,17 @@
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
+import static java.util.concurrent.Executors.newFixedThreadPool;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.mock;
@@ -82,6 +92,66 @@ void test(final int writeSize) throws IOException {
}
}
+ /**
+ * Tests an EC offline reconstruction scenario in which none of the ChunkInfo in an EC stripe have stripeChecksum.
+ * Such ChunkInfo will exist for any EC data that was written in a version in which the ChunkInfo protobuf message did
+ * not have the stripeChecksum field. Here, we assert that executePutBlock during reconstruction does not throw an
+ * exception because of missing stripeChecksum. This essentially tests compatibility between an Ozone version that
+ * did not have stripeChecksum and a version that has stripeChecksum.
+ */
+ @Test
+ public void testMissingStripeChecksumDoesNotMakeExecutePutBlockFailDuringECReconstruction() throws IOException {
+ // setup some parameters required for creating ECBlockOutputStream
+ OzoneClientConfig config = new OzoneClientConfig();
+ ECReplicationConfig replicationConfig = new ECReplicationConfig(3, 2);
+ BlockID blockID = new BlockID(1, 1);
+ DatanodeDetails datanodeDetails = MockDatanodeDetails.randomDatanodeDetails();
+ Pipeline pipeline = Pipeline.newBuilder()
+ .setId(PipelineID.valueOf(datanodeDetails.getUuid()))
+ .setReplicationConfig(replicationConfig)
+ .setNodes(ImmutableList.of(datanodeDetails))
+ .setState(Pipeline.PipelineState.CLOSED)
+ // we'll executePutBlock for the parity index 5 because stripeChecksum is written to either the first or the
+ // parity indexes
+ .setReplicaIndexes(ImmutableMap.of(datanodeDetails, 5)).build();
+
+ BlockLocationInfo locationInfo = new BlockLocationInfo.Builder()
+ .setBlockID(blockID)
+ .setOffset(1)
+ .setLength(10)
+ .setPipeline(pipeline).build();
+
+ /*
+ The array of BlockData contains metadata about blocks and their chunks, and is read in executePutBlock. In
+ this test, we deliberately don't write stripeChecksum to any chunk. The expectation is that executePutBlock
+ should not throw an exception because of missing stripeChecksum.
+ */
+ BlockData[] blockData = createBlockDataWithoutStripeChecksum(blockID, replicationConfig);
+ try (ECBlockOutputStream ecBlockOutputStream = createECBlockOutputStream(config, replicationConfig, blockID,
+ pipeline)) {
+ Assertions.assertDoesNotThrow(() -> ecBlockOutputStream.executePutBlock(true, true, locationInfo.getLength(),
+ blockData));
+ }
+ }
+
+ /**
+ * Creates a BlockData array with {@link ECReplicationConfig#getRequiredNodes()} number of elements.
+ */
+ private BlockData[] createBlockDataWithoutStripeChecksum(BlockID blockID, ECReplicationConfig replicationConfig) {
+ int requiredNodes = replicationConfig.getRequiredNodes();
+ BlockData[] blockDataArray = new BlockData[requiredNodes];
+
+ // add just one ChunkInfo to each BlockData.
+ for (int i = 0; i < requiredNodes; i++) {
+ BlockData data = new BlockData(blockID);
+ // create a ChunkInfo with no stripeChecksum
+ ChunkInfo chunkInfo = new ChunkInfo("abc", 0, 10);
+ data.addChunk(chunkInfo.getProtoBufMessage());
+ blockDataArray[i] = data;
+ }
+ return blockDataArray;
+ }
+
private BlockOutputStream createBlockOutputStream(BufferPool bufferPool)
throws IOException {
@@ -108,7 +178,23 @@ private BlockOutputStream createBlockOutputStream(BufferPool bufferPool)
bufferPool,
config,
null,
- ContainerClientMetrics.acquire(), streamBufferArgs);
+ ContainerClientMetrics.acquire(),
+ streamBufferArgs,
+ () -> newFixedThreadPool(10));
+ }
+
+ private ECBlockOutputStream createECBlockOutputStream(OzoneClientConfig clientConfig,
+ ECReplicationConfig repConfig, BlockID blockID, Pipeline pipeline) throws IOException {
+ final XceiverClientManager xcm = mock(XceiverClientManager.class);
+ when(xcm.acquireClient(any()))
+ .thenReturn(new MockXceiverClientSpi(pipeline));
+
+ ContainerClientMetrics clientMetrics = ContainerClientMetrics.acquire();
+ StreamBufferArgs streamBufferArgs =
+ StreamBufferArgs.getDefaultStreamBufferArgs(repConfig, clientConfig);
+
+ return new ECBlockOutputStream(blockID, xcm, pipeline, BufferPool.empty(), clientConfig, null,
+ clientMetrics, streamBufferArgs, () -> newFixedThreadPool(2));
}
/**
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 41bf46a8ea20..049037bc4dce 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
@@ -22,6 +22,7 @@
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
@@ -258,9 +259,10 @@ public synchronized void setFailIndexes(Integer... fail) {
public synchronized BlockExtendedInputStream create(
ReplicationConfig repConfig,
BlockLocationInfo blockInfo, Pipeline pipeline,
- Token token, boolean verifyChecksum,
+ Token token,
XceiverClientFactory xceiverFactory,
- Function refreshFunction) {
+ Function refreshFunction,
+ OzoneClientConfig config) {
int repInd = currentPipeline.getReplicaIndex(pipeline.getNodes().get(0));
TestBlockInputStream stream = new TestBlockInputStream(
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java
index cf3f4f13ef94..623f7a4f86f1 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java
@@ -21,9 +21,11 @@
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
@@ -43,6 +45,8 @@
*/
public class TestBlockInputStreamFactoryImpl {
+ private OzoneConfiguration conf = new OzoneConfiguration();
+
@Test
public void testNonECGivesBlockInputStream() {
BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl();
@@ -52,9 +56,12 @@ public void testNonECGivesBlockInputStream() {
BlockLocationInfo blockInfo = createKeyLocationInfo(repConfig, 3,
1024 * 1024 * 10);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
BlockExtendedInputStream stream =
factory.create(repConfig, blockInfo, blockInfo.getPipeline(),
- blockInfo.getToken(), true, null, null);
+ blockInfo.getToken(), null, null,
+ clientConfig);
assertInstanceOf(BlockInputStream.class, stream);
assertEquals(stream.getBlockID(), blockInfo.getBlockID());
assertEquals(stream.getLength(), blockInfo.getLength());
@@ -69,9 +76,12 @@ public void testECGivesECBlockInputStream() {
BlockLocationInfo blockInfo =
createKeyLocationInfo(repConfig, 5, 1024 * 1024 * 10);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
BlockExtendedInputStream stream =
factory.create(repConfig, blockInfo, blockInfo.getPipeline(),
- blockInfo.getToken(), true, null, null);
+ blockInfo.getToken(), null, null,
+ clientConfig);
assertInstanceOf(ECBlockInputStreamProxy.class, stream);
assertEquals(stream.getBlockID(), blockInfo.getBlockID());
assertEquals(stream.getLength(), blockInfo.getLength());
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 bd34e7546c12..60974b35a95c 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
@@ -20,9 +20,11 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
@@ -59,6 +61,7 @@ public class TestECBlockInputStream {
private ECReplicationConfig repConfig;
private TestBlockInputStreamFactory streamFactory;
+ private OzoneConfiguration conf = new OzoneConfiguration();
@BeforeEach
public void setup() {
@@ -72,15 +75,19 @@ public void testSufficientLocations() {
// EC-3-2, 5MB block, so all 3 data locations are needed
BlockLocationInfo keyInfo = ECStreamTestUtil
.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ keyInfo, null, null, new TestBlockInputStreamFactory(),
+ clientConfig)) {
assertTrue(ecb.hasSufficientLocations());
}
// EC-3-2, very large block, so all 3 data locations are needed
keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5, 5000 * ONEMB);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ keyInfo, null, null, new TestBlockInputStreamFactory(),
+ clientConfig)) {
assertTrue(ecb.hasSufficientLocations());
}
@@ -90,7 +97,8 @@ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB - 1, dnMap);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ keyInfo, null, null, new TestBlockInputStreamFactory(),
+ clientConfig)) {
assertTrue(ecb.hasSufficientLocations());
}
@@ -100,7 +108,8 @@ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5 * ONEMB, dnMap);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ keyInfo, null, null, new TestBlockInputStreamFactory(),
+ clientConfig)) {
assertFalse(ecb.hasSufficientLocations());
}
@@ -112,7 +121,8 @@ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5 * ONEMB, dnMap);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ keyInfo, null, null, new TestBlockInputStreamFactory(),
+ clientConfig)) {
assertFalse(ecb.hasSufficientLocations());
}
}
@@ -124,8 +134,11 @@ public void testCorrectBlockSizePassedToBlockStreamLessThanCell()
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB - 100);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.read(buf);
// We expect only 1 block stream and it should have a length passed of
// ONEMB - 100.
@@ -141,8 +154,11 @@ public void testCorrectBlockSizePassedToBlockStreamTwoCells()
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB + 100);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.read(buf);
List streams = streamFactory.getBlockStreams();
assertEquals(ONEMB, streams.get(0).getLength());
@@ -157,8 +173,11 @@ public void testCorrectBlockSizePassedToBlockStreamThreeCells()
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.read(buf);
List streams = streamFactory.getBlockStreams();
assertEquals(ONEMB, streams.get(0).getLength());
@@ -174,8 +193,11 @@ public void testCorrectBlockSizePassedToBlockStreamThreeFullAndPartialStripe()
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.read(buf);
List streams = streamFactory.getBlockStreams();
assertEquals(4 * ONEMB, streams.get(0).getLength());
@@ -191,8 +213,11 @@ public void testCorrectBlockSizePassedToBlockStreamSingleFullCell()
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.read(buf);
List streams = streamFactory.getBlockStreams();
assertEquals(ONEMB, streams.get(0).getLength());
@@ -206,8 +231,11 @@ public void testCorrectBlockSizePassedToBlockStreamSeveralFullCells()
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 9 * ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.read(buf);
List streams = streamFactory.getBlockStreams();
assertEquals(3 * ONEMB, streams.get(0).getLength());
@@ -220,8 +248,11 @@ public void testCorrectBlockSizePassedToBlockStreamSeveralFullCells()
public void testSimpleRead() throws IOException {
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ByteBuffer buf = ByteBuffer.allocate(100);
@@ -243,8 +274,11 @@ public void testSimpleRead() throws IOException {
public void testSimpleReadUnderOneChunk() throws IOException {
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 1, ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ByteBuffer buf = ByteBuffer.allocate(100);
@@ -262,8 +296,11 @@ public void testSimpleReadUnderOneChunk() throws IOException {
public void testReadPastEOF() throws IOException {
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 50);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ByteBuffer buf = ByteBuffer.allocate(100);
@@ -281,8 +318,11 @@ public void testReadCrossingMultipleECChunkBounds() throws IOException {
100);
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
// EC Chunk size is 100 and 3-2. Create a byte buffer to read 3.5 chunks,
// so 350
@@ -316,8 +356,11 @@ public void testSeekPastBlockLength() throws IOException {
ONEMB);
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
assertThrows(EOFException.class, () -> ecb.seek(1000));
}
}
@@ -328,8 +371,11 @@ public void testSeekToLength() throws IOException {
ONEMB);
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
// When seek more than the length, should throw EOFException.
assertThrows(EOFException.class, () -> ecb.seek(101));
}
@@ -341,8 +387,11 @@ public void testSeekToLengthZeroLengthBlock() throws IOException {
ONEMB);
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 0);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.seek(0);
assertEquals(0, ecb.getPos());
assertEquals(0, ecb.getRemaining());
@@ -355,8 +404,11 @@ public void testSeekToValidPosition() throws IOException {
ONEMB);
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
ecb.seek(ONEMB - 1);
assertEquals(ONEMB - 1, ecb.getPos());
assertEquals(ONEMB * 4 + 1, ecb.getRemaining());
@@ -384,8 +436,11 @@ public void testErrorReadingBlockReportsBadLocation() throws IOException {
ONEMB);
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
// Read a full stripe to ensure all streams are created in the stream
// factory
ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
@@ -415,8 +470,11 @@ public void testNoErrorIfSpareLocationToRead() throws IOException {
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, 8 * ONEMB, datanodes);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
// Read a full stripe to ensure all streams are created in the stream
// factory
ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
@@ -479,8 +537,11 @@ public void testEcPipelineRefreshFunction() {
return blockLocation;
};
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
- keyInfo, true, null, null, streamFactory)) {
+ keyInfo, null, null, streamFactory,
+ clientConfig)) {
Pipeline pipeline =
ecb.ecPipelineRefreshFunction(3, refreshFunction)
.apply(blockID)
@@ -513,8 +574,9 @@ public synchronized List getBlockStreams() {
public synchronized BlockExtendedInputStream create(
ReplicationConfig repConfig, BlockLocationInfo blockInfo,
Pipeline pipeline, Token token,
- boolean verifyChecksum, XceiverClientFactory xceiverFactory,
- Function refreshFunction) {
+ XceiverClientFactory xceiverFactory,
+ Function refreshFunction,
+ OzoneClientConfig config) {
TestBlockInputStream stream = new TestBlockInputStream(
blockInfo.getBlockID(), blockInfo.getLength(),
(byte)blockStreams.size());
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStreamProxy.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStreamProxy.java
index 97bf71c204ad..ca0b9710a960 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStreamProxy.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockInputStreamProxy.java
@@ -20,7 +20,9 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
@@ -52,6 +54,7 @@ public class TestECBlockInputStreamProxy {
private long randomSeed;
private ThreadLocalRandom random = ThreadLocalRandom.current();
private SplittableRandom dataGenerator;
+ private OzoneConfiguration conf = new OzoneConfiguration();
@BeforeEach
public void setup() {
@@ -342,8 +345,11 @@ private void resetAndAdvanceDataGenerator(long position) {
private ECBlockInputStreamProxy createBISProxy(ECReplicationConfig rConfig,
BlockLocationInfo blockInfo) {
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
return new ECBlockInputStreamProxy(
- rConfig, blockInfo, true, null, null, streamFactory);
+ rConfig, blockInfo, null, null, streamFactory,
+ clientConfig);
}
private static class TestECBlockInputStreamFactory
@@ -372,8 +378,9 @@ public List getFailedLocations() {
public BlockExtendedInputStream create(boolean missingLocations,
List failedDatanodes,
ReplicationConfig repConfig, BlockLocationInfo blockInfo,
- boolean verifyChecksum, XceiverClientFactory xceiverFactory,
- Function refreshFunction) {
+ XceiverClientFactory xceiverFactory,
+ Function refreshFunction,
+ OzoneClientConfig config) {
this.failedLocations = failedDatanodes;
ByteBuffer wrappedBuffer =
ByteBuffer.wrap(data.array(), 0, data.capacity());
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedInputStream.java
index 0425f6943a48..6b60bef66af4 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedInputStream.java
@@ -19,7 +19,9 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.io.ElasticByteBufferPool;
@@ -54,6 +56,7 @@ public class TestECBlockReconstructedInputStream {
private ByteBufferPool bufferPool = new ElasticByteBufferPool();
private ExecutorService ecReconstructExecutor =
Executors.newFixedThreadPool(3);
+ private OzoneConfiguration conf = new OzoneConfiguration();
@BeforeEach
public void setup() throws IOException {
@@ -74,8 +77,11 @@ private ECBlockReconstructedStripeInputStream createStripeInputStream(
BlockLocationInfo keyInfo =
ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
streamFactory.setCurrentPipeline(keyInfo.getPipeline());
- return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
- null, null, streamFactory, bufferPool, ecReconstructExecutor);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
+ return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo,
+ null, null, streamFactory, bufferPool, ecReconstructExecutor,
+ clientConfig);
}
@Test
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedStripeInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedStripeInputStream.java
index f7a4bb0643ec..e526b12a5142 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedStripeInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockReconstructedStripeInputStream.java
@@ -20,8 +20,10 @@
import com.google.common.collect.ImmutableSet;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.io.ElasticByteBufferPool;
@@ -73,7 +75,8 @@ public class TestECBlockReconstructedStripeInputStream {
private ByteBufferPool bufferPool = new ElasticByteBufferPool();
private ExecutorService ecReconstructExecutor =
Executors.newFixedThreadPool(3);
-
+ private OzoneConfiguration conf = new OzoneConfiguration();
+
static List> recoveryCases() { // TODO better name
List> params = new ArrayList<>();
params.add(emptySet()); // non-recovery
@@ -808,8 +811,11 @@ public void testFailedLocationsAreNotRead() throws IOException {
private ECBlockReconstructedStripeInputStream createInputStream(
BlockLocationInfo keyInfo) {
- return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
- null, null, streamFactory, bufferPool, ecReconstructExecutor);
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumVerify(true);
+ return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo,
+ null, null, streamFactory, bufferPool, ecReconstructExecutor,
+ clientConfig);
}
private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml
index 20dce15d4d1b..2e0c96ac1d64 100644
--- a/hadoop-hdds/common/pom.xml
+++ b/hadoop-hdds/common/pom.xml
@@ -135,10 +135,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
-
- org.apache.commons
- commons-pool2
-
org.bouncycastle
bcpkix-jdk18on
@@ -181,6 +177,12 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
grpc-api
${io.grpc.version}
compile
+
+
+ com.google.code.findbugs
+ jsr305
+
+
@@ -200,11 +202,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
hdds-test-utils
test
-
- org.junit.jupiter
- junit-jupiter-engine
- test
-
org.junit.platform
junit-platform-launcher
diff --git a/hadoop-hdds/common/src/main/java/com/google/protobuf/Proto2Utils.java b/hadoop-hdds/common/src/main/java/com/google/protobuf/Proto2Utils.java
new file mode 100644
index 000000000000..73cbd3e66f15
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/com/google/protobuf/Proto2Utils.java
@@ -0,0 +1,32 @@
+/*
+ * 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 com.google.protobuf;
+
+/** Utilities for protobuf v2. */
+public final class Proto2Utils {
+ /**
+ * Similar to {@link ByteString#copyFrom(byte[])} except that this method does not copy.
+ * This method is safe only if the content of the array remains unchanged.
+ * Otherwise, it violates the immutability of {@link ByteString}.
+ */
+ public static ByteString unsafeByteString(byte[] array) {
+ return array != null && array.length > 0 ? new LiteralByteString(array) : ByteString.EMPTY;
+ }
+
+ private Proto2Utils() { }
+}
diff --git a/hadoop-hdds/rocks-native/src/main/native/Pipe.cpp b/hadoop-hdds/common/src/main/java/com/google/protobuf/package-info.java
similarity index 70%
rename from hadoop-hdds/rocks-native/src/main/native/Pipe.cpp
rename to hadoop-hdds/common/src/main/java/com/google/protobuf/package-info.java
index f1dd54438700..0cabebdb6965 100644
--- a/hadoop-hdds/rocks-native/src/main/native/Pipe.cpp
+++ b/hadoop-hdds/common/src/main/java/com/google/protobuf/package-info.java
@@ -16,22 +16,7 @@
* limitations under the License.
*/
-#include "Pipe.h"
-#include
-
-const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
-const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
-
-Pipe::Pipe() {
- pipe(p);
- open = true;
-}
-
-Pipe::~Pipe() {
- ::close(p[Pipe::READ_FILE_DESCRIPTOR_IDX]);
- ::close(p[Pipe::WRITE_FILE_DESCRIPTOR_IDX]);
-}
-
-void Pipe::close() {
- open = false;
-}
+/**
+ * Classes using the protobuf internal APIs.
+ */
+package com.google.protobuf;
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java
index 30f9df597b51..e35d20d53e15 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java
@@ -31,6 +31,8 @@ public enum DatanodeVersion implements ComponentVersion {
DEFAULT_VERSION(0, "Initial version"),
SEPARATE_RATIS_PORTS_AVAILABLE(1, "Version with separated Ratis port."),
+ COMBINED_PUTBLOCK_WRITECHUNK_RPC(2, "WriteChunk can optionally support " +
+ "a PutBlock request"),
FUTURE_VERSION(-1, "Used internally in the client when the server side is "
+ " newer and an unknown server version has arrived to the client.");
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 787f023df2ea..609baeeaf7fa 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
@@ -152,6 +152,11 @@ public final class HddsConfigKeys {
+ ".name";
public static final String HDDS_PUBLIC_KEY_FILE_NAME_DEFAULT = "public.pem";
+ public static final String HDDS_HTTP_SERVER_KEYSTORE_TYPE = "ssl.server.keystore.type";
+ public static final String HDDS_HTTP_SERVER_KEYSTORE_TYPE_DEFAULT = "jks";
+ public static final String HDDS_HTTP_SERVER_TRUSTSTORE_TYPE = "ssl.server.truststore.type";
+ public static final String HDDS_HTTP_SERVER_TRUSTSTORE_TYPE_DEFAULT = "jks";
+
public static final String HDDS_BLOCK_TOKEN_EXPIRY_TIME =
"hdds.block.token.expiry.time";
public static final String HDDS_BLOCK_TOKEN_EXPIRY_TIME_DEFAULT = "1d";
@@ -338,6 +343,9 @@ private HddsConfigKeys() {
HDDS_SECURITY_CLIENT_SCM_SECRET_KEY_DATANODE_PROTOCOL_ACL =
"hdds.security.client.scm.secretkey.datanode.protocol.acl";
+ public static final String OZONE_SECURITY_RECONFIGURE_PROTOCOL_ACL =
+ "ozone.security.reconfigure.protocol.acl";
+
// Determines if the Container Chunk Manager will write user data to disk
// Set to false only for specific performance tests
public static final String HDDS_CONTAINER_PERSISTDATA =
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
index ee1c9669a1b1..794b972f1509 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
@@ -438,6 +438,9 @@ public static boolean isReadOnly(
case StreamInit:
case StreamWrite:
case FinalizeBlock:
+ return false;
+ case Echo:
+ return proto.getEcho().hasReadOnly() && proto.getEcho().getReadOnly();
default:
return false;
}
@@ -807,7 +810,7 @@ public static Map processForLogging(OzoneConfiguration conf) {
}
@Nonnull
- public static String threadNamePrefix(@Nullable String id) {
+ public static String threadNamePrefix(@Nullable Object id) {
return id != null && !"".equals(id)
? id + "-"
: "";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java
new file mode 100644
index 000000000000..7d5b610b0875
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java
@@ -0,0 +1,153 @@
+/*
+ * 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.client;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import jakarta.annotation.Nullable;
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.annotation.InterfaceStability;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Decommission specific stateless utility functions.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public final class DecommissionUtils {
+
+
+ private static final Logger LOG = LoggerFactory.getLogger(DecommissionUtils.class);
+
+ private DecommissionUtils() {
+ }
+
+ /**
+ * Returns the list of uuid or ipAddress matching decommissioning status nodes.
+ *
+ * @param allNodes All datanodes which are in decommissioning status.
+ * @param uuid node uuid.
+ * @param ipAddress node ipAddress
+ * @return the list of uuid or ipAddress matching decommissioning status nodes.
+ */
+ public static List getDecommissioningNodesList(Stream allNodes,
+ String uuid,
+ String ipAddress) {
+ List decommissioningNodes;
+ if (!Strings.isNullOrEmpty(uuid)) {
+ decommissioningNodes = allNodes.filter(p -> p.getNodeID().getUuid()
+ .equals(uuid)).collect(Collectors.toList());
+ } else if (!Strings.isNullOrEmpty(ipAddress)) {
+ decommissioningNodes = allNodes.filter(p -> p.getNodeID().getIpAddress()
+ .compareToIgnoreCase(ipAddress) == 0).collect(Collectors.toList());
+ } else {
+ decommissioningNodes = allNodes.collect(Collectors.toList());
+ }
+ return decommissioningNodes;
+ }
+
+ /**
+ * Returns Json node of datanode metrics.
+ *
+ * @param metricsJson
+ * @return Json node of datanode metrics
+ * @throws IOException
+ */
+ public static JsonNode getBeansJsonNode(String metricsJson) throws IOException {
+ JsonNode jsonNode;
+ ObjectMapper objectMapper = new ObjectMapper();
+ JsonFactory factory = objectMapper.getFactory();
+ JsonParser parser = factory.createParser(metricsJson);
+ jsonNode = (JsonNode) objectMapper.readTree(parser).get("beans").get(0);
+ return jsonNode;
+ }
+
+ /**
+ * Returns the number of decommissioning nodes.
+ *
+ * @param jsonNode
+ * @return
+ */
+ public static int getNumDecomNodes(JsonNode jsonNode) {
+ int numDecomNodes;
+ JsonNode totalDecom = jsonNode.get("DecommissioningMaintenanceNodesTotal");
+ numDecomNodes = (totalDecom == null ? -1 : Integer.parseInt(totalDecom.toString()));
+ return numDecomNodes;
+ }
+
+ /**
+ * Returns the counts of following info attributes.
+ * - decommissionStartTime
+ * - numOfUnclosedPipelines
+ * - numOfUnderReplicatedContainers
+ * - numOfUnclosedContainers
+ *
+ * @param datanode
+ * @param counts
+ * @param numDecomNodes
+ * @param countsMap
+ * @param errMsg
+ * @return
+ * @throws IOException
+ */
+ @Nullable
+ public static Map getCountsMap(DatanodeDetails datanode, JsonNode counts, int numDecomNodes,
+ Map countsMap, String errMsg)
+ throws IOException {
+ for (int i = 1; i <= numDecomNodes; i++) {
+ if (datanode.getHostName().equals(counts.get("tag.datanode." + i).asText())) {
+ JsonNode pipelinesDN = counts.get("PipelinesWaitingToCloseDN." + i);
+ JsonNode underReplicatedDN = counts.get("UnderReplicatedDN." + i);
+ JsonNode unclosedDN = counts.get("UnclosedContainersDN." + i);
+ JsonNode startTimeDN = counts.get("StartTimeDN." + i);
+ if (pipelinesDN == null || underReplicatedDN == null || unclosedDN == null || startTimeDN == null) {
+ throw new IOException(errMsg);
+ }
+
+ int pipelines = Integer.parseInt(pipelinesDN.toString());
+ double underReplicated = Double.parseDouble(underReplicatedDN.toString());
+ double unclosed = Double.parseDouble(unclosedDN.toString());
+ long startTime = Long.parseLong(startTimeDN.toString());
+ Date date = new Date(startTime);
+ DateFormat formatter = new SimpleDateFormat("dd/MM/yyyy hh:mm:ss z");
+ countsMap.put("decommissionStartTime", formatter.format(date));
+ countsMap.put("numOfUnclosedPipelines", pipelines);
+ countsMap.put("numOfUnderReplicatedContainers", underReplicated);
+ countsMap.put("numOfUnclosedContainers", unclosed);
+ return countsMap;
+ }
+ }
+ return null;
+ }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
index 3e3990b8b4ba..18a931546d1b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdds.client;
+import net.jcip.annotations.Immutable;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import java.util.Objects;
@@ -24,6 +25,7 @@
/**
* Replication configuration for EC replication.
*/
+@Immutable
public class DefaultReplicationConfig {
private final ECReplicationConfig ecReplicationConfig;
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
index 25ea315af284..a6dbd933ff1d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.client;
import com.fasterxml.jackson.annotation.JsonIgnore;
+import net.jcip.annotations.Immutable;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import java.util.EnumSet;
@@ -30,6 +31,7 @@
/**
* Replication configuration for EC replication.
*/
+@Immutable
public class ECReplicationConfig implements ReplicationConfig {
public static final String EC_REPLICATION_PARAMS_DELIMITER = "-";
@@ -60,17 +62,16 @@ public static String allValuesAsString() {
private static final Pattern STRING_FORMAT
= Pattern.compile("([a-zA-Z]+)-(\\d+)-(\\d+)-(\\d+)([kK])?");
- private int data;
+ private final int data;
- private int parity;
+ private final int parity;
- private int ecChunkSize = 1024 * 1024;
+ private final int ecChunkSize;
- private EcCodec codec = EcCodec.RS;
+ private final EcCodec codec;
public ECReplicationConfig(int data, int parity) {
- this.data = data;
- this.parity = parity;
+ this(data, parity, EcCodec.RS, 1024 * 1024);
}
public ECReplicationConfig(int data, int parity, EcCodec codec,
@@ -121,7 +122,7 @@ public ECReplicationConfig(String string) {
") be greater than zero");
}
if (matcher.group(5) != null) {
- // The "k" modifier is present, so multiple by 1024
+ // The "k" modifier is present, so multiply by 1024
chunkSize = chunkSize * 1024;
}
ecChunkSize = chunkSize;
@@ -154,6 +155,14 @@ public String getReplication() {
+ chunkKB();
}
+ /** Similar to {@link #getReplication()}, but applies to proto structure, without any validation. */
+ public static String toString(HddsProtos.ECReplicationConfig proto) {
+ return proto.getCodec() + EC_REPLICATION_PARAMS_DELIMITER
+ + proto.getData() + EC_REPLICATION_PARAMS_DELIMITER
+ + proto.getParity() + EC_REPLICATION_PARAMS_DELIMITER
+ + proto.getEcChunkSize();
+ }
+
public HddsProtos.ECReplicationConfig toProto() {
return HddsProtos.ECReplicationConfig.newBuilder()
.setData(data)
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/OzoneQuota.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/OzoneQuota.java
index b3a762e2eda1..c8cf4fdd42ba 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/OzoneQuota.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/OzoneQuota.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
@@ -19,12 +19,13 @@
package org.apache.hadoop.hdds.client;
import com.google.common.base.Strings;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.ratis.util.Preconditions;
-import static org.apache.hadoop.ozone.OzoneConsts.GB;
-import static org.apache.hadoop.ozone.OzoneConsts.KB;
-import static org.apache.hadoop.ozone.OzoneConsts.MB;
-import static org.apache.hadoop.ozone.OzoneConsts.TB;
-
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
/**
* represents an OzoneQuota Object that can be applied to
@@ -32,39 +33,71 @@
*/
public final class OzoneQuota {
- public static final String OZONE_QUOTA_B = "B";
- public static final String OZONE_QUOTA_KB = "KB";
- public static final String OZONE_QUOTA_MB = "MB";
- public static final String OZONE_QUOTA_GB = "GB";
- public static final String OZONE_QUOTA_TB = "TB";
-
/** Quota Units.*/
- public enum Units { B, KB, MB, GB, TB }
+ public enum Units {
+ // the names and the ordering are important
+ B(1),
+ KB(OzoneConsts.KB),
+ MB(OzoneConsts.MB),
+ GB(OzoneConsts.GB),
+ TB(OzoneConsts.TB),
+ PB(OzoneConsts.PB),
+ EB(OzoneConsts.EB);
+
+ private final long size;
+ private final List cache;
+
+ Units(long size) {
+ this.size = size;
+ this.cache = createCache(this);
+ }
- // Quota to decide how many buckets can be created.
- private long quotaInNamespace;
- // Quota to decide how many storage space will be used in bytes.
- private long quotaInBytes;
- private RawQuotaInBytes rawQuotaInBytes;
- // Data class of Quota.
- private static QuotaList quotaList;
+ private static List createCache(Units unit) {
+ final List quotas = new ArrayList<>(1024);
+ for (int i = 0; i < 1024; i++) {
+ quotas.add(new RawQuotaInBytes(unit, i));
+ }
+ return Collections.unmodifiableList(quotas);
+ }
+
+ public long getSize() {
+ return size;
+ }
- /** Setting QuotaList parameters from large to small. */
+ RawQuotaInBytes getRawQuotaInBytes(long b) {
+ return b < cache.size() ? cache.get(Math.toIntExact(b))
+ : new RawQuotaInBytes(this, b);
+ }
+ }
+
+ private static final List PARSE_ORDER;
static {
- quotaList = new QuotaList();
- quotaList.addQuotaList(OZONE_QUOTA_TB, Units.TB, TB);
- quotaList.addQuotaList(OZONE_QUOTA_GB, Units.GB, GB);
- quotaList.addQuotaList(OZONE_QUOTA_MB, Units.MB, MB);
- quotaList.addQuotaList(OZONE_QUOTA_KB, Units.KB, KB);
- quotaList.addQuotaList(OZONE_QUOTA_B, Units.B, 1L);
+ List reversed = new ArrayList<>(Arrays.asList(Units.values()));
+ Collections.reverse(reversed);
+ PARSE_ORDER = Collections.unmodifiableList(reversed);
}
+ // Quota to decide how many buckets can be created.
+ private long quotaInNamespace;
+ // Quota to decide how many storage space will be used in bytes.
+ private final long quotaInBytes;
+ private final RawQuotaInBytes rawQuotaInBytes;
+
/**
* Used to convert user input values into bytes such as: 1MB-> 1048576.
*/
private static class RawQuotaInBytes {
- private Units unit;
- private long size;
+ static RawQuotaInBytes valueOf(long quotaInBytes) {
+ Preconditions.assertTrue(quotaInBytes >= 0, () -> "quotaInBytes = " + quotaInBytes + " must be >= 0");
+ final int i = Long.numberOfTrailingZeros(quotaInBytes) / 10;
+ final Units unit = Units.values()[i];
+ final RawQuotaInBytes b = unit.getRawQuotaInBytes(quotaInBytes >> (i * 10));
+ Preconditions.assertSame(quotaInBytes, b.sizeInBytes(), "sizeInBytes");
+ return b;
+ }
+
+ private final Units unit;
+ private final long size;
RawQuotaInBytes(Units unit, long size) {
this.unit = unit;
@@ -83,14 +116,7 @@ public long getSize() {
* Returns size in Bytes or negative num if there is no Quota.
*/
public long sizeInBytes() {
- long sQuota = -1L;
- for (Units quota : quotaList.getUnitQuotaArray()) {
- if (quota == this.unit) {
- sQuota = quotaList.getQuotaSize(quota);
- break;
- }
- }
- return this.getSize() * sQuota;
+ return this.getSize() * getUnit().getSize();
}
@Override
@@ -158,20 +184,21 @@ public static OzoneQuota parseSpaceQuota(String quotaInBytes) {
String uppercase = quotaInBytes.toUpperCase()
.replaceAll("\\s+", "");
String size = "";
- long nSize = 0;
+ final long nSize;
Units currUnit = Units.B;
try {
- for (String quota : quotaList.getOzoneQuotaArray()) {
+ for (Units unit : PARSE_ORDER) {
+ final String quota = unit.name();
if (uppercase.endsWith((quota))) {
size = uppercase
.substring(0, uppercase.length() - quota.length());
- currUnit = quotaList.getUnits(quota);
+ currUnit = unit;
break;
}
}
// there might be no unit specified.
- if (size.equals("")) {
+ if (size.isEmpty()) {
size = uppercase;
}
nSize = Long.parseLong(size);
@@ -240,15 +267,7 @@ public static OzoneQuota parseQuota(String quotaInBytes,
*/
public static OzoneQuota getOzoneQuota(long quotaInBytes,
long quotaInNamespace) {
- long size = 1L;
- Units unit = Units.B;
- for (Long quota : quotaList.getSizeQuotaArray()) {
- if (quotaInBytes % quota == 0) {
- size = quotaInBytes / quota;
- unit = quotaList.getQuotaUnit(quota);
- }
- }
- return new OzoneQuota(quotaInNamespace, new RawQuotaInBytes(unit, size));
+ return new OzoneQuota(quotaInNamespace, RawQuotaInBytes.valueOf(quotaInBytes));
}
public long getQuotaInNamespace() {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/QuotaList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/QuotaList.java
deleted file mode 100644
index 230b825f4d45..000000000000
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/QuotaList.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * 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.client;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *This class contains arraylist for storage constant used in OzoneQuota.
- */
-public class QuotaList {
- private final ArrayList ozoneQuota;
- private final ArrayList unitQuota;
- private final ArrayList sizeQuota;
-
- public QuotaList() {
- ozoneQuota = new ArrayList<>();
- unitQuota = new ArrayList<>();
- sizeQuota = new ArrayList<>();
- }
-
- public void addQuotaList(
- String oQuota, OzoneQuota.Units uQuota, Long sQuota) {
- ozoneQuota.add(oQuota);
- unitQuota.add(uQuota);
- sizeQuota.add(sQuota);
- }
-
- public List getOzoneQuotaArray() {
- return this.ozoneQuota;
- }
-
- public List getSizeQuotaArray() {
- return this.sizeQuota;
- }
-
- public List getUnitQuotaArray() {
- return this.unitQuota;
- }
-
- public OzoneQuota.Units getUnits(String oQuota) {
- return unitQuota.get(ozoneQuota.indexOf(oQuota));
- }
-
- public Long getQuotaSize(OzoneQuota.Units uQuota) {
- return sizeQuota.get(unitQuota.indexOf(uQuota));
- }
-
- public OzoneQuota.Units getQuotaUnit(Long sQuota) {
- return unitQuota.get(sizeQuota.indexOf(sQuota));
- }
-
-}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
index 377f8cd998a7..36d4d90e1afb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
@@ -20,6 +20,7 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
+import net.jcip.annotations.Immutable;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -31,6 +32,7 @@
/**
* Replication configuration for Ratis replication.
*/
+@Immutable
public final class RatisReplicationConfig
implements ReplicatedReplicationConfig {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
index e9d77e9bebaf..9ca2dfb538a9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
@@ -20,6 +20,7 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
+import net.jcip.annotations.Immutable;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -31,6 +32,7 @@
/**
* Replication configuration for STANDALONE replication.
*/
+@Immutable
public final class StandaloneReplicationConfig implements
ReplicatedReplicationConfig {
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 69cce8db6d6b..b8742c6ba929 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
@@ -30,6 +30,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
+import java.util.Collections;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.List;
@@ -47,6 +48,7 @@
import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.ratis.server.RaftServerConfigKeys;
import static java.util.Collections.unmodifiableSortedSet;
@@ -162,15 +164,11 @@ public XMLConfiguration() {
}
public XMLConfiguration(List properties) {
- this.properties = properties;
+ this.properties = new ArrayList<>(properties);
}
public List getProperties() {
- return properties;
- }
-
- public void setProperties(List properties) {
- this.properties = properties;
+ return Collections.unmodifiableList(properties);
}
}
@@ -323,7 +321,67 @@ private static void addDeprecatedKeys() {
new DeprecationDelta("ozone.scm.chunk.layout",
ScmConfigKeys.OZONE_SCM_CONTAINER_LAYOUT_KEY),
new DeprecationDelta("hdds.datanode.replication.work.dir",
- OZONE_CONTAINER_COPY_WORKDIR)
+ OZONE_CONTAINER_COPY_WORKDIR),
+ new DeprecationDelta("dfs.container.chunk.write.sync",
+ OzoneConfigKeys.HDDS_CONTAINER_CHUNK_WRITE_SYNC_KEY),
+ new DeprecationDelta("dfs.container.ipc",
+ OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT),
+ new DeprecationDelta("dfs.container.ipc.random.port",
+ OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT),
+ new DeprecationDelta("dfs.container.ratis.admin.port",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_ADMIN_PORT),
+ new DeprecationDelta("dfs.container.ratis.datanode.storage.dir",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATANODE_STORAGE_DIR),
+ new DeprecationDelta("dfs.container.ratis.datastream.enabled",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATASTREAM_ENABLED),
+ new DeprecationDelta("dfs.container.ratis.datastream.port",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATASTREAM_PORT),
+ new DeprecationDelta("dfs.container.ratis.datastream.random.port",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT),
+ new DeprecationDelta("dfs.container.ratis.enabled",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_ENABLED_KEY),
+ new DeprecationDelta("dfs.container.ratis.ipc",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_PORT),
+ new DeprecationDelta("dfs.container.ratis.ipc.random.port",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_RANDOM_PORT),
+ new DeprecationDelta("dfs.container.ratis.leader.pending.bytes.limit",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT),
+ new DeprecationDelta("dfs.container.ratis.log.appender.queue.byte-limit",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT),
+ new DeprecationDelta("dfs.container.ratis.log.appender.queue.num-elements",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS),
+ new DeprecationDelta("dfs.container.ratis.log.purge.gap",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_LOG_PURGE_GAP),
+ new DeprecationDelta("dfs.container.ratis.log.queue.byte-limit",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_LOG_QUEUE_BYTE_LIMIT),
+ new DeprecationDelta("dfs.container.ratis.log.queue.num-elements",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_LOG_QUEUE_NUM_ELEMENTS),
+ new DeprecationDelta("dfs.container.ratis.num.container.op.executors",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_KEY),
+ new DeprecationDelta("dfs.container.ratis.num.write.chunk.threads.per.volume",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME),
+ new DeprecationDelta("dfs.container.ratis.replication.level",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY),
+ new DeprecationDelta("dfs.container.ratis.rpc.type",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_RPC_TYPE_KEY),
+ new DeprecationDelta("dfs.container.ratis.segment.preallocated.size",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY),
+ new DeprecationDelta("dfs.container.ratis.segment.size",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_SEGMENT_SIZE_KEY),
+ new DeprecationDelta("dfs.container.ratis.server.port",
+ OzoneConfigKeys.HDDS_CONTAINER_RATIS_SERVER_PORT),
+ new DeprecationDelta("dfs.container.ratis.statemachinedata.sync.retries",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_RETRIES),
+ new DeprecationDelta("dfs.container.ratis.statemachinedata.sync.timeout",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT),
+ new DeprecationDelta("dfs.container.ratis.statemachine.max.pending.apply-transactions",
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS),
+ new DeprecationDelta("dfs.ratis.leader.election.minimum.timeout.duration",
+ ScmConfigKeys.HDDS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY),
+ new DeprecationDelta("dfs.ratis.server.retry-cache.timeout.duration",
+ ScmConfigKeys.HDDS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY),
+ new DeprecationDelta("dfs.ratis.snapshot.threshold",
+ ScmConfigKeys.HDDS_RATIS_SNAPSHOT_THRESHOLD_KEY)
});
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java
index 2d29dc8565c8..ba203f9c8e2d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.freon;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.UUID;
@@ -36,40 +37,49 @@
* Class to store pre-generated topology information for load-tests.
*/
@SuppressWarnings("java:S2245") // no need for secure random
-public class FakeClusterTopology {
+public final class FakeClusterTopology {
private static final Logger LOGGER =
LoggerFactory.getLogger(FakeClusterTopology.class);
- public static final FakeClusterTopology INSTANCE = new FakeClusterTopology();
+ public static final FakeClusterTopology INSTANCE = newFakeClusterTopology();
- private List datanodes = new ArrayList<>();
+ private final List datanodes;
- private List pipelines = new ArrayList<>();
+ private final List pipelines;
- private Random random = new Random();
+ private final Random random = new Random();
- public FakeClusterTopology() {
+ private static FakeClusterTopology newFakeClusterTopology() {
+ final int nodeCount = 9;
+ final List datanodes = new ArrayList<>(nodeCount);
+ final List pipelines = new ArrayList<>(nodeCount / 3);
try {
- for (int i = 0; i < 9; i++) {
+ for (int i = 0; i < nodeCount; i++) {
datanodes.add(createDatanode());
if ((i + 1) % 3 == 0) {
pipelines.add(Pipeline.newBuilder()
.setId(PipelineID.randomId().getProtobuf())
.setFactor(ReplicationFactor.THREE)
.setType(ReplicationType.RATIS)
- .addMembers(getDatanode(i - 2))
- .addMembers(getDatanode(i - 1))
- .addMembers(getDatanode(i))
+ .addMembers(datanodes.get(i - 2))
+ .addMembers(datanodes.get(i - 1))
+ .addMembers(datanodes.get(i))
.build());
}
}
} catch (Exception ex) {
LOGGER.error("Can't initialize FakeClusterTopology", ex);
}
+ return new FakeClusterTopology(datanodes, pipelines);
}
- private DatanodeDetailsProto createDatanode() {
+ private FakeClusterTopology(List datanodes, List pipelines) {
+ this.datanodes = Collections.unmodifiableList(datanodes);
+ this.pipelines = Collections.unmodifiableList(pipelines);
+ }
+
+ private static DatanodeDetailsProto createDatanode() {
return DatanodeDetailsProto.newBuilder()
.setUuid(UUID.randomUUID().toString())
.setHostName("localhost")
@@ -79,15 +89,11 @@ private DatanodeDetailsProto createDatanode() {
.build();
}
- public DatanodeDetailsProto getDatanode(int i) {
- return datanodes.get(i);
- }
-
public Pipeline getRandomPipeline() {
return pipelines.get(random.nextInt(pipelines.size()));
}
- public List getAllDatanodes() {
+ public Iterable getAllDatanodes() {
return datanodes;
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java
index 6f776072d9c3..b9a2f87a03da 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java
@@ -94,7 +94,19 @@ public void incrementUsedSpace(long usedSpace) {
}
public void decrementUsedSpace(long reclaimedSpace) {
- cachedValue.addAndGet(-1 * reclaimedSpace);
+ cachedValue.updateAndGet(current -> {
+ long newValue = current - reclaimedSpace;
+ if (newValue < 0) {
+ if (current > 0) {
+ LOG.warn("Attempted to decrement used space to a negative value. " +
+ "Current: {}, Decrement: {}, Source: {}",
+ current, reclaimedSpace, source);
+ }
+ return 0;
+ } else {
+ return newValue;
+ }
+ });
}
public void start() {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/SpaceUsageSource.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/SpaceUsageSource.java
index c25c0a40c53e..a367cfbdc061 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/SpaceUsageSource.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/SpaceUsageSource.java
@@ -57,9 +57,9 @@ final class Fixed implements SpaceUsageSource {
private final long available;
private final long used;
- Fixed(long capacity, long available, long used) {
+ public Fixed(long capacity, long available, long used) {
this.capacity = capacity;
- this.available = available;
+ this.available = Math.max(Math.min(available, capacity - used), 0);
this.used = used;
}
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 5b6fb6fe9b81..68a640e6e9fc 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
@@ -26,8 +26,11 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.google.common.collect.ImmutableSet;
+import com.google.protobuf.ByteString;
import org.apache.hadoop.hdds.DatanodeVersion;
import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.scm.net.NetUtils;
+import org.apache.hadoop.util.StringWithByteString;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.annotation.InterfaceStability;
import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
@@ -81,75 +84,58 @@ public static Codec getCodec() {
* DataNode's unique identifier in the cluster.
*/
private final UUID uuid;
- private final String uuidString;
+ private final StringWithByteString uuidString;
private final String threadNamePrefix;
-
- private String ipAddress;
- private String hostName;
- private List ports;
+ private StringWithByteString ipAddress;
+ private StringWithByteString hostName;
+ private final List ports;
private String certSerialId;
private String version;
private long setupTime;
private String revision;
private String buildDate;
private volatile HddsProtos.NodeOperationalState persistedOpState;
- private volatile long persistedOpStateExpiryEpochSec = 0;
+ private volatile long persistedOpStateExpiryEpochSec;
private int initialVersion;
private int currentVersion;
- /**
- * Constructs DatanodeDetails instance. DatanodeDetails.Builder is used
- * for instantiating DatanodeDetails.
- * @param uuid DataNode's UUID
- * @param ipAddress IP Address of this DataNode
- * @param hostName DataNode's hostname
- * @param networkLocation DataNode's network location path
- * @param ports Ports used by the DataNode
- * @param certSerialId serial id from SCM issued certificate.
- * @param version DataNode's version
- * @param setupTime the setup time of DataNode
- * @param revision DataNodes's revision
- * @param buildDate DataNodes's build timestamp
- * @param persistedOpState Operational State stored on DN.
- * @param persistedOpStateExpiryEpochSec Seconds after the epoch the stored
- * state should expire.
- */
- @SuppressWarnings("parameternumber")
- private DatanodeDetails(UUID uuid, String ipAddress, String hostName,
- String networkLocation, List ports, String certSerialId,
- String version, long setupTime, String revision, String buildDate,
- HddsProtos.NodeOperationalState persistedOpState,
- long persistedOpStateExpiryEpochSec,
- int initialVersion, int currentVersion) {
- super(hostName, networkLocation, NetConstants.NODE_COST_DEFAULT);
- this.uuid = uuid;
- this.uuidString = uuid.toString();
+ private DatanodeDetails(Builder b) {
+ super(b.hostName, b.networkLocation, NetConstants.NODE_COST_DEFAULT);
+ uuid = b.id;
+ uuidString = StringWithByteString.valueOf(uuid.toString());
threadNamePrefix = HddsUtils.threadNamePrefix(uuidString);
- this.ipAddress = ipAddress;
- this.hostName = hostName;
- this.ports = ports;
- this.certSerialId = certSerialId;
- this.version = version;
- this.setupTime = setupTime;
- this.revision = revision;
- this.buildDate = buildDate;
- this.persistedOpState = persistedOpState;
- this.persistedOpStateExpiryEpochSec = persistedOpStateExpiryEpochSec;
- this.initialVersion = initialVersion;
- this.currentVersion = currentVersion;
+ ipAddress = b.ipAddress;
+ hostName = b.hostName;
+ ports = b.ports;
+ certSerialId = b.certSerialId;
+ version = b.version;
+ setupTime = b.setupTime;
+ revision = b.revision;
+ buildDate = b.buildDate;
+ persistedOpState = b.persistedOpState;
+ persistedOpStateExpiryEpochSec = b.persistedOpStateExpiryEpochSec;
+ initialVersion = b.initialVersion;
+ currentVersion = b.currentVersion;
+ if (b.networkName != null) {
+ setNetworkName(b.networkName);
+ }
+ if (b.level > 0) {
+ setLevel(b.level);
+ }
}
public DatanodeDetails(DatanodeDetails datanodeDetails) {
- super(datanodeDetails.getHostName(), datanodeDetails.getNetworkLocation(),
+ super(datanodeDetails.getHostNameAsByteString(), datanodeDetails.getNetworkLocationAsByteString(),
datanodeDetails.getParent(), datanodeDetails.getLevel(),
datanodeDetails.getCost());
this.uuid = datanodeDetails.uuid;
- this.uuidString = uuid.toString();
+ this.uuidString = datanodeDetails.uuidString;
threadNamePrefix = HddsUtils.threadNamePrefix(uuidString);
this.ipAddress = datanodeDetails.ipAddress;
this.hostName = datanodeDetails.hostName;
this.ports = datanodeDetails.ports;
- this.setNetworkName(datanodeDetails.getNetworkName());
+ this.certSerialId = datanodeDetails.certSerialId;
+ this.setNetworkName(datanodeDetails.getNetworkNameAsByteString());
this.setParent(datanodeDetails.getParent());
this.version = datanodeDetails.version;
this.setupTime = datanodeDetails.setupTime;
@@ -177,7 +163,7 @@ public UUID getUuid() {
* @return UUID of DataNode
*/
public String getUuidString() {
- return uuidString;
+ return uuidString.getString();
}
/**
@@ -186,7 +172,7 @@ public String getUuidString() {
* @param ip IP Address
*/
public void setIpAddress(String ip) {
- this.ipAddress = ip;
+ this.ipAddress = StringWithByteString.valueOf(ip);
}
/**
@@ -195,6 +181,15 @@ public void setIpAddress(String ip) {
* @return IP address
*/
public String getIpAddress() {
+ return ipAddress.getString();
+ }
+
+ /**
+ * Returns IP address of DataNode as a StringWithByteString object.
+ *
+ * @return IP address as ByteString
+ */
+ public StringWithByteString getIpAddressAsByteString() {
return ipAddress;
}
@@ -204,7 +199,7 @@ public String getIpAddress() {
* @param host hostname
*/
public void setHostName(String host) {
- this.hostName = host;
+ this.hostName = StringWithByteString.valueOf(host);
}
/**
@@ -213,6 +208,15 @@ public void setHostName(String host) {
* @return Hostname
*/
public String getHostName() {
+ return hostName.getString();
+ }
+
+ /**
+ * Returns IP address of DataNode as a StringWithByteString object.
+ *
+ * @return Hostname
+ */
+ public StringWithByteString getHostNameAsByteString() {
return hostName;
}
@@ -238,7 +242,16 @@ public synchronized void setPort(Name name, int port) {
* @return DataNode Ports
*/
public synchronized List getPorts() {
- return ports;
+ return new ArrayList<>(ports);
+ }
+
+ public synchronized boolean hasPort(int port) {
+ for (Port p : ports) {
+ if (p.getValue() == port) {
+ return true;
+ }
+ }
+ return false;
}
/**
@@ -343,10 +356,10 @@ public static DatanodeDetails.Builder newBuilder(
}
if (datanodeDetailsProto.hasIpAddress()) {
- builder.setIpAddress(datanodeDetailsProto.getIpAddress());
+ builder.setIpAddress(datanodeDetailsProto.getIpAddress(), datanodeDetailsProto.getIpAddressBytes());
}
if (datanodeDetailsProto.hasHostName()) {
- builder.setHostName(datanodeDetailsProto.getHostName());
+ builder.setHostName(datanodeDetailsProto.getHostName(), datanodeDetailsProto.getHostNameBytes());
}
if (datanodeDetailsProto.hasCertSerialId()) {
builder.setCertSerialId(datanodeDetailsProto.getCertSerialId());
@@ -359,10 +372,15 @@ public static DatanodeDetails.Builder newBuilder(
}
}
if (datanodeDetailsProto.hasNetworkName()) {
- builder.setNetworkName(datanodeDetailsProto.getNetworkName());
+ builder.setNetworkName(
+ datanodeDetailsProto.getNetworkName(), datanodeDetailsProto.getNetworkNameBytes());
}
if (datanodeDetailsProto.hasNetworkLocation()) {
- builder.setNetworkLocation(datanodeDetailsProto.getNetworkLocation());
+ builder.setNetworkLocation(
+ datanodeDetailsProto.getNetworkLocation(), datanodeDetailsProto.getNetworkLocationBytes());
+ }
+ if (datanodeDetailsProto.hasLevel()) {
+ builder.setLevel(datanodeDetailsProto.getLevel());
}
if (datanodeDetailsProto.hasPersistedOpState()) {
builder.setPersistedOpState(datanodeDetailsProto.getPersistedOpState());
@@ -371,6 +389,9 @@ public static DatanodeDetails.Builder newBuilder(
builder.setPersistedOpStateExpiry(
datanodeDetailsProto.getPersistedOpStateExpiry());
}
+ if (datanodeDetailsProto.hasCurrentVersion()) {
+ builder.setCurrentVersion(datanodeDetailsProto.getCurrentVersion());
+ }
return builder;
}
@@ -439,22 +460,25 @@ public HddsProtos.DatanodeDetailsProto.Builder toProtoBuilder(
HddsProtos.DatanodeDetailsProto.newBuilder()
.setUuid128(uuid128);
- builder.setUuid(getUuidString());
+ builder.setUuidBytes(uuidString.getBytes());
if (ipAddress != null) {
- builder.setIpAddress(ipAddress);
+ builder.setIpAddressBytes(ipAddress.getBytes());
}
if (hostName != null) {
- builder.setHostName(hostName);
+ builder.setHostNameBytes(hostName.getBytes());
}
if (certSerialId != null) {
builder.setCertSerialId(certSerialId);
}
if (!Strings.isNullOrEmpty(getNetworkName())) {
- builder.setNetworkName(getNetworkName());
+ builder.setNetworkNameBytes(getNetworkNameAsByteString().getBytes());
}
if (!Strings.isNullOrEmpty(getNetworkLocation())) {
- builder.setNetworkLocation(getNetworkLocation());
+ builder.setNetworkLocationBytes(getNetworkLocationAsByteString().getBytes());
+ }
+ if (getLevel() > 0) {
+ builder.setLevel(getLevel());
}
if (persistedOpState != null) {
builder.setPersistedOpState(persistedOpState);
@@ -475,6 +499,8 @@ public HddsProtos.DatanodeDetailsProto.Builder toProtoBuilder(
}
}
+ builder.setCurrentVersion(currentVersion);
+
return builder;
}
@@ -505,6 +531,7 @@ public ExtendedDatanodeDetailsProto getExtendedProtoBufMessage() {
}
/**
+ * Note: Datanode initial version is not passed to the client due to no use case. See HDDS-9884
* @return the version this datanode was initially created with
*/
public int getInitialVersion() {
@@ -581,10 +608,11 @@ public String threadNamePrefix() {
*/
public static final class Builder {
private UUID id;
- private String ipAddress;
- private String hostName;
- private String networkName;
- private String networkLocation;
+ private StringWithByteString ipAddress;
+ private StringWithByteString hostName;
+ private StringWithByteString networkName;
+ private StringWithByteString networkLocation;
+ private int level;
private List ports;
private String certSerialId;
private String version;
@@ -612,10 +640,11 @@ private Builder() {
*/
public Builder setDatanodeDetails(DatanodeDetails details) {
this.id = details.getUuid();
- this.ipAddress = details.getIpAddress();
- this.hostName = details.getHostName();
- this.networkName = details.getNetworkName();
- this.networkLocation = details.getNetworkLocation();
+ this.ipAddress = details.getIpAddressAsByteString();
+ this.hostName = details.getHostNameAsByteString();
+ this.networkName = details.getHostNameAsByteString();
+ this.networkLocation = details.getNetworkLocationAsByteString();
+ this.level = details.getLevel();
this.ports = details.getPorts();
this.certSerialId = details.getCertSerialId();
this.version = details.getVersion();
@@ -646,7 +675,19 @@ public Builder setUuid(UUID uuid) {
* @return DatanodeDetails.Builder
*/
public Builder setIpAddress(String ip) {
- this.ipAddress = ip;
+ this.ipAddress = StringWithByteString.valueOf(ip);
+ return this;
+ }
+
+ /**
+ * Sets the IP address of DataNode.
+ *
+ * @param ip address
+ * @param ipBytes address in Bytes
+ * @return DatanodeDetails.Builder
+ */
+ public Builder setIpAddress(String ip, ByteString ipBytes) {
+ this.ipAddress = new StringWithByteString(ip, ipBytes);
return this;
}
@@ -657,7 +698,19 @@ public Builder setIpAddress(String ip) {
* @return DatanodeDetails.Builder
*/
public Builder setHostName(String host) {
- this.hostName = host;
+ this.hostName = StringWithByteString.valueOf(host);
+ return this;
+ }
+
+ /**
+ * Sets the hostname of DataNode.
+ *
+ * @param host hostname
+ * @param hostBytes hostname
+ * @return DatanodeDetails.Builder
+ */
+ public Builder setHostName(String host, ByteString hostBytes) {
+ this.hostName = new StringWithByteString(host, hostBytes);
return this;
}
@@ -665,10 +718,11 @@ public Builder setHostName(String host) {
* Sets the network name of DataNode.
*
* @param name network name
+ * @param nameBytes network name
* @return DatanodeDetails.Builder
*/
- public Builder setNetworkName(String name) {
- this.networkName = name;
+ public Builder setNetworkName(String name, ByteString nameBytes) {
+ this.networkName = new StringWithByteString(name, nameBytes);
return this;
}
@@ -679,7 +733,19 @@ public Builder setNetworkName(String name) {
* @return DatanodeDetails.Builder
*/
public Builder setNetworkLocation(String loc) {
- this.networkLocation = loc;
+ return setNetworkLocation(loc, null);
+ }
+
+ public Builder setNetworkLocation(String loc, ByteString locBytes) {
+ final String normalized = NetUtils.normalize(loc);
+ this.networkLocation = normalized.equals(loc) && locBytes != null
+ ? new StringWithByteString(normalized, locBytes)
+ : StringWithByteString.valueOf(normalized);
+ return this;
+ }
+
+ public Builder setLevel(int level) {
+ this.level = level;
return this;
}
@@ -797,17 +863,10 @@ public Builder setCurrentVersion(int v) {
*/
public DatanodeDetails build() {
Preconditions.checkNotNull(id);
- if (networkLocation == null) {
- networkLocation = NetConstants.DEFAULT_RACK;
+ if (networkLocation == null || networkLocation.getString().isEmpty()) {
+ networkLocation = NetConstants.BYTE_STRING_DEFAULT_RACK;
}
- DatanodeDetails dn = new DatanodeDetails(id, ipAddress, hostName,
- networkLocation, ports, certSerialId, version, setupTime, revision,
- buildDate, persistedOpState, persistedOpStateExpiryEpochSec,
- initialVersion, currentVersion);
- if (networkName != null) {
- dn.setNetworkName(networkName);
- }
- return dn;
+ return new DatanodeDetails(this);
}
}
@@ -854,9 +913,6 @@ public enum Name {
/**
* Private constructor for constructing Port object. Use
* DatanodeDetails#newPort to create a new Port object.
- *
- * @param name
- * @param value
*/
private Port(Name name, Integer value) {
this.name = name;
@@ -1011,4 +1067,12 @@ public String getBuildDate() {
public void setBuildDate(String date) {
this.buildDate = date;
}
+
+ @Override
+ public HddsProtos.NetworkNode toProtobuf(
+ int clientVersion) {
+ return HddsProtos.NetworkNode.newBuilder()
+ .setDatanodeDetails(toProtoBuilder(clientVersion).build())
+ .build();
+ }
}
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 cb7f6f8a3b31..5288c0bf50bb 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
@@ -61,6 +61,7 @@
import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.protocol.RoutingTable;
+import org.apache.ratis.retry.RetryPolicies;
import org.apache.ratis.retry.RetryPolicy;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
@@ -234,8 +235,8 @@ public static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline,
private static RpcType getRpcType(ConfigurationSource conf) {
return SupportedRpcType.valueOfIgnoreCase(conf.get(
- ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
- ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT));
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_RPC_TYPE_KEY,
+ ScmConfigKeys.HDDS_CONTAINER_RATIS_RPC_TYPE_DEFAULT));
}
public static BiFunction newRaftClient(
@@ -244,6 +245,12 @@ public static BiFunction newRaftClient(
RatisHelper.createRetryPolicy(conf), tlsConfig, conf);
}
+ public static BiFunction newRaftClientNoRetry(
+ ConfigurationSource conf) {
+ return (leader, tlsConfig) -> newRaftClient(getRpcType(conf), leader,
+ RetryPolicies.noRetry(), tlsConfig, conf);
+ }
+
public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
RetryPolicy retryPolicy, GrpcTlsConfig tlsConfig,
ConfigurationSource configuration) {
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 42a74dd12c2e..dbbfa9923e8a 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
@@ -41,95 +41,95 @@ public final class ScmConfigKeys {
public static final String OZONE_SCM_DB_DIRS_PERMISSIONS =
"ozone.scm.db.dirs.permissions";
- public static final String DFS_CONTAINER_RATIS_ENABLED_KEY
- = "dfs.container.ratis.enabled";
- public static final boolean DFS_CONTAINER_RATIS_ENABLED_DEFAULT
+ public static final String HDDS_CONTAINER_RATIS_ENABLED_KEY
+ = "hdds.container.ratis.enabled";
+ public static final boolean HDDS_CONTAINER_RATIS_ENABLED_DEFAULT
= false;
- public static final String DFS_CONTAINER_RATIS_RPC_TYPE_KEY
- = "dfs.container.ratis.rpc.type";
- public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
+ public static final String HDDS_CONTAINER_RATIS_RPC_TYPE_KEY
+ = "hdds.container.ratis.rpc.type";
+ public static final String HDDS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
= "GRPC";
public static final String
- DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME
- = "dfs.container.ratis.num.write.chunk.threads.per.volume";
+ HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME
+ = "hdds.container.ratis.num.write.chunk.threads.per.volume";
public static final int
- DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT
+ HDDS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT
= 10;
- public static final String DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY
- = "dfs.container.ratis.replication.level";
+ public static final String HDDS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY
+ = "hdds.container.ratis.replication.level";
public static final ReplicationLevel
- DFS_CONTAINER_RATIS_REPLICATION_LEVEL_DEFAULT = ReplicationLevel.MAJORITY;
- public static final String DFS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_KEY
- = "dfs.container.ratis.num.container.op.executors";
- public static final int DFS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_DEFAULT
+ HDDS_CONTAINER_RATIS_REPLICATION_LEVEL_DEFAULT = ReplicationLevel.MAJORITY;
+ public static final String HDDS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_KEY
+ = "hdds.container.ratis.num.container.op.executors";
+ public static final int HDDS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_DEFAULT
= 10;
- public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY =
- "dfs.container.ratis.segment.size";
- public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT =
+ public static final String HDDS_CONTAINER_RATIS_SEGMENT_SIZE_KEY =
+ "hdds.container.ratis.segment.size";
+ public static final String HDDS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT =
"64MB";
- public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY =
- "dfs.container.ratis.segment.preallocated.size";
+ public static final String HDDS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY =
+ "hdds.container.ratis.segment.preallocated.size";
public static final String
- DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT = "4MB";
+ HDDS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT = "4MB";
public static final String
- DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT =
- "dfs.container.ratis.statemachinedata.sync.timeout";
+ HDDS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT =
+ "hdds.container.ratis.statemachinedata.sync.timeout";
public static final TimeDuration
- DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT =
+ HDDS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT =
TimeDuration.valueOf(10, TimeUnit.SECONDS);
public static final String
- DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_RETRIES =
- "dfs.container.ratis.statemachinedata.sync.retries";
+ HDDS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_RETRIES =
+ "hdds.container.ratis.statemachinedata.sync.retries";
public static final String
- DFS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS =
- "dfs.container.ratis.statemachine.max.pending.apply-transactions";
+ HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS =
+ "hdds.container.ratis.statemachine.max.pending.apply-transactions";
// The default value of maximum number of pending state machine apply
// transactions is kept same as default snapshot threshold.
public static final int
- DFS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT =
+ HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT =
100000;
- public static final String DFS_CONTAINER_RATIS_LOG_QUEUE_NUM_ELEMENTS =
- "dfs.container.ratis.log.queue.num-elements";
- public static final int DFS_CONTAINER_RATIS_LOG_QUEUE_NUM_ELEMENTS_DEFAULT =
+ public static final String HDDS_CONTAINER_RATIS_LOG_QUEUE_NUM_ELEMENTS =
+ "hdds.container.ratis.log.queue.num-elements";
+ public static final int HDDS_CONTAINER_RATIS_LOG_QUEUE_NUM_ELEMENTS_DEFAULT =
1024;
- public static final String DFS_CONTAINER_RATIS_LOG_QUEUE_BYTE_LIMIT =
- "dfs.container.ratis.log.queue.byte-limit";
- public static final String DFS_CONTAINER_RATIS_LOG_QUEUE_BYTE_LIMIT_DEFAULT =
+ public static final String HDDS_CONTAINER_RATIS_LOG_QUEUE_BYTE_LIMIT =
+ "hdds.container.ratis.log.queue.byte-limit";
+ public static final String HDDS_CONTAINER_RATIS_LOG_QUEUE_BYTE_LIMIT_DEFAULT =
"4GB";
public static final String
- DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS =
- "dfs.container.ratis.log.appender.queue.num-elements";
+ HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS =
+ "hdds.container.ratis.log.appender.queue.num-elements";
public static final int
- DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT = 1;
- public static final String DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT =
- "dfs.container.ratis.log.appender.queue.byte-limit";
+ HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT = 1024;
+ public static final String HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT =
+ "hdds.container.ratis.log.appender.queue.byte-limit";
public static final String
- DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT = "32MB";
- public static final String DFS_CONTAINER_RATIS_LOG_PURGE_GAP =
- "dfs.container.ratis.log.purge.gap";
+ HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT = "32MB";
+ public static final String HDDS_CONTAINER_RATIS_LOG_PURGE_GAP =
+ "hdds.container.ratis.log.purge.gap";
// TODO: Set to 1024 once RATIS issue around purge is fixed.
- public static final int DFS_CONTAINER_RATIS_LOG_PURGE_GAP_DEFAULT =
+ public static final int HDDS_CONTAINER_RATIS_LOG_PURGE_GAP_DEFAULT =
1000000;
- public static final String DFS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT =
- "dfs.container.ratis.leader.pending.bytes.limit";
+ public static final String HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT =
+ "hdds.container.ratis.leader.pending.bytes.limit";
public static final String
- DFS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT_DEFAULT = "1GB";
+ HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT_DEFAULT = "1GB";
- public static final String DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY =
- "dfs.ratis.server.retry-cache.timeout.duration";
+ public static final String HDDS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY =
+ "hdds.ratis.server.retry-cache.timeout.duration";
public static final TimeDuration
- DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT =
+ HDDS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT =
TimeDuration.valueOf(600000, TimeUnit.MILLISECONDS);
public static final String
- DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY =
- "dfs.ratis.leader.election.minimum.timeout.duration";
+ HDDS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY =
+ "hdds.ratis.leader.election.minimum.timeout.duration";
public static final TimeDuration
- DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT =
+ HDDS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT =
TimeDuration.valueOf(5, TimeUnit.SECONDS);
- public static final String DFS_RATIS_SNAPSHOT_THRESHOLD_KEY =
- "dfs.ratis.snapshot.threshold";
- public static final long DFS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT = 100000;
+ public static final String HDDS_RATIS_SNAPSHOT_THRESHOLD_KEY =
+ "hdds.ratis.snapshot.threshold";
+ public static final long HDDS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT = 100000;
// TODO : this is copied from OzoneConsts, may need to move to a better place
public static final String OZONE_SCM_CHUNK_SIZE_KEY = "ozone.scm.chunk.size";
@@ -146,8 +146,8 @@ public final class ScmConfigKeys {
"32KB";
public static final String OZONE_CHUNK_LIST_INCREMENTAL =
- "ozone.chunk.list.incremental";
- public static final boolean OZONE_CHUNK_LIST_INCREMENTAL_DEFAULT = false;
+ "ozone.incremental.chunk.list";
+ public static final boolean OZONE_CHUNK_LIST_INCREMENTAL_DEFAULT = true;
public static final String OZONE_SCM_CONTAINER_LAYOUT_KEY =
"ozone.scm.container.layout";
@@ -227,17 +227,7 @@ public final class ScmConfigKeys {
"hdds.datanode.dir.du.reserved";
public static final String HDDS_DATANODE_DIR_DU_RESERVED_PERCENT =
"hdds.datanode.dir.du.reserved.percent";
- public static final float HDDS_DATANODE_DIR_DU_RESERVED_PERCENT_DEFAULT = 0;
- public static final String HDDS_REST_CSRF_ENABLED_KEY =
- "hdds.rest.rest-csrf.enabled";
- public static final boolean HDDS_REST_CSRF_ENABLED_DEFAULT = false;
- public static final String HDDS_REST_NETTY_HIGH_WATERMARK =
- "hdds.rest.netty.high.watermark";
- public static final int HDDS_REST_NETTY_HIGH_WATERMARK_DEFAULT = 65536;
- public static final int HDDS_REST_NETTY_LOW_WATERMARK_DEFAULT = 32768;
- public static final String HDDS_REST_NETTY_LOW_WATERMARK =
- "hdds.rest.netty.low.watermark";
-
+ public static final float HDDS_DATANODE_DIR_DU_RESERVED_PERCENT_DEFAULT = 0.0001f;
public static final String OZONE_SCM_HANDLER_COUNT_KEY =
"ozone.scm.handler.count.key";
public static final String OZONE_SCM_CLIENT_HANDLER_COUNT_KEY =
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java
index b9d823e8d817..19c39698dec7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
/**
@@ -26,9 +27,9 @@
* contains clusterId and the SCM Id.
*/
public final class ScmInfo {
- private String clusterId;
- private String scmId;
- private List peerRoles;
+ private final String clusterId;
+ private final String scmId;
+ private final List peerRoles;
/**
* Builder for ScmInfo.
@@ -36,7 +37,7 @@ public final class ScmInfo {
public static class Builder {
private String clusterId;
private String scmId;
- private List peerRoles;
+ private final List peerRoles;
public Builder() {
peerRoles = new ArrayList<>();
@@ -80,7 +81,7 @@ public ScmInfo build() {
private ScmInfo(String clusterId, String scmId, List peerRoles) {
this.clusterId = clusterId;
this.scmId = scmId;
- this.peerRoles = peerRoles;
+ this.peerRoles = Collections.unmodifiableList(peerRoles);
}
/**
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java
index b6834aba1eb1..61fd0d8f033d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto;
+import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
@@ -60,7 +61,7 @@ public void setLogIndex(long logIndex) {
}
public List getDatanodes() {
- return datanodes;
+ return Collections.unmodifiableList(datanodes);
}
public void addDatanode(DatanodeDetails dn) {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
index 402398e36c3f..14fb0a40cd00 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
@@ -217,11 +217,12 @@ List queryNode(HddsProtos.NodeOperationalState opState,
* Allows a list of hosts to be decommissioned. The hosts are identified
* by their hostname and optionally port in the format foo.com:port.
* @param hosts A list of hostnames, optionally with port
+ * @param force true to forcefully decommission Datanodes
* @throws IOException
* @return A list of DatanodeAdminError for any hosts which failed to
* decommission
*/
- List decommissionNodes(List hosts)
+ List decommissionNodes(List hosts, boolean force)
throws IOException;
/**
@@ -251,7 +252,7 @@ List recommissionNodes(List hosts)
* @throws IOException
*/
List startMaintenanceNodes(List hosts,
- int endHours) throws IOException;
+ int endHours, boolean force) throws IOException;
/**
* Creates a specified replication pipeline.
@@ -356,13 +357,20 @@ Map> getSafeModeRuleStatuses()
/**
* Start ContainerBalancer.
*/
+ @SuppressWarnings("checkstyle:parameternumber")
StartContainerBalancerResponseProto startContainerBalancer(
Optional threshold,
Optional iterations,
Optional maxDatanodesPercentageToInvolvePerIteration,
Optional maxSizeToMovePerIterationInGB,
Optional maxSizeEnteringTargetInGB,
- Optional maxSizeLeavingSourceInGB) throws IOException;
+ Optional maxSizeLeavingSourceInGB,
+ Optional balancingInterval,
+ Optional moveTimeout,
+ Optional moveReplicationTimeout,
+ Optional networkTopologyEnable,
+ Optional includeNodes,
+ Optional excludeNodes) throws IOException;
/**
* Stop ContainerBalancer.
@@ -452,4 +460,6 @@ StatusAndMessages queryUpgradeFinalizationProgress(
DecommissionScmResponseProto decommissionScm(
String scmId) throws IOException;
+
+ String getMetrics(String query) throws IOException;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
index b11428581e7b..6bf2d5500c88 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
@@ -90,31 +90,19 @@ public static Codec getCodec() {
// container replica should have the same sequenceId.
private long sequenceId;
- @SuppressWarnings("parameternumber")
- private ContainerInfo(
- long containerID,
- HddsProtos.LifeCycleState state,
- PipelineID pipelineID,
- long usedBytes,
- long numberOfKeys,
- long stateEnterTime,
- String owner,
- long deleteTransactionId,
- long sequenceId,
- ReplicationConfig repConfig,
- Clock clock) {
- this.containerID = ContainerID.valueOf(containerID);
- this.pipelineID = pipelineID;
- this.usedBytes = usedBytes;
- this.numberOfKeys = numberOfKeys;
- this.lastUsed = clock.instant();
- this.state = state;
- this.stateEnterTime = Instant.ofEpochMilli(stateEnterTime);
- this.owner = owner;
- this.deleteTransactionId = deleteTransactionId;
- this.sequenceId = sequenceId;
- this.replicationConfig = repConfig;
- this.clock = clock;
+ private ContainerInfo(Builder b) {
+ containerID = ContainerID.valueOf(b.containerID);
+ pipelineID = b.pipelineID;
+ usedBytes = b.used;
+ numberOfKeys = b.keys;
+ lastUsed = b.clock.instant();
+ state = b.state;
+ stateEnterTime = Instant.ofEpochMilli(b.stateEnterTime);
+ owner = b.owner;
+ deleteTransactionId = b.deleteTransactionId;
+ sequenceId = b.sequenceId;
+ replicationConfig = b.replicationConfig;
+ clock = b.clock;
}
public static ContainerInfo fromProtobuf(HddsProtos.ContainerInfoProto info) {
@@ -445,9 +433,7 @@ public Builder setClock(Clock clock) {
}
public ContainerInfo build() {
- return new ContainerInfo(containerID, state, pipelineID,
- used, keys, stateEnterTime, owner, deleteTransactionId,
- sequenceId, replicationConfig, clock);
+ return new ContainerInfo(this);
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
index 7ac0401af117..5a1d8f90ea84 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
@@ -26,8 +26,8 @@
* contains a Pipeline and the key.
*/
public final class AllocatedBlock {
- private Pipeline pipeline;
- private ContainerBlockID containerBlockID;
+ private final Pipeline pipeline;
+ private final ContainerBlockID containerBlockID;
/**
* Builder for AllocatedBlock.
@@ -63,4 +63,14 @@ public Pipeline getPipeline() {
public ContainerBlockID getBlockID() {
return containerBlockID;
}
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ public Builder toBuilder() {
+ return new Builder()
+ .setContainerBlockID(containerBlockID)
+ .setPipeline(pipeline);
+ }
}
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 2577a1e5ea2a..258c0be89612 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
@@ -25,6 +25,7 @@
import java.time.Clock;
import java.time.ZoneOffset;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
@@ -38,28 +39,24 @@
*/
public class ExcludeList {
- private final Map datanodes;
- private final Set containerIds;
- private final Set pipelineIds;
+ private final Map datanodes = new ConcurrentHashMap<>();
+ private final Set containerIds = new HashSet<>();
+ private final Set pipelineIds = new HashSet<>();
private long expiryTime = 0;
- private java.time.Clock clock;
+ private final Clock clock;
public ExcludeList() {
- datanodes = new ConcurrentHashMap<>();
- containerIds = new HashSet<>();
- pipelineIds = new HashSet<>();
clock = Clock.system(ZoneOffset.UTC);
}
- public ExcludeList(long autoExpiryTime, java.time.Clock clock) {
- this();
+ public ExcludeList(long autoExpiryTime, Clock clock) {
this.expiryTime = autoExpiryTime;
this.clock = clock;
}
public Set getContainerIds() {
- return containerIds;
+ return Collections.unmodifiableSet(containerIds);
}
public Set getDatanodes() {
@@ -99,7 +96,7 @@ public void addPipeline(PipelineID pipelineId) {
}
public Set getPipelineIds() {
- return pipelineIds;
+ return Collections.unmodifiableSet(pipelineIds);
}
public HddsProtos.ExcludeListProto getProtoBuf() {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java
index 80e09af172b6..af4e72993839 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java
@@ -65,7 +65,7 @@ public final class SCMHAUtils {
public static final Logger LOG =
LoggerFactory.getLogger(SCMHAUtils.class);
- private static final List>
+ private static final ImmutableList>
RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST =
ImmutableList.>builder()
.add(LeaderNotReadyException.class)
@@ -74,7 +74,7 @@ public final class SCMHAUtils {
.add(ResourceUnavailableException.class)
.build();
- private static final List>
+ private static final ImmutableList>
NON_RETRIABLE_EXCEPTION_LIST =
ImmutableList.>builder()
.add(SCMException.class)
@@ -316,7 +316,7 @@ public static Throwable getExceptionForClass(Exception e,
return null;
}
- public static List> getRetriableWithNoFailoverExceptionList() {
return RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNode.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNode.java
index c87d826d2529..6074e7da0afc 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNode.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNode.java
@@ -20,6 +20,8 @@
import java.util.Collection;
import java.util.List;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
/**
* The interface defines an inner node in a network topology.
* An inner node represents network topology entities, such as data center,
@@ -89,4 +91,16 @@ N newInnerNode(String name, String location, InnerNode parent, int level,
*/
Node getLeaf(int leafIndex, List excludedScopes,
Collection excludedNodes, int ancestorGen);
+
+ @Override
+ HddsProtos.NetworkNode toProtobuf(int clientVersion);
+
+ boolean equals(Object o);
+
+ int hashCode();
+
+ static InnerNode fromProtobuf(
+ HddsProtos.InnerNode innerNode) {
+ return InnerNodeImpl.fromProtobuf(innerNode);
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNodeImpl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNodeImpl.java
index f2648f3d294c..332dddac25c9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNodeImpl.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/InnerNodeImpl.java
@@ -27,6 +27,7 @@
import java.util.Map;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,10 +48,10 @@ public InnerNodeImpl newInnerNode(String name, String location,
}
}
- static final Factory FACTORY = new Factory();
+ public static final Factory FACTORY = new Factory();
// a map of node's network name to Node for quick search and keep
// the insert order
- private final HashMap childrenMap =
+ private HashMap childrenMap =
new LinkedHashMap();
// number of descendant leaves under this node
private int numOfLeaves;
@@ -66,6 +67,76 @@ protected InnerNodeImpl(String name, String location, InnerNode parent,
super(name, location, parent, level, cost);
}
+ /**
+ * Construct an InnerNode from its name, network location, level, cost,
+ * childrenMap and number of leaves. This constructor is used as part of
+ * protobuf deserialization.
+ */
+ protected InnerNodeImpl(String name, String location, int level, int cost,
+ HashMap childrenMap, int numOfLeaves) {
+ super(name, location, null, level, cost);
+ this.childrenMap = childrenMap;
+ this.numOfLeaves = numOfLeaves;
+ }
+
+ /**
+ * InnerNodeImpl Builder to help construct an InnerNodeImpl object from
+ * protobuf objects.
+ */
+ public static class Builder {
+ private String name;
+ private String location;
+ private int cost;
+ private int level;
+ private HashMap childrenMap = new LinkedHashMap<>();
+ private int numOfLeaves;
+
+ public Builder setName(String name) {
+ this.name = name;
+ return this;
+ }
+
+ public Builder setLocation(String location) {
+ this.location = location;
+ return this;
+ }
+
+ public Builder setCost(int cost) {
+ this.cost = cost;
+ return this;
+ }
+
+ public Builder setLevel(int level) {
+ this.level = level;
+ return this;
+ }
+
+ public Builder setChildrenMap(
+ List childrenMapList) {
+ HashMap newChildrenMap = new LinkedHashMap<>();
+ for (HddsProtos.ChildrenMap childrenMapProto :
+ childrenMapList) {
+ String networkName = childrenMapProto.hasNetworkName() ?
+ childrenMapProto.getNetworkName() : null;
+ Node node = childrenMapProto.hasNetworkNode() ?
+ Node.fromProtobuf(childrenMapProto.getNetworkNode()) : null;
+ newChildrenMap.put(networkName, node);
+ }
+ this.childrenMap = newChildrenMap;
+ return this;
+ }
+
+ public Builder setNumOfLeaves(int numOfLeaves) {
+ this.numOfLeaves = numOfLeaves;
+ return this;
+ }
+
+ public InnerNodeImpl build() {
+ return new InnerNodeImpl(name, location, level, cost, childrenMap,
+ numOfLeaves);
+ }
+ }
+
/** @return the number of children this node has */
private int getNumOfChildren() {
return childrenMap.size();
@@ -77,6 +148,11 @@ public int getNumOfLeaves() {
return numOfLeaves;
}
+ /** @return a map of node's network name to Node. */
+ public HashMap getChildrenMap() {
+ return childrenMap;
+ }
+
/**
* @return number of its all nodes at level level. Here level is a
* relative level. If level is 1, means node itself. If level is 2, means its
@@ -390,14 +466,83 @@ public Node getLeaf(int leafIndex, List excludedScopes,
}
@Override
- public boolean equals(Object to) {
- if (to == null) {
- return false;
+ public HddsProtos.NetworkNode toProtobuf(
+ int clientVersion) {
+
+ HddsProtos.InnerNode.Builder innerNode =
+ HddsProtos.InnerNode.newBuilder()
+ .setNumOfLeaves(numOfLeaves)
+ .setNodeTopology(
+ NodeImpl.toProtobuf(getNetworkName(), getNetworkLocation(),
+ getLevel(), getCost()));
+
+ if (childrenMap != null && !childrenMap.isEmpty()) {
+ for (Map.Entry entry : childrenMap.entrySet()) {
+ if (entry.getValue() != null) {
+ HddsProtos.ChildrenMap childrenMapProto =
+ HddsProtos.ChildrenMap.newBuilder()
+ .setNetworkName(entry.getKey())
+ .setNetworkNode(entry.getValue().toProtobuf(clientVersion))
+ .build();
+ innerNode.addChildrenMap(childrenMapProto);
+ }
+ }
+ }
+ innerNode.build();
+
+ HddsProtos.NetworkNode networkNode =
+ HddsProtos.NetworkNode.newBuilder()
+ .setInnerNode(innerNode).build();
+
+ return networkNode;
+ }
+
+ public static InnerNode fromProtobuf(HddsProtos.InnerNode innerNode) {
+ InnerNodeImpl.Builder builder = new InnerNodeImpl.Builder();
+
+ if (innerNode.hasNodeTopology()) {
+ HddsProtos.NodeTopology nodeTopology = innerNode.getNodeTopology();
+
+ if (nodeTopology.hasName()) {
+ builder.setName(nodeTopology.getName());
+ }
+ if (nodeTopology.hasLocation()) {
+ builder.setLocation(nodeTopology.getLocation());
+ }
+ if (nodeTopology.hasLevel()) {
+ builder.setLevel(nodeTopology.getLevel());
+ }
+ if (nodeTopology.hasCost()) {
+ builder.setCost(nodeTopology.getCost());
+ }
+ }
+
+ if (!innerNode.getChildrenMapList().isEmpty()) {
+ builder.setChildrenMap(innerNode.getChildrenMapList());
+ }
+ if (innerNode.hasNumOfLeaves()) {
+ builder.setNumOfLeaves(innerNode.getNumOfLeaves());
}
- if (this == to) {
+
+ return builder.build();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
return true;
}
- return this.toString().equals(to.toString());
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ InnerNodeImpl innerNode = (InnerNodeImpl) o;
+ return this.getNetworkName().equals(innerNode.getNetworkName()) &&
+ this.getNetworkLocation().equals(innerNode.getNetworkLocation()) &&
+ this.getLevel() == innerNode.getLevel() &&
+ this.getCost() == innerNode.getCost() &&
+ this.numOfLeaves == innerNode.numOfLeaves &&
+ this.childrenMap.size() == innerNode.childrenMap.size() &&
+ this.childrenMap.equals(innerNode.childrenMap);
}
@Override
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetConstants.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetConstants.java
index 8ee6decc9c4d..bd1aa71ebd72 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetConstants.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetConstants.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.net;
import org.apache.hadoop.hdds.scm.net.NodeSchema.LayerType;
+import org.apache.hadoop.util.StringWithByteString;
/**
* Class to hold network topology related constants and configurations.
@@ -32,11 +33,13 @@ private NetConstants() {
public static final String SCOPE_REVERSE_STR = "~";
/** string representation of root. */
public static final String ROOT = "";
+ public static final StringWithByteString BYTE_STRING_ROOT = StringWithByteString.valueOf(ROOT);
public static final int INNER_NODE_COST_DEFAULT = 1;
public static final int NODE_COST_DEFAULT = 0;
public static final int ANCESTOR_GENERATION_DEFAULT = 0;
public static final int ROOT_LEVEL = 1;
public static final String DEFAULT_RACK = "/default-rack";
+ public static final StringWithByteString BYTE_STRING_DEFAULT_RACK = StringWithByteString.valueOf(DEFAULT_RACK);
public static final String DEFAULT_NODEGROUP = "/default-nodegroup";
public static final String DEFAULT_DATACENTER = "/default-datacenter";
public static final String DEFAULT_REGION = "/default-dataregion";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java
index 2dc86c1b6856..1f3d0f02e6de 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java
@@ -30,6 +30,7 @@
import java.util.LinkedHashSet;
import java.util.List;
import java.util.NavigableMap;
+import java.util.Objects;
import java.util.TreeMap;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.locks.ReadWriteLock;
@@ -75,6 +76,15 @@ public NetworkTopologyImpl(ConfigurationSource conf) {
schemaManager.getCost(NetConstants.ROOT_LEVEL));
}
+ public NetworkTopologyImpl(String schemaFile, InnerNode clusterTree) {
+ schemaManager = NodeSchemaManager.getInstance();
+ schemaManager.init(schemaFile);
+ maxLevel = schemaManager.getMaxLevel();
+ shuffleOperation = Collections::shuffle;
+ factory = InnerNodeImpl.FACTORY;
+ this.clusterTree = clusterTree;
+ }
+
@VisibleForTesting
public NetworkTopologyImpl(NodeSchemaManager manager,
Consumer> shuffleOperation) {
@@ -223,10 +233,10 @@ public boolean contains(Node node) {
private boolean containsNode(Node node) {
Node parent = node.getParent();
- while (parent != null && parent != clusterTree) {
+ while (parent != null && !Objects.equals(parent, clusterTree)) {
parent = parent.getParent();
}
- return parent == clusterTree;
+ return Objects.equals(parent, clusterTree);
}
/**
@@ -240,7 +250,9 @@ public boolean isSameAncestor(Node node1, Node node2, int ancestorGen) {
}
netlock.readLock().lock();
try {
- return node1.getAncestor(ancestorGen) == node2.getAncestor(ancestorGen);
+ Node ancestor1 = node1.getAncestor(ancestorGen);
+ Node ancestor2 = node2.getAncestor(ancestorGen);
+ return Objects.equals(ancestor1, ancestor2);
} finally {
netlock.readLock().unlock();
}
@@ -259,7 +271,7 @@ public boolean isSameParent(Node node1, Node node2) {
try {
node1 = node1.getParent();
node2 = node2.getParent();
- return node1 == node2;
+ return Objects.equals(node1, node2);
} finally {
netlock.readLock().unlock();
}
@@ -704,8 +716,7 @@ private Node chooseNodeInternal(String scope, int leafIndex,
*/
@Override
public int getDistanceCost(Node node1, Node node2) {
- if ((node1 != null && node1.equals(node2)) ||
- (node1 == null && node2 == null)) {
+ if (Objects.equals(node1, node2)) {
return 0;
}
if (node1 == null || node2 == null) {
@@ -726,8 +737,10 @@ public int getDistanceCost(Node node1, Node node2) {
int cost = 0;
netlock.readLock().lock();
try {
- if ((node1.getAncestor(level1 - 1) != clusterTree) ||
- (node2.getAncestor(level2 - 1) != clusterTree)) {
+ Node ancestor1 = node1.getAncestor(level1 - 1);
+ Node ancestor2 = node2.getAncestor(level2 - 1);
+ if (!Objects.equals(ancestor1, clusterTree) ||
+ !Objects.equals(ancestor2, clusterTree)) {
LOG.debug("One of the nodes is outside of network topology");
return Integer.MAX_VALUE;
}
@@ -741,7 +754,7 @@ public int getDistanceCost(Node node1, Node node2) {
level2--;
cost += node2 == null ? 0 : node2.getCost();
}
- while (node1 != null && node2 != null && node1 != node2) {
+ while (node1 != null && node2 != null && !Objects.equals(node1, node2)) {
node1 = node1.getParent();
node2 = node2.getParent();
cost += node1 == null ? 0 : node1.getCost();
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/Node.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/Node.java
index 9884888a1dd4..50f702cce08e 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/Node.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/Node.java
@@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hdds.scm.net;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
/**
* The interface defines a node in a network topology.
* A node may be a leave representing a data node or an inner
@@ -126,4 +129,21 @@ public interface Node {
* @return true if this node is under a specific scope
*/
boolean isDescendant(String nodePath);
+
+ default HddsProtos.NetworkNode toProtobuf(
+ int clientVersion) {
+ return null;
+ }
+
+ static Node fromProtobuf(
+ HddsProtos.NetworkNode networkNode) {
+ if (networkNode.hasDatanodeDetails()) {
+ return DatanodeDetails.getFromProtoBuf(
+ networkNode.getDatanodeDetails());
+ } else if (networkNode.hasInnerNode()) {
+ return InnerNode.fromProtobuf(networkNode.getInnerNode());
+ } else {
+ return null;
+ }
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeImpl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeImpl.java
index e7a45f649b6e..f5f6cec099b2 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeImpl.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeImpl.java
@@ -18,8 +18,10 @@
package org.apache.hadoop.hdds.scm.net;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.util.StringWithByteString;
-import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.BYTE_STRING_ROOT;
import static org.apache.hadoop.hdds.scm.net.NetConstants.PATH_SEPARATOR_STR;
/**
@@ -27,9 +29,9 @@
*/
public class NodeImpl implements Node {
// host:port#
- private String name;
+ private StringWithByteString name;
// string representation of this node's location, such as /dc1/rack1
- private String location;
+ private StringWithByteString location;
// location + "/" + name
private String path;
// which level of the tree the node resides, start from 1 for root
@@ -45,18 +47,22 @@ public class NodeImpl implements Node {
* {@link NetConstants#PATH_SEPARATOR})
* @param location this node's location
*/
- public NodeImpl(String name, String location, int cost) {
- if (name != null && name.contains(PATH_SEPARATOR_STR)) {
+ public NodeImpl(StringWithByteString name, StringWithByteString location, int cost) {
+ if (name != null && name.getString().contains(PATH_SEPARATOR_STR)) {
throw new IllegalArgumentException(
"Network location name:" + name + " should not contain " +
PATH_SEPARATOR_STR);
}
- this.name = (name == null) ? ROOT : name;
- this.location = NetUtils.normalize(location);
+ this.name = name == null ? BYTE_STRING_ROOT : name;
+ this.location = location;
this.path = getPath();
this.cost = cost;
}
+ public NodeImpl(String name, String location, int cost) {
+ this(StringWithByteString.valueOf(name), StringWithByteString.valueOf(NetUtils.normalize(location)), cost);
+ }
+
/**
* Construct a node from its name and its location.
*
@@ -74,11 +80,25 @@ public NodeImpl(String name, String location, InnerNode parent, int level,
this.level = level;
}
+ public NodeImpl(StringWithByteString name, StringWithByteString location, InnerNode parent, int level,
+ int cost) {
+ this(name, location, cost);
+ this.parent = parent;
+ this.level = level;
+ }
+
/**
* @return this node's name
*/
@Override
public String getNetworkName() {
+ return name.getString();
+ }
+
+ /**
+ * @return this node's name
+ */
+ public StringWithByteString getNetworkNameAsByteString() {
return name;
}
@@ -88,6 +108,15 @@ public String getNetworkName() {
*/
@Override
public void setNetworkName(String networkName) {
+ this.name = StringWithByteString.valueOf(networkName);
+ this.path = getPath();
+ }
+
+ /**
+ * Set this node's name, can be hostname or Ipaddress.
+ * @param networkName it's network name
+ */
+ public void setNetworkName(StringWithByteString networkName) {
this.name = networkName;
this.path = getPath();
}
@@ -97,6 +126,13 @@ public void setNetworkName(String networkName) {
*/
@Override
public String getNetworkLocation() {
+ return location.getString();
+ }
+
+ /**
+ * @return this node's network location
+ */
+ public StringWithByteString getNetworkLocationAsByteString() {
return location;
}
@@ -106,7 +142,7 @@ public String getNetworkLocation() {
*/
@Override
public void setNetworkLocation(String networkLocation) {
- this.location = networkLocation;
+ this.location = StringWithByteString.valueOf(networkLocation);
this.path = getPath();
}
@@ -229,6 +265,20 @@ public boolean isDescendant(String nodePath) {
NetUtils.addSuffix(nodePath));
}
+ public static HddsProtos.NodeTopology toProtobuf(String name, String location,
+ int level, int cost) {
+
+ HddsProtos.NodeTopology.Builder nodeTopologyBuilder =
+ HddsProtos.NodeTopology.newBuilder()
+ .setName(name)
+ .setLocation(location)
+ .setLevel(level)
+ .setCost(cost);
+
+ HddsProtos.NodeTopology nodeTopology = nodeTopologyBuilder.build();
+ return nodeTopology;
+ }
+
@Override
public boolean equals(Object to) {
if (to == null) {
@@ -254,8 +304,8 @@ public String toString() {
}
private String getPath() {
- return this.location.equals(PATH_SEPARATOR_STR) ?
- this.location + this.name :
+ return this.location.getString().equals(PATH_SEPARATOR_STR) ?
+ this.location + this.name.getString() :
this.location + PATH_SEPARATOR_STR + this.name;
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java
index eecd79876720..fb37b214cad1 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NodeSchemaManager.java
@@ -62,6 +62,14 @@ public void init(ConfigurationSource conf) {
String schemaFile = conf.get(
ScmConfigKeys.OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE,
ScmConfigKeys.OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE_DEFAULT);
+ loadSchemaFile(schemaFile);
+ }
+
+ public void init(String schemaFile) {
+ loadSchemaFile(schemaFile);
+ }
+
+ private void loadSchemaFile(String schemaFile) {
NodeSchemaLoadResult result;
try {
result = NodeSchemaLoader.getInstance().loadSchemaFromFile(schemaFile);
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
index 9d95cee48366..6ea92f74c193 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
@@ -23,7 +23,6 @@
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
@@ -34,6 +33,8 @@
import java.util.UUID;
import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
@@ -76,10 +77,10 @@ public static Codec getCodec() {
private final ReplicationConfig replicationConfig;
private final PipelineState state;
- private Map nodeStatus;
- private Map replicaIndexes;
+ private final Map nodeStatus;
+ private final Map replicaIndexes;
// nodes with ordered distance to client
- private List nodesInOrder = new ArrayList<>();
+ private final ImmutableList nodesInOrder;
// Current reported Leader for the pipeline
private UUID leaderId;
// Timestamp for pipeline upon creation
@@ -103,17 +104,17 @@ public static Codec getCodec() {
* set to Instant.now when you crate the Pipeline object as part of
* state change.
*/
- private Pipeline(PipelineID id,
- ReplicationConfig replicationConfig, PipelineState state,
- Map nodeStatus, UUID suggestedLeaderId) {
- this.id = id;
- this.replicationConfig = replicationConfig;
- this.state = state;
- this.nodeStatus = nodeStatus;
- this.creationTimestamp = Instant.now();
- this.suggestedLeaderId = suggestedLeaderId;
- this.replicaIndexes = new HashMap<>();
- this.stateEnterTime = Instant.now();
+ private Pipeline(Builder b) {
+ id = b.id;
+ replicationConfig = b.replicationConfig;
+ state = b.state;
+ leaderId = b.leaderId;
+ suggestedLeaderId = b.suggestedLeaderId;
+ nodeStatus = b.nodeStatus;
+ nodesInOrder = b.nodesInOrder != null ? ImmutableList.copyOf(b.nodesInOrder) : ImmutableList.of();
+ replicaIndexes = b.replicaIndexes;
+ creationTimestamp = b.creationTimestamp != null ? b.creationTimestamp : Instant.now();
+ stateEnterTime = Instant.now();
}
/**
@@ -310,19 +311,6 @@ public boolean isOpen() {
return state == PipelineState.OPEN;
}
- public boolean isAllocationTimeout() {
- //TODO: define a system property to control the timeout value
- return false;
- }
-
- public void setNodesInOrder(List nodes) {
- nodesInOrder.clear();
- if (null == nodes) {
- return;
- }
- nodesInOrder.addAll(nodes);
- }
-
public List getNodesInOrder() {
if (nodesInOrder.isEmpty()) {
LOG.debug("Nodes in order is empty, delegate to getNodes");
@@ -406,33 +394,39 @@ public HddsProtos.Pipeline getProtobufMessage(int clientVersion)
// To save the message size on wire, only transfer the node order based on
// network topology
- List nodes = nodesInOrder;
- if (!nodes.isEmpty()) {
- for (int i = 0; i < nodes.size(); i++) {
+ if (!nodesInOrder.isEmpty()) {
+ for (int i = 0; i < nodesInOrder.size(); i++) {
Iterator it = nodeStatus.keySet().iterator();
for (int j = 0; j < nodeStatus.keySet().size(); j++) {
- if (it.next().equals(nodes.get(i))) {
+ if (it.next().equals(nodesInOrder.get(i))) {
builder.addMemberOrders(j);
break;
}
}
}
if (LOG.isDebugEnabled()) {
- LOG.debug("Serialize pipeline {} with nodesInOrder {}", id, nodes);
+ LOG.debug("Serialize pipeline {} with nodesInOrder {}", id, nodesInOrder);
}
}
return builder.build();
}
- static Pipeline getFromProtobufSetCreationTimestamp(
+ private static Pipeline getFromProtobufSetCreationTimestamp(
HddsProtos.Pipeline proto) throws UnknownPipelineStateException {
- final Pipeline pipeline = getFromProtobuf(proto);
- // When SCM is restarted, set Creation time with current time.
- pipeline.setCreationTimestamp(Instant.now());
- return pipeline;
+ return toBuilder(proto)
+ .setCreateTimestamp(Instant.now())
+ .build();
}
- public static Pipeline getFromProtobuf(HddsProtos.Pipeline pipeline)
+ public Pipeline copyWithNodesInOrder(List nodes) {
+ return toBuilder().setNodesInOrder(nodes).build();
+ }
+
+ public Builder toBuilder() {
+ return newBuilder(this);
+ }
+
+ public static Builder toBuilder(HddsProtos.Pipeline pipeline)
throws UnknownPipelineStateException {
Preconditions.checkNotNull(pipeline, "Pipeline is null");
@@ -473,9 +467,13 @@ public static Pipeline getFromProtobuf(HddsProtos.Pipeline pipeline)
.setReplicaIndexes(nodes)
.setLeaderId(leaderId)
.setSuggestedLeaderId(suggestedLeaderId)
- .setNodesInOrder(pipeline.getMemberOrdersList())
- .setCreateTimestamp(pipeline.getCreationTimeStamp())
- .build();
+ .setNodeOrder(pipeline.getMemberOrdersList())
+ .setCreateTimestamp(pipeline.getCreationTimeStamp());
+ }
+
+ public static Pipeline getFromProtobuf(HddsProtos.Pipeline pipeline)
+ throws UnknownPipelineStateException {
+ return toBuilder(pipeline).build();
}
@Override
@@ -529,10 +527,6 @@ public static Builder newBuilder(Pipeline pipeline) {
return new Builder(pipeline);
}
- private void setReplicaIndexes(Map replicaIndexes) {
- this.replicaIndexes = replicaIndexes;
- }
-
/**
* Builder class for Pipeline.
*/
@@ -546,7 +540,7 @@ public static class Builder {
private UUID leaderId = null;
private Instant creationTimestamp = null;
private UUID suggestedLeaderId = null;
- private Map replicaIndexes = new HashMap<>();
+ private Map replicaIndexes = ImmutableMap.of();
public Builder() { }
@@ -559,14 +553,15 @@ public Builder(Pipeline pipeline) {
this.leaderId = pipeline.getLeaderId();
this.creationTimestamp = pipeline.getCreationTimestamp();
this.suggestedLeaderId = pipeline.getSuggestedLeaderId();
- this.replicaIndexes = new HashMap<>();
if (nodeStatus != null) {
+ final ImmutableMap.Builder b = ImmutableMap.builder();
for (DatanodeDetails dn : nodeStatus.keySet()) {
int index = pipeline.getReplicaIndex(dn);
if (index > 0) {
- replicaIndexes.put(dn, index);
+ b.put(dn, index);
}
}
+ replicaIndexes = b.build();
}
}
@@ -601,8 +596,19 @@ public Builder setNodes(List nodes) {
return this;
}
- public Builder setNodesInOrder(List orders) {
- this.nodeOrder = orders;
+ public Builder setNodeOrder(List orders) {
+ // for build from ProtoBuf
+ this.nodeOrder = Collections.unmodifiableList(orders);
+ return this;
+ }
+
+ public Builder setNodesInOrder(List nodes) {
+ this.nodesInOrder = new LinkedList<>(nodes);
+ return this;
+ }
+
+ public Builder setCreateTimestamp(Instant instant) {
+ this.creationTimestamp = instant;
return this;
}
@@ -618,7 +624,7 @@ public Builder setSuggestedLeaderId(UUID uuid) {
public Builder setReplicaIndexes(Map indexes) {
- this.replicaIndexes = indexes;
+ this.replicaIndexes = indexes == null ? ImmutableMap.of() : ImmutableMap.copyOf(indexes);
return this;
}
@@ -627,19 +633,8 @@ public Pipeline build() {
Preconditions.checkNotNull(replicationConfig);
Preconditions.checkNotNull(state);
Preconditions.checkNotNull(nodeStatus);
- Pipeline pipeline =
- new Pipeline(id, replicationConfig, state, nodeStatus,
- suggestedLeaderId);
- pipeline.setLeaderId(leaderId);
- // overwrite with original creationTimestamp
- if (creationTimestamp != null) {
- pipeline.setCreationTimestamp(creationTimestamp);
- }
-
- pipeline.setReplicaIndexes(replicaIndexes);
if (nodeOrder != null && !nodeOrder.isEmpty()) {
- // This branch is for build from ProtoBuf
List nodesWithOrder = new ArrayList<>();
for (int i = 0; i < nodeOrder.size(); i++) {
int nodeIndex = nodeOrder.get(i);
@@ -657,13 +652,10 @@ public Pipeline build() {
LOG.debug("Deserialize nodesInOrder {} in pipeline {}",
nodesWithOrder, id);
}
- pipeline.setNodesInOrder(nodesWithOrder);
- } else if (nodesInOrder != null) {
- // This branch is for pipeline clone
- pipeline.setNodesInOrder(nodesInOrder);
+ nodesInOrder = nodesWithOrder;
}
- return pipeline;
+ return new Pipeline(this);
}
}
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 e8bddb42cfbd..df8ed02cf7f0 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
@@ -245,14 +245,14 @@ List queryNode(HddsProtos.NodeOperationalState opState,
HddsProtos.Node queryNode(UUID uuid) throws IOException;
- List decommissionNodes(List nodes)
+ List decommissionNodes(List nodes, boolean force)
throws IOException;
List recommissionNodes(List nodes)
throws IOException;
List startMaintenanceNodes(List nodes,
- int endInHours) throws IOException;
+ int endInHours, boolean force) throws IOException;
/**
* Close a container.
@@ -402,13 +402,20 @@ Map> getSafeModeRuleStatuses()
* @return {@link StartContainerBalancerResponseProto} that contains the
* start status and an optional message.
*/
+ @SuppressWarnings("checkstyle:parameternumber")
StartContainerBalancerResponseProto startContainerBalancer(
Optional threshold,
Optional iterations,
Optional maxDatanodesPercentageToInvolvePerIteration,
Optional maxSizeToMovePerIterationInGB,
Optional maxSizeEnteringTargetInGB,
- Optional maxSizeLeavingSourceInGB) throws IOException;
+ Optional maxSizeLeavingSourceInGB,
+ Optional balancingInterval,
+ Optional moveTimeout,
+ Optional moveReplicationTimeout,
+ Optional networkTopologyEnable,
+ Optional includeNodes,
+ Optional excludeNodes) throws IOException;
/**
* Stop ContainerBalancer.
@@ -474,4 +481,6 @@ List getListOfContainers(
DecommissionScmResponseProto decommissionScm(
String scmId) throws IOException;
+
+ String getMetrics(String query) throws IOException;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java
index 9acb0e5c33a7..d3f39c023b73 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java
@@ -21,6 +21,8 @@
import java.nio.ByteBuffer;
import java.util.List;
import java.util.function.Function;
+
+import org.apache.commons.lang3.RandomUtils;
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;
@@ -38,10 +40,12 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ListBlockResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
import org.apache.hadoop.ozone.common.ChunkBuffer;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;
import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion;
@@ -210,6 +214,28 @@ public static ContainerCommandResponseProto getPutFileResponseSuccess(
.build();
}
+ /**
+ * Gets a response for the WriteChunk RPC.
+ * @param msg - ContainerCommandRequestProto
+ * @return - ContainerCommandResponseProto
+ */
+ public static ContainerCommandResponseProto getWriteChunkResponseSuccess(
+ ContainerCommandRequestProto msg, BlockData blockData) {
+
+ WriteChunkResponseProto.Builder writeChunk =
+ WriteChunkResponseProto.newBuilder();
+ if (blockData != null) {
+ writeChunk.setCommittedBlockLength(
+ getCommittedBlockLengthResponseBuilder(
+ blockData.getSize(), blockData.getBlockID()));
+
+ }
+ return getSuccessResponseBuilder(msg)
+ .setCmdType(Type.WriteChunk)
+ .setWriteChunk(writeChunk)
+ .build();
+ }
+
/**
* Gets a response to the read small file call.
* @param request - Msg
@@ -319,6 +345,31 @@ public static ContainerCommandResponseProto getFinalizeBlockResponse(
.build();
}
+ public static ContainerCommandResponseProto getEchoResponse(
+ ContainerCommandRequestProto msg) {
+
+ ContainerProtos.EchoRequestProto echoRequest = msg.getEcho();
+ int responsePayload = echoRequest.getPayloadSizeResp();
+
+ int sleepTimeMs = echoRequest.getSleepTimeMs();
+ try {
+ if (sleepTimeMs > 0) {
+ Thread.sleep(sleepTimeMs);
+ }
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+
+ ContainerProtos.EchoResponseProto.Builder echo =
+ ContainerProtos.EchoResponseProto
+ .newBuilder()
+ .setPayload(UnsafeByteOperations.unsafeWrap(RandomUtils.nextBytes(responsePayload)));
+
+ return getSuccessResponseBuilder(msg)
+ .setEcho(echo)
+ .build();
+ }
+
private ContainerCommandResponseBuilders() {
throw new UnsupportedOperationException("no instances");
}
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 c85405566ca5..37cc075f2195 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
@@ -29,6 +29,9 @@
import java.util.concurrent.ExecutionException;
import java.util.function.Function;
+import io.opentracing.Scope;
+import io.opentracing.Span;
+import io.opentracing.util.GlobalTracer;
import org.apache.hadoop.hdds.annotation.InterfaceStability;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -57,6 +60,8 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.FinalizeBlockRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.EchoRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.EchoResponseProto;
import org.apache.hadoop.hdds.scm.XceiverClientReply;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator;
@@ -65,6 +70,7 @@
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.hdds.tracing.TracingUtil;
import org.apache.hadoop.ozone.common.Checksum;
import org.apache.hadoop.ozone.common.ChecksumData;
import org.apache.hadoop.security.token.Token;
@@ -76,6 +82,7 @@
import org.slf4j.LoggerFactory;
import static java.util.Collections.singletonList;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED;
/**
* Implementation of all container protocol calls performed by Container
@@ -128,6 +135,10 @@ public static ListBlockResponseProto listBlock(XceiverClientSpi xceiverClient,
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
ContainerCommandRequestProto request = builder.build();
ContainerCommandResponseProto response =
@@ -146,6 +157,17 @@ static T tryEachDatanode(Pipeline pipeline,
try {
return op.apply(d);
} catch (IOException e) {
+ Span span = GlobalTracer.get().activeSpan();
+ if (e instanceof StorageContainerException) {
+ StorageContainerException sce = (StorageContainerException)e;
+ // Block token expired. There's no point retrying other DN.
+ // Throw the exception to request a new block token right away.
+ if (sce.getResult() == BLOCK_TOKEN_VERIFICATION_FAILED) {
+ span.log("block token verification failed at DN " + d);
+ throw e;
+ }
+ }
+ span.log("failed to connect to DN " + d);
excluded.add(d);
if (excluded.size() < pipeline.size()) {
LOG.warn(toErrorMessage.apply(d)
@@ -203,6 +225,10 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
List validators,
ContainerCommandRequestProto.Builder builder,
DatanodeDetails datanode) throws IOException {
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
final ContainerCommandRequestProto request = builder
.setDatanodeUuid(datanode.getUuidString()).build();
ContainerCommandResponseProto response =
@@ -238,6 +264,10 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
ContainerCommandRequestProto request = builder.build();
ContainerCommandResponseProto response =
xceiverClient.sendCommand(request, getValidatorList());
@@ -341,10 +371,19 @@ public static ContainerProtos.ReadChunkResponseProto readChunk(
builder.setEncodedToken(token.encodeToUrlString());
}
- return tryEachDatanode(xceiverClient.getPipeline(),
- d -> readChunk(xceiverClient, chunk, blockID,
- validators, builder, d),
- d -> toErrorMessage(chunk, blockID, d));
+ Span span = GlobalTracer.get()
+ .buildSpan("readChunk").start();
+ try (Scope ignored = GlobalTracer.get().activateSpan(span)) {
+ span.setTag("offset", chunk.getOffset())
+ .setTag("length", chunk.getLen())
+ .setTag("block", blockID.toString());
+ return tryEachDatanode(xceiverClient.getPipeline(),
+ d -> readChunk(xceiverClient, chunk, blockID,
+ validators, builder, d),
+ d -> toErrorMessage(chunk, blockID, d));
+ } finally {
+ span.finish();
+ }
}
private static ContainerProtos.ReadChunkResponseProto readChunk(
@@ -352,10 +391,15 @@ private static ContainerProtos.ReadChunkResponseProto readChunk(
List validators,
ContainerCommandRequestProto.Builder builder,
DatanodeDetails d) throws IOException {
- final ContainerCommandRequestProto request = builder
- .setDatanodeUuid(d.getUuidString()).build();
+ ContainerCommandRequestProto.Builder requestBuilder = builder
+ .setDatanodeUuid(d.getUuidString());
+ Span span = GlobalTracer.get().activeSpan();
+ String traceId = TracingUtil.exportSpan(span);
+ if (traceId != null) {
+ requestBuilder = requestBuilder.setTraceID(traceId);
+ }
ContainerCommandResponseProto reply =
- xceiverClient.sendCommand(request, validators);
+ xceiverClient.sendCommand(requestBuilder.build(), validators);
final ReadChunkResponseProto response = reply.getReadChunk();
final long readLen = getLen(response);
if (readLen != chunk.getLen()) {
@@ -394,8 +438,10 @@ static long getLen(ReadChunkResponseProto response) {
*/
public static XceiverClientReply writeChunkAsync(
XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID,
- ByteString data, String tokenString, int replicationIndex)
+ ByteString data, String tokenString,
+ int replicationIndex, BlockData blockData)
throws IOException, ExecutionException, InterruptedException {
+
WriteChunkRequestProto.Builder writeChunkRequest =
WriteChunkRequestProto.newBuilder()
.setBlockID(DatanodeBlockID.newBuilder()
@@ -406,6 +452,12 @@ public static XceiverClientReply writeChunkAsync(
.build())
.setChunkData(chunk)
.setData(data);
+ if (blockData != null) {
+ PutBlockRequestProto.Builder createBlockRequest =
+ PutBlockRequestProto.newBuilder()
+ .setBlockData(blockData);
+ writeChunkRequest.setBlock(createBlockRequest);
+ }
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder builder =
ContainerCommandRequestProto.newBuilder()
@@ -537,6 +589,11 @@ public static void createContainer(XceiverClientSpi client,
if (encodedToken != null) {
request.setEncodedToken(encodedToken);
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ request.setTraceID(traceId);
+ }
+
request.setCmdType(ContainerProtos.Type.CreateContainer);
request.setContainerID(containerID);
request.setCreateContainer(createRequest.build());
@@ -566,6 +623,10 @@ public static void deleteContainer(XceiverClientSpi client, long containerID,
if (encodedToken != null) {
request.setEncodedToken(encodedToken);
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ request.setTraceID(traceId);
+ }
client.sendCommand(request.build(), getValidatorList());
}
@@ -588,6 +649,10 @@ public static void closeContainer(XceiverClientSpi client,
if (encodedToken != null) {
request.setEncodedToken(encodedToken);
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ request.setTraceID(traceId);
+ }
client.sendCommand(request.build(), getValidatorList());
}
@@ -611,6 +676,10 @@ public static ReadContainerResponseProto readContainer(
if (encodedToken != null) {
request.setEncodedToken(encodedToken);
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ request.setTraceID(traceId);
+ }
ContainerCommandResponseProto response =
client.sendCommand(request.build(), getValidatorList());
@@ -646,12 +715,53 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
ContainerCommandRequestProto request = builder.build();
ContainerCommandResponseProto response =
client.sendCommand(request, getValidatorList());
return response.getGetSmallFile();
}
+ /**
+ * Send an echo to DataNode.
+ *
+ * @return EchoResponseProto
+ */
+ public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID,
+ long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs, boolean readOnly)
+ throws IOException {
+ ContainerProtos.EchoRequestProto getEcho =
+ EchoRequestProto
+ .newBuilder()
+ .setPayload(payloadReqBytes)
+ .setPayloadSizeResp(payloadRespSizeKB)
+ .setSleepTimeMs(sleepTimeMs)
+ .setReadOnly(readOnly)
+ .build();
+ String id = client.getPipeline().getClosestNode().getUuidString();
+
+ ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto
+ .newBuilder()
+ .setCmdType(Type.Echo)
+ .setContainerID(containerID)
+ .setDatanodeUuid(id)
+ .setEcho(getEcho);
+ if (!encodedContainerID.isEmpty()) {
+ builder.setEncodedToken(encodedContainerID);
+ }
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
+ ContainerCommandRequestProto request = builder.build();
+ ContainerCommandResponseProto response =
+ client.sendCommand(request, getValidatorList());
+ return response.getEcho();
+ }
+
/**
* Validates a response from a container protocol call. Any non-successful
* return code is mapped to a corresponding exception and thrown.
@@ -675,7 +785,7 @@ public static void validateContainerResponse(
response.getMessage(), response.getResult());
}
- public static List getValidatorList() {
+ private static List getValidatorList() {
return VALIDATORS;
}
@@ -716,6 +826,10 @@ public static List toValidatorList(Validator validator) {
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
+ String traceId = TracingUtil.exportCurrentSpan();
+ if (traceId != null) {
+ builder.setTraceID(traceId);
+ }
ContainerCommandRequestProto request = builder.build();
Map