function, int maxLength) {
final int limit = data.limit();
try {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java
new file mode 100644
index 00000000000..0f6482919a3
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumCache.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.common;
+
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Cache previous checksums to avoid recomputing them.
+ * This is a stop-gap solution to reduce checksum calc overhead inside critical section
+ * without having to do a major refactoring/overhaul over protobuf and interfaces.
+ * This is only supposed to be used by BlockOutputStream, for now.
+ *
+ * Each BlockOutputStream has its own Checksum instance.
+ * Each block chunk (4 MB default) is divided into 16 KB (default) each for checksum calculation.
+ * For CRC32/CRC32C, each checksum takes 4 bytes. Thus each block chunk has 4 MB / 16 KB * 4 B = 1 KB of checksum data.
+ */
+public class ChecksumCache {
+ public static final Logger LOG = LoggerFactory.getLogger(ChecksumCache.class);
+
+ private final int bytesPerChecksum;
+ private final List checksums;
+ // Chunk length last time the checksum is computed
+ private int prevChunkLength;
+ // This only serves as a hint for array list initial allocation. The array list will still grow as needed.
+ private static final int BLOCK_CHUNK_SIZE = 4 * 1024 * 1024; // 4 MB
+
+ public ChecksumCache(int bytesPerChecksum) {
+ LOG.info("Initializing ChecksumCache with bytesPerChecksum = {}", bytesPerChecksum);
+ this.prevChunkLength = 0;
+ this.bytesPerChecksum = bytesPerChecksum;
+ // Set initialCapacity to avoid costly resizes
+ this.checksums = new ArrayList<>(BLOCK_CHUNK_SIZE / bytesPerChecksum);
+ }
+
+ /**
+ * Clear cached checksums. And reset the written index.
+ */
+ public void clear() {
+ prevChunkLength = 0;
+ checksums.clear();
+ }
+
+ public List getChecksums() {
+ return checksums;
+ }
+
+ public List computeChecksum(ChunkBuffer data, Function function) {
+ // Indicates how much data the current chunk buffer holds
+ final int currChunkLength = data.limit();
+
+ if (currChunkLength == prevChunkLength) {
+ LOG.debug("ChunkBuffer data limit same as last time ({}). No new checksums need to be computed", prevChunkLength);
+ return checksums;
+ }
+
+ // Sanity check
+ if (currChunkLength < prevChunkLength) {
+ // If currChunkLength <= lastChunkLength, it indicates a bug that needs to be addressed.
+ // It means BOS has not properly clear()ed the cache when a new chunk is started in that code path.
+ throw new IllegalArgumentException("ChunkBuffer data limit (" + currChunkLength + ")" +
+ " must not be smaller than last time (" + prevChunkLength + ")");
+ }
+
+ // One or more checksums need to be computed
+
+ // Start of the checksum index that need to be (re)computed
+ final int ciStart = prevChunkLength / bytesPerChecksum;
+ final int ciEnd = currChunkLength / bytesPerChecksum + (currChunkLength % bytesPerChecksum == 0 ? 0 : 1);
+ int i = 0;
+ for (ByteBuffer b : data.iterate(bytesPerChecksum)) {
+ if (i < ciStart) {
+ i++;
+ continue;
+ }
+
+ // variable i can either point to:
+ // 1. the last element in the list -- in which case the checksum needs to be updated
+ // 2. one after the last element -- in which case a new checksum needs to be added
+ assert i == checksums.size() - 1 || i == checksums.size();
+
+ // TODO: Furthermore for CRC32/CRC32C, it can be even more efficient by updating the last checksum byte-by-byte.
+ final ByteString checksum = Checksum.computeChecksum(b, function, bytesPerChecksum);
+ if (i == checksums.size()) {
+ checksums.add(checksum);
+ } else {
+ checksums.set(i, checksum);
+ }
+
+ i++;
+ }
+
+ // Sanity check
+ if (i != ciEnd) {
+ throw new IllegalStateException("ChecksumCache: Checksum index end does not match expectation");
+ }
+
+ // Update last written index
+ prevChunkLength = currChunkLength;
+ return checksums;
+ }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBuffer.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBuffer.java
index 36c16e92bf0..254be93dc4a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBuffer.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBuffer.java
@@ -25,9 +25,9 @@
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Objects;
-import java.util.UUID;
import java.util.function.Function;
+import org.apache.hadoop.ozone.common.utils.BufferUtils;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.util.UncheckedAutoCloseable;
@@ -35,7 +35,6 @@
final class ChunkBufferImplWithByteBuffer implements ChunkBuffer {
private final ByteBuffer buffer;
private final UncheckedAutoCloseable underlying;
- private final UUID identity = UUID.randomUUID();
ChunkBufferImplWithByteBuffer(ByteBuffer buffer) {
this(buffer, null);
@@ -104,7 +103,7 @@ public List asByteBufferList() {
@Override
public long writeTo(GatheringByteChannel channel) throws IOException {
- return channel.write(buffer);
+ return BufferUtils.writeFully(channel, buffer);
}
@Override
@@ -163,6 +162,6 @@ public int hashCode() {
@Override
public String toString() {
return getClass().getSimpleName() + ":limit=" + buffer.limit()
- + "@" + identity;
+ + "@" + Integer.toHexString(super.hashCode());
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBufferList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBufferList.java
index a3b5f9d2eef..e1f169662f8 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBufferList.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBufferList.java
@@ -23,6 +23,8 @@
import java.util.Collections;
import java.util.Iterator;
import java.util.NoSuchElementException;
+
+import org.apache.hadoop.ozone.common.utils.BufferUtils;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import java.io.IOException;
@@ -246,9 +248,9 @@ public List asByteBufferList() {
@Override
public long writeTo(GatheringByteChannel channel) throws IOException {
- long bytes = channel.write(buffers.toArray(new ByteBuffer[0]));
+ final long written = BufferUtils.writeFully(channel, buffers);
findCurrent();
- return bytes;
+ return written;
}
@Override
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/IncrementalChunkBuffer.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/IncrementalChunkBuffer.java
index dda4fae0d2b..732af4b6850 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/IncrementalChunkBuffer.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/IncrementalChunkBuffer.java
@@ -19,6 +19,7 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.utils.db.CodecBuffer;
+import org.apache.hadoop.ozone.common.utils.BufferUtils;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import java.io.IOException;
@@ -279,7 +280,7 @@ public List asByteBufferList() {
@Override
public long writeTo(GatheringByteChannel channel) throws IOException {
- return channel.write(buffers.toArray(new ByteBuffer[0]));
+ return BufferUtils.writeFully(channel, buffers);
}
@Override
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java
index c6ad754f19b..a266c3615b0 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/utils/BufferUtils.java
@@ -19,15 +19,23 @@
package org.apache.hadoop.ozone.common.utils;
import com.google.common.base.Preconditions;
+
+import java.io.IOException;
import java.nio.ByteBuffer;
+import java.nio.channels.GatheringByteChannel;
import java.util.ArrayList;
import java.util.List;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* Utilities for buffers.
*/
public final class BufferUtils {
+ public static final Logger LOG = LoggerFactory.getLogger(BufferUtils.class);
+
+ private static final ByteBuffer[] EMPTY_BYTE_BUFFER_ARRAY = {};
/** Utility classes should not be constructed. **/
private BufferUtils() {
@@ -136,4 +144,46 @@ public static int getNumberOfBins(long numElements, int maxElementsPerBin) {
}
return Math.toIntExact(n);
}
+
+ /**
+ * Write all remaining bytes in buffer to the given channel.
+ */
+ public static long writeFully(GatheringByteChannel ch, ByteBuffer bb) throws IOException {
+ long written = 0;
+ while (bb.remaining() > 0) {
+ int n = ch.write(bb);
+ if (n < 0) {
+ throw new IllegalStateException("GatheringByteChannel.write returns " + n + " < 0 for " + ch);
+ }
+ written += n;
+ }
+ return written;
+ }
+
+ public static long writeFully(GatheringByteChannel ch, List buffers) throws IOException {
+ return BufferUtils.writeFully(ch, buffers.toArray(EMPTY_BYTE_BUFFER_ARRAY));
+ }
+
+ public static long writeFully(GatheringByteChannel ch, ByteBuffer[] buffers) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ for (int i = 0; i < buffers.length; i++) {
+ LOG.debug("buffer[{}]: remaining={}", i, buffers[i].remaining());
+ }
+ }
+
+ long written = 0;
+ for (int i = 0; i < buffers.length; i++) {
+ while (buffers[i].remaining() > 0) {
+ final long n = ch.write(buffers, i, buffers.length - i);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("buffer[{}]: remaining={}, written={}", i, buffers[i].remaining(), n);
+ }
+ if (n < 0) {
+ throw new IllegalStateException("GatheringByteChannel.write returns " + n + " < 0 for " + ch);
+ }
+ written += n;
+ }
+ }
+ return written;
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
index ea5c5453f3f..4fee39921b6 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
@@ -39,7 +39,8 @@ public class BlockData {
private static final Codec CODEC = new DelegatedCodec<>(
Proto3Codec.get(ContainerProtos.BlockData.getDefaultInstance()),
BlockData::getFromProtoBuf,
- BlockData::getProtoBufMessage);
+ BlockData::getProtoBufMessage,
+ BlockData.class);
public static Codec getCodec() {
return CODEC;
@@ -253,7 +254,7 @@ public void setChunks(List chunks) {
size = singleChunk.getLen();
} else {
chunkList = chunks;
- size = chunks.parallelStream()
+ size = chunks.stream()
.mapToLong(ContainerProtos.ChunkInfo::getLen)
.sum();
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfoList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfoList.java
index 832ab40d30f..ab5d39e9c3d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfoList.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfoList.java
@@ -36,6 +36,7 @@ public class ChunkInfoList {
Proto3Codec.get(ContainerProtos.ChunkInfoList.getDefaultInstance()),
ChunkInfoList::getFromProtoBuf,
ChunkInfoList::getProtoBufMessage,
+ ChunkInfoList.class,
DelegatedCodec.CopyType.SHALLOW);
public static Codec getCodec() {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/grpc/metrics/GrpcMetrics.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/grpc/metrics/GrpcMetrics.java
index 6bd83b44a93..6e0dde66986 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/grpc/metrics/GrpcMetrics.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/grpc/metrics/GrpcMetrics.java
@@ -33,6 +33,7 @@
import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.util.MetricUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -100,6 +101,8 @@ public static synchronized GrpcMetrics create(Configuration conf) {
*/
public void unRegister() {
DefaultMetricsSystem.instance().unregisterSource(SOURCE_NAME);
+ MetricUtil.stop(grpcProcessingTimeMillisQuantiles);
+ MetricUtil.stop(grpcQueueTimeMillisQuantiles);
}
@Override
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/MetricUtil.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/MetricUtil.java
index 23ff3c0f29e..9d903b900ac 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/MetricUtil.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/MetricUtil.java
@@ -110,4 +110,20 @@ public static List createQuantiles(MetricsRegistry registry,
sampleName, valueName, interval);
}).collect(Collectors.toList());
}
+
+ public static void stop(MutableQuantiles... quantiles) {
+ if (quantiles != null) {
+ stop(Arrays.asList(quantiles));
+ }
+ }
+
+ public static void stop(Iterable quantiles) {
+ if (quantiles != null) {
+ for (MutableQuantiles q : quantiles) {
+ if (q != null) {
+ q.stop();
+ }
+ }
+ }
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetrics.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetrics.java
index 3f5150bd62c..39e887eaa49 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetrics.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetrics.java
@@ -22,7 +22,9 @@
import org.apache.hadoop.metrics2.lib.MutableQuantiles;
import org.apache.hadoop.metrics2.lib.MutableStat;
+import java.io.Closeable;
import java.util.List;
+import java.util.Map;
/**
* The {@code PerformanceMetrics} class encapsulates a collection of related
@@ -30,7 +32,7 @@
* This class provides methods to update these metrics and to
* snapshot their values for reporting.
*/
-public class PerformanceMetrics {
+public class PerformanceMetrics implements Closeable {
private final MutableStat stat;
private final List quantiles;
private final MutableMinMax minMax;
@@ -43,12 +45,13 @@ public class PerformanceMetrics {
* @param intervals the intervals for quantiles computation. Note, each
* interval in 'intervals' increases memory usage, as it corresponds
* to a separate quantile calculator.
+ * @return {@link PerformanceMetrics} instances created, mapped by field name
*/
- public static synchronized void initializeMetrics(T source,
+ public static synchronized Map initializeMetrics(T source,
MetricsRegistry registry, String sampleName, String valueName,
int[] intervals) {
try {
- PerformanceMetricsInitializer.initialize(
+ return PerformanceMetricsInitializer.initialize(
source, registry, sampleName, valueName, intervals);
} catch (IllegalAccessException e) {
throw new RuntimeException("Failed to initialize PerformanceMetrics", e);
@@ -73,6 +76,11 @@ public PerformanceMetrics(
minMax = new MutableMinMax(registry, name, description, valueName);
}
+ @Override
+ public void close() {
+ MetricUtil.stop(quantiles);
+ }
+
/**
* Adds a value to all the aggregated metrics.
*
@@ -95,6 +103,5 @@ public void snapshot(MetricsRecordBuilder recordBuilder, boolean all) {
this.quantiles.forEach(quantile -> quantile.snapshot(recordBuilder, all));
this.minMax.snapshot(recordBuilder, all);
}
-
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetricsInitializer.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetricsInitializer.java
index b2e83bb780c..cb6f77e9f5c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetricsInitializer.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/PerformanceMetricsInitializer.java
@@ -21,6 +21,8 @@
import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import java.lang.reflect.Field;
+import java.util.HashMap;
+import java.util.Map;
/**
* Utility class for initializing PerformanceMetrics in a MetricsSource.
@@ -36,11 +38,13 @@ private PerformanceMetricsInitializer() { }
* @param sampleName sample name
* @param valueName value name
* @param intervals intervals for quantiles
+ * @return {@link PerformanceMetrics} instances created, mapped by field name
* @throws IllegalAccessException if unable to access the field
*/
- public static void initialize(T source, MetricsRegistry registry,
+ public static Map initialize(T source, MetricsRegistry registry,
String sampleName, String valueName, int[] intervals)
throws IllegalAccessException {
+ Map instances = new HashMap<>();
Field[] fields = source.getClass().getDeclaredFields();
for (Field field : fields) {
@@ -54,8 +58,11 @@ public static void initialize(T source, MetricsRegistry registry,
sampleName, valueName, intervals);
field.setAccessible(true);
field.set(source, performanceMetrics);
+ instances.put(name, performanceMetrics);
}
}
}
+
+ return instances;
}
}
diff --git a/hadoop-hdds/common/src/main/resources/hdds-version-info.properties b/hadoop-hdds/common/src/main/resources/hdds-version-info.properties
index bf887021c5b..3ba2c2cbfa2 100644
--- a/hadoop-hdds/common/src/main/resources/hdds-version-info.properties
+++ b/hadoop-hdds/common/src/main/resources/hdds-version-info.properties
@@ -18,9 +18,6 @@
version=${declared.hdds.version}
revision=${version-info.scm.commit}
-branch=${version-info.scm.branch}
-user=${user.name}
-date=${version-info.build.time}
url=${version-info.scm.uri}
srcChecksum=${version-info.source.md5}
hadoopProtoc2Version=${proto2.hadooprpc.protobuf.version}
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 9b0ff0e9625..fdeb5c1c043 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -160,6 +160,13 @@
this not set. Ideally, this should be mapped to a fast disk like an SSD.
+
+ ozone.scm.container.list.max.count
+ 4096
+ OZONE, SCM, CONTAINER
+ The max number of containers info could be included in
+ response of ListContainer request.
+
hdds.datanode.dir
@@ -272,16 +279,16 @@
hdds.ratis.snapshot.threshold
- 10000
- OZONE, RATIS
+ 100000
+ OZONE, CONTAINER, RATIS
Number of transactions after which a ratis snapshot should be
taken.
hdds.container.ratis.statemachine.max.pending.apply-transactions
- 10000
- OZONE, RATIS
+ 100000
+ OZONE, CONTAINER, RATIS
Maximum number of pending apply transactions in a data
pipeline. The default value is kept same as default snapshot threshold
hdds.ratis.snapshot.threshold.
@@ -1570,7 +1577,7 @@
hdds.datanode.metadata.rocksdb.cache.size
- 64MB
+ 1GB
OZONE, DATANODE, MANAGEMENT
Size of the block metadata cache shared among RocksDB instances on each
@@ -3469,9 +3476,9 @@
ozone.s3g.client.buffer.size
OZONE, S3GATEWAY
- 4KB
+ 4MB
- The size of the buffer which is for read block. (4KB by default).
+ The size of the buffer which is for read block. (4MB by default).
@@ -4530,19 +4537,31 @@
- ozone.ec.grpc.zerocopy.enabled
- true
+ ozone.om.max.buckets
+ 100000
+ OZONE, OM
+
+ maximum number of buckets across all volumes.
+
+
+
+
+ ozone.volume.io.percentiles.intervals.seconds
+ 60
OZONE, DATANODE
- Specify if zero-copy should be enabled for EC GRPC protocol.
+ This setting specifies the interval (in seconds) for monitoring percentile performance metrics.
+ It helps in tracking the read and write performance of DataNodes in real-time,
+ allowing for better identification and analysis of performance issues.
+
- ozone.om.max.buckets
- 100000
+ ozone.om.server.list.max.size
+ 1000
OZONE, OM
- maximum number of buckets across all volumes.
+ Configuration property to configure the max server side response size for list calls on om.
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/TestDatanodeDetails.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/TestDatanodeDetails.java
index aeb1e207e70..78465fd2816 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/TestDatanodeDetails.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/TestDatanodeDetails.java
@@ -17,12 +17,16 @@
*/
package org.apache.hadoop.hdds.protocol;
+import com.google.common.collect.ImmutableSet;
import org.apache.hadoop.hdds.DatanodeVersion;
import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.junit.jupiter.api.Test;
import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.ALL_PORTS;
import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.V0_PORTS;
@@ -48,21 +52,36 @@ void protoIncludesNewPortsOnlyForV1() {
subject.toProto(VERSION_HANDLES_UNKNOWN_DN_PORTS.toProtoValue());
assertPorts(protoV1, ALL_PORTS);
}
+ @Test
+ void testRequiredPortsProto() {
+ DatanodeDetails subject = MockDatanodeDetails.randomDatanodeDetails();
+ Set requiredPorts = Stream.of(Port.Name.STANDALONE, Port.Name.RATIS)
+ .collect(Collectors.toSet());
+ HddsProtos.DatanodeDetailsProto proto =
+ subject.toProto(subject.getCurrentVersion(), requiredPorts);
+ assertPorts(proto, ImmutableSet.copyOf(requiredPorts));
+
+ HddsProtos.DatanodeDetailsProto ioPortProto =
+ subject.toProto(subject.getCurrentVersion(), Name.IO_PORTS);
+ assertPorts(ioPortProto, ImmutableSet.copyOf(Name.IO_PORTS));
+ }
@Test
public void testNewBuilderCurrentVersion() {
// test that if the current version is not set (Ozone 1.4.0 and earlier),
// it falls back to SEPARATE_RATIS_PORTS_AVAILABLE
DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails();
+ Set requiredPorts = Stream.of(Port.Name.STANDALONE, Port.Name.RATIS)
+ .collect(Collectors.toSet());
HddsProtos.DatanodeDetailsProto.Builder protoBuilder =
- dn.toProtoBuilder(DEFAULT_VERSION.toProtoValue());
+ dn.toProtoBuilder(DEFAULT_VERSION.toProtoValue(), requiredPorts);
protoBuilder.clearCurrentVersion();
DatanodeDetails dn2 = DatanodeDetails.newBuilder(protoBuilder.build()).build();
assertEquals(DatanodeVersion.SEPARATE_RATIS_PORTS_AVAILABLE.toProtoValue(), dn2.getCurrentVersion());
// test that if the current version is set, it is used
protoBuilder =
- dn.toProtoBuilder(DEFAULT_VERSION.toProtoValue());
+ dn.toProtoBuilder(DEFAULT_VERSION.toProtoValue(), requiredPorts);
DatanodeDetails dn3 = DatanodeDetails.newBuilder(protoBuilder.build()).build();
assertEquals(DatanodeVersion.CURRENT.toProtoValue(), dn3.getCurrentVersion());
}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
index 5571330ee64..d8af0c4d5ab 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
@@ -37,19 +37,19 @@ public class TestExcludeList {
public void excludeNodesShouldBeCleanedBasedOnGivenTime() {
ExcludeList list = new ExcludeList(10, clock);
list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
- .setIpAddress("127.0.0.1").setHostName("localhost").addPort(
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .setIpAddress("127.0.0.1").setHostName("localhost")
+ .addPort(DatanodeDetails.newStandalonePort(2001))
.build());
assertEquals(1, list.getDatanodes().size());
clock.fastForward(11);
assertEquals(0, list.getDatanodes().size());
list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
- .setIpAddress("127.0.0.2").setHostName("localhost").addPort(
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .setIpAddress("127.0.0.2").setHostName("localhost")
+ .addPort(DatanodeDetails.newStandalonePort(2001))
.build());
list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
- .setIpAddress("127.0.0.3").setHostName("localhost").addPort(
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .setIpAddress("127.0.0.3").setHostName("localhost")
+ .addPort(DatanodeDetails.newStandalonePort(2001))
.build());
assertEquals(2, list.getDatanodes().size());
}
@@ -58,8 +58,8 @@ public void excludeNodesShouldBeCleanedBasedOnGivenTime() {
public void excludeNodeShouldNotBeCleanedIfExpiryTimeIsZero() {
ExcludeList list = new ExcludeList(0, clock);
list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
- .setIpAddress("127.0.0.1").setHostName("localhost").addPort(
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .setIpAddress("127.0.0.1").setHostName("localhost")
+ .addPort(DatanodeDetails.newStandalonePort(2001))
.build());
assertEquals(1, list.getDatanodes().size());
clock.fastForward(1);
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/MockGatheringChannel.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/MockGatheringChannel.java
index ce6f58dadcb..83b68512380 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/MockGatheringChannel.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/MockGatheringChannel.java
@@ -21,8 +21,11 @@
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.ThreadLocalRandom;
import static com.google.common.base.Preconditions.checkElementIndex;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* {@link GatheringByteChannel} implementation for testing. Delegates
@@ -45,11 +48,32 @@ public long write(ByteBuffer[] srcs, int offset, int length)
checkElementIndex(offset, srcs.length, "offset");
checkElementIndex(offset + length - 1, srcs.length, "offset+length");
- long bytes = 0;
- for (ByteBuffer b : srcs) {
- bytes += write(b);
+ long fullLength = 0;
+ for (int i = offset; i < srcs.length; i++) {
+ fullLength += srcs[i].remaining();
}
- return bytes;
+ if (fullLength <= 0) {
+ return 0;
+ }
+
+ // simulate partial write by setting a random partial length
+ final long partialLength = ThreadLocalRandom.current().nextLong(fullLength + 1);
+
+ long written = 0;
+ for (int i = offset; i < srcs.length; i++) {
+ for (final ByteBuffer src = srcs[i]; src.hasRemaining();) {
+ final long n = partialLength - written; // write at most n bytes
+ assertThat(n).isGreaterThanOrEqualTo(0);
+ if (n == 0) {
+ return written;
+ }
+
+ final int remaining = src.remaining();
+ final int adjustment = remaining <= n ? 0 : Math.toIntExact(remaining - n);
+ written += adjustedWrite(src, adjustment);
+ }
+ }
+ return written;
}
@Override
@@ -59,7 +83,40 @@ public long write(ByteBuffer[] srcs) throws IOException {
@Override
public int write(ByteBuffer src) throws IOException {
- return delegate.write(src);
+ final int remaining = src.remaining();
+ if (remaining <= 0) {
+ return 0;
+ }
+ // Simulate partial write by a random adjustment.
+ final int adjustment = ThreadLocalRandom.current().nextInt(remaining + 1);
+ return adjustedWrite(src, adjustment);
+ }
+
+ /** Simulate partial write by the given adjustment. */
+ private int adjustedWrite(ByteBuffer src, int adjustment) throws IOException {
+ assertThat(adjustment).isGreaterThanOrEqualTo(0);
+ final int remaining = src.remaining();
+ if (remaining <= 0) {
+ return 0;
+ }
+ assertThat(adjustment).isLessThanOrEqualTo(remaining);
+
+ final int oldLimit = src.limit();
+ final int newLimit = oldLimit - adjustment;
+ src.limit(newLimit);
+ assertEquals(newLimit, src.limit());
+ final int toWrite = remaining - adjustment;
+ assertEquals(toWrite, src.remaining());
+
+ final int written = delegate.write(src);
+ assertEquals(newLimit, src.limit());
+ assertEquals(toWrite - written, src.remaining());
+
+ src.limit(oldLimit);
+ assertEquals(oldLimit, src.limit());
+ assertEquals(remaining - written, src.remaining());
+
+ return written;
}
@Override
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksum.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksum.java
index 829f4bb150c..7ddb605c0f8 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksum.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksum.java
@@ -19,7 +19,10 @@
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.nio.ByteBuffer;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -35,23 +38,25 @@ public class TestChecksum {
private static final ContainerProtos.ChecksumType CHECKSUM_TYPE_DEFAULT =
ContainerProtos.ChecksumType.SHA256;
- private Checksum getChecksum(ContainerProtos.ChecksumType type) {
+ private Checksum getChecksum(ContainerProtos.ChecksumType type, boolean allowChecksumCache) {
if (type == null) {
type = CHECKSUM_TYPE_DEFAULT;
}
- return new Checksum(type, BYTES_PER_CHECKSUM);
+ return new Checksum(type, BYTES_PER_CHECKSUM, allowChecksumCache);
}
/**
* Tests {@link Checksum#verifyChecksum(byte[], ChecksumData)}.
*/
- @Test
- public void testVerifyChecksum() throws Exception {
- Checksum checksum = getChecksum(null);
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void testVerifyChecksum(boolean useChecksumCache) throws Exception {
+ Checksum checksum = getChecksum(null, useChecksumCache);
int dataLen = 55;
byte[] data = RandomStringUtils.randomAlphabetic(dataLen).getBytes(UTF_8);
+ ByteBuffer byteBuffer = ByteBuffer.wrap(data);
- ChecksumData checksumData = checksum.computeChecksum(data);
+ ChecksumData checksumData = checksum.computeChecksum(byteBuffer, useChecksumCache);
// A checksum is calculate for each bytesPerChecksum number of bytes in
// the data. Since that value is 10 here and the data length is 55, we
@@ -65,11 +70,13 @@ public void testVerifyChecksum() throws Exception {
/**
* Tests that if data is modified, then the checksums should not match.
*/
- @Test
- public void testIncorrectChecksum() throws Exception {
- Checksum checksum = getChecksum(null);
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void testIncorrectChecksum(boolean useChecksumCache) throws Exception {
+ Checksum checksum = getChecksum(null, useChecksumCache);
byte[] data = RandomStringUtils.randomAlphabetic(55).getBytes(UTF_8);
- ChecksumData originalChecksumData = checksum.computeChecksum(data);
+ ByteBuffer byteBuffer = ByteBuffer.wrap(data);
+ ChecksumData originalChecksumData = checksum.computeChecksum(byteBuffer, useChecksumCache);
// Change the data and check if new checksum matches the original checksum.
// Modifying one byte of data should be enough for the checksum data to
@@ -83,13 +90,14 @@ public void testIncorrectChecksum() throws Exception {
* Tests that checksum calculated using two different checksumTypes should
* not match.
*/
- @Test
- public void testChecksumMismatchForDifferentChecksumTypes() {
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void testChecksumMismatchForDifferentChecksumTypes(boolean useChecksumCache) {
// Checksum1 of type SHA-256
- Checksum checksum1 = getChecksum(null);
+ Checksum checksum1 = getChecksum(null, useChecksumCache);
// Checksum2 of type CRC32
- Checksum checksum2 = getChecksum(ContainerProtos.ChecksumType.CRC32);
+ Checksum checksum2 = getChecksum(ContainerProtos.ChecksumType.CRC32, useChecksumCache);
// The two checksums should not match as they have different types
assertNotEquals(checksum1, checksum2, "Checksums should not match for different checksum types");
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java
new file mode 100644
index 00000000000..49e0b75127a
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumCache.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.common;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
+import org.apache.hadoop.ozone.common.Checksum.Algorithm;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Test class for {@link ChecksumCache}.
+ */
+class TestChecksumCache {
+ public static final Logger LOG = LoggerFactory.getLogger(TestChecksumCache.class);
+
+ @ParameterizedTest
+ @EnumSource(ChecksumType.class)
+ void testComputeChecksum(ChecksumType checksumType) throws Exception {
+ final int bytesPerChecksum = 16;
+ ChecksumCache checksumCache = new ChecksumCache(bytesPerChecksum);
+
+ final int size = 66;
+ byte[] byteArray = new byte[size];
+ // Fill byteArray with bytes from 0 to 127 for deterministic testing
+ for (int i = 0; i < size; i++) {
+ byteArray[i] = (byte) (i % 128);
+ }
+
+ final Function function = Algorithm.valueOf(checksumType).newChecksumFunction();
+
+ int iEnd = size / bytesPerChecksum + (size % bytesPerChecksum == 0 ? 0 : 1);
+ List lastRes = null;
+ for (int i = 0; i < iEnd; i++) {
+ int byteBufferLength = Integer.min(byteArray.length, bytesPerChecksum * (i + 1));
+ ByteBuffer byteBuffer = ByteBuffer.wrap(byteArray, 0, byteBufferLength);
+
+ try (ChunkBuffer chunkBuffer = ChunkBuffer.wrap(byteBuffer.asReadOnlyBuffer())) {
+ List res = checksumCache.computeChecksum(chunkBuffer, function);
+ System.out.println(res);
+ // Verify that every entry in the res list except the last one is the same as the one in lastRes list
+ if (i > 0) {
+ for (int j = 0; j < res.size() - 1; j++) {
+ Assertions.assertEquals(lastRes.get(j), res.get(j));
+ }
+ }
+ lastRes = res;
+ }
+ }
+
+ // Sanity check
+ checksumCache.clear();
+ }
+}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index 2b7592e1c35..20372dcc6ea 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -293,18 +293,31 @@ public static Builder newReadChunkRequestBuilder(Pipeline pipeline,
*/
public static ContainerCommandRequestProto getCreateContainerRequest(
long containerID, Pipeline pipeline) throws IOException {
+ return getCreateContainerRequest(containerID, pipeline, ContainerProtos.ContainerDataProto.State.OPEN);
+ }
+
+
+ /**
+ * Returns a create container command for test purposes. There are a bunch of
+ * tests where we need to just send a request and get a reply.
+ *
+ * @return ContainerCommandRequestProto.
+ */
+ public static ContainerCommandRequestProto getCreateContainerRequest(
+ long containerID, Pipeline pipeline, ContainerProtos.ContainerDataProto.State state) throws IOException {
LOG.trace("addContainer: {}", containerID);
- return getContainerCommandRequestBuilder(containerID, pipeline).build();
+ return getContainerCommandRequestBuilder(containerID, pipeline, state)
+ .build();
}
private static Builder getContainerCommandRequestBuilder(long containerID,
- Pipeline pipeline) throws IOException {
+ Pipeline pipeline, ContainerProtos.ContainerDataProto.State state) throws IOException {
Builder request =
ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.CreateContainer);
request.setContainerID(containerID);
request.setCreateContainer(
- ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
+ ContainerProtos.CreateContainerRequestProto.getDefaultInstance().toBuilder().setState(state).build());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request;
@@ -320,7 +333,8 @@ public static ContainerCommandRequestProto getCreateContainerSecureRequest(
long containerID, Pipeline pipeline, Token> token) throws IOException {
LOG.trace("addContainer: {}", containerID);
- Builder request = getContainerCommandRequestBuilder(containerID, pipeline);
+ Builder request = getContainerCommandRequestBuilder(containerID, pipeline,
+ ContainerProtos.ContainerDataProto.State.OPEN);
if (token != null) {
request.setEncodedToken(token.encodeToUrlString());
}
diff --git a/hadoop-hdds/config/pom.xml b/hadoop-hdds/config/pom.xml
index 1c71bf3d90a..60c63475ae3 100644
--- a/hadoop-hdds/config/pom.xml
+++ b/hadoop-hdds/config/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozone
hdds
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
hdds-config
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
Apache Ozone Distributed Data Store Config Tools
Apache Ozone HDDS Config
jar
diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml
index d73bea95895..c21ca8203b5 100644
--- a/hadoop-hdds/container-service/pom.xml
+++ b/hadoop-hdds/container-service/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.ozone
hdds
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
hdds-container-service
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
Apache Ozone Distributed Data Store Container Service
Apache Ozone HDDS Container Service
jar
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java
index d36fcdb6fc7..9c077a8e27b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java
@@ -26,4 +26,32 @@
*/
@InterfaceAudience.Private
public interface DNMXBean extends ServiceRuntimeInfo {
+
+ /**
+ * Gets the datanode hostname.
+ *
+ * @return the datanode hostname for the datanode.
+ */
+ String getHostname();
+
+ /**
+ * Gets the client rpc port.
+ *
+ * @return the client rpc port
+ */
+ String getClientRpcPort();
+
+ /**
+ * Gets the http port.
+ *
+ * @return the http port
+ */
+ String getHttpPort();
+
+ /**
+ * Gets the https port.
+ *
+ * @return the http port
+ */
+ String getHttpsPort();
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java
index f7b484c6bb3..5a0a4556636 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java
@@ -25,8 +25,53 @@
* This is the JMX management class for DN information.
*/
public class DNMXBeanImpl extends ServiceRuntimeInfoImpl implements DNMXBean {
- public DNMXBeanImpl(
- VersionInfo versionInfo) {
+
+ private String hostName;
+ private String clientRpcPort;
+ private String httpPort;
+ private String httpsPort;
+
+ public DNMXBeanImpl(VersionInfo versionInfo) {
super(versionInfo);
}
+
+ @Override
+ public String getHostname() {
+ return hostName;
+ }
+
+ @Override
+ public String getClientRpcPort() {
+ return clientRpcPort;
+ }
+
+ @Override
+ public String getHttpPort() {
+ return httpPort;
+ }
+
+ @Override
+ public String getHttpsPort() {
+ return httpsPort;
+ }
+
+ public void setHttpPort(String httpPort) {
+ this.httpPort = httpPort;
+ }
+
+ public void setHostName(String hostName) {
+ this.hostName = hostName;
+ }
+
+ public void setClientRpcPort(String rpcPort) {
+ this.clientRpcPort = rpcPort;
+ }
+
+ public String getHostName() {
+ return hostName;
+ }
+
+ public void setHttpsPort(String httpsPort) {
+ this.httpsPort = httpsPort;
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index 6b32b74dc7c..de21e37503a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -40,6 +40,7 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.SecretKeyProtocol;
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
import org.apache.hadoop.hdds.security.SecurityConfig;
import org.apache.hadoop.hdds.security.symmetric.DefaultSecretKeyClient;
import org.apache.hadoop.hdds.security.symmetric.SecretKeyClient;
@@ -116,8 +117,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
private final Map ratisMetricsMap =
new ConcurrentHashMap<>();
private List ratisReporterList = null;
- private DNMXBeanImpl serviceRuntimeInfo =
- new DNMXBeanImpl(HddsVersionInfo.HDDS_VERSION_INFO) { };
+ private DNMXBeanImpl serviceRuntimeInfo;
private ObjectName dnInfoBeanName;
private HddsDatanodeClientProtocolServer clientProtocolServer;
private OzoneAdmins admins;
@@ -210,6 +210,12 @@ public void start(OzoneConfiguration configuration) {
}
public void start() {
+ serviceRuntimeInfo = new DNMXBeanImpl(HddsVersionInfo.HDDS_VERSION_INFO) {
+ @Override
+ public String getNamespace() {
+ return SCMHAUtils.getScmServiceId(conf);
+ }
+ };
serviceRuntimeInfo.setStartTime();
ratisReporterList = RatisDropwizardExports
@@ -222,13 +228,13 @@ public void start() {
String ip = InetAddress.getByName(hostname).getHostAddress();
datanodeDetails = initializeDatanodeDetails();
datanodeDetails.setHostName(hostname);
+ serviceRuntimeInfo.setHostName(hostname);
datanodeDetails.setIpAddress(ip);
datanodeDetails.setVersion(
HddsVersionInfo.HDDS_VERSION_INFO.getVersion());
datanodeDetails.setSetupTime(Time.now());
datanodeDetails.setRevision(
HddsVersionInfo.HDDS_VERSION_INFO.getRevision());
- datanodeDetails.setBuildDate(HddsVersionInfo.HDDS_VERSION_INFO.getDate());
TracingUtil.initTracing(
"HddsDatanodeService." + datanodeDetails.getUuidString()
.substring(0, 8), conf);
@@ -295,23 +301,30 @@ public void start() {
httpServer = new HddsDatanodeHttpServer(conf);
httpServer.start();
HttpConfig.Policy policy = HttpConfig.getHttpPolicy(conf);
+
if (policy.isHttpEnabled()) {
- datanodeDetails.setPort(DatanodeDetails.newPort(HTTP,
- httpServer.getHttpAddress().getPort()));
+ int httpPort = httpServer.getHttpAddress().getPort();
+ datanodeDetails.setPort(DatanodeDetails.newPort(HTTP, httpPort));
+ serviceRuntimeInfo.setHttpPort(String.valueOf(httpPort));
}
+
if (policy.isHttpsEnabled()) {
- datanodeDetails.setPort(DatanodeDetails.newPort(HTTPS,
- httpServer.getHttpsAddress().getPort()));
+ int httpsPort = httpServer.getHttpAddress().getPort();
+ datanodeDetails.setPort(DatanodeDetails.newPort(HTTPS, httpsPort));
+ serviceRuntimeInfo.setHttpsPort(String.valueOf(httpsPort));
}
+
} catch (Exception ex) {
LOG.error("HttpServer failed to start.", ex);
}
-
clientProtocolServer = new HddsDatanodeClientProtocolServer(
datanodeDetails, conf, HddsVersionInfo.HDDS_VERSION_INFO,
reconfigurationHandler);
+ int clientRpcport = clientProtocolServer.getClientRpcAddress().getPort();
+ serviceRuntimeInfo.setClientRpcPort(String.valueOf(clientRpcport));
+
// Get admin list
String starterUser =
UserGroupInformation.getCurrentUser().getShortUserName();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsPolicyProvider.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsPolicyProvider.java
index eeed4fab5f7..52217ce7f83 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsPolicyProvider.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsPolicyProvider.java
@@ -24,7 +24,7 @@
import org.apache.hadoop.security.authorize.Service;
import org.apache.ratis.util.MemoizedSupplier;
-import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
import java.util.function.Supplier;
@@ -50,7 +50,7 @@ public static HddsPolicyProvider getInstance() {
}
private static final List DN_SERVICES =
- Arrays.asList(
+ Collections.singletonList(
new Service(
OZONE_SECURITY_RECONFIGURE_PROTOCOL_ACL,
ReconfigureProtocol.class)
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/CommandHandlerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/CommandHandlerMetrics.java
index a6e4d6258d9..e52565952a5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/CommandHandlerMetrics.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/CommandHandlerMetrics.java
@@ -34,6 +34,7 @@
import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.TotalRunTimeMs;
import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.QueueWaitingTaskCount;
import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.InvocationCount;
+import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.AvgRunTimeMs;
import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.ThreadPoolActivePoolSize;
import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.ThreadPoolMaxPoolSize;
import static org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics.CommandMetricsMetricsInfo.CommandReceivedCount;
@@ -46,6 +47,7 @@ public final class CommandHandlerMetrics implements MetricsSource {
enum CommandMetricsMetricsInfo implements MetricsInfo {
Command("The type of the SCM command"),
TotalRunTimeMs("The total runtime of the command handler in milliseconds"),
+ AvgRunTimeMs("Average run time of the command handler in milliseconds"),
QueueWaitingTaskCount("The number of queued tasks waiting for execution"),
InvocationCount("The number of times the command handler has been invoked"),
ThreadPoolActivePoolSize("The number of active threads in the thread pool"),
@@ -108,6 +110,7 @@ public void getMetrics(MetricsCollector collector, boolean all) {
commandHandler.getCommandType().name());
builder.addGauge(TotalRunTimeMs, commandHandler.getTotalRunTime());
+ builder.addGauge(AvgRunTimeMs, commandHandler.getAverageRunTime());
builder.addGauge(QueueWaitingTaskCount, commandHandler.getQueuedCount());
builder.addGauge(InvocationCount, commandHandler.getInvocationCount());
int activePoolSize = commandHandler.getThreadPoolActivePoolSize();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
index 91bdb17cda9..03dbce061bb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
@@ -30,7 +30,9 @@
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableQuantiles;
import org.apache.hadoop.metrics2.lib.MutableRate;
+import org.apache.hadoop.ozone.util.MetricUtil;
+import java.io.Closeable;
import java.util.EnumMap;
/**
@@ -47,7 +49,7 @@
*/
@InterfaceAudience.Private
@Metrics(about = "Storage Container DataNode Metrics", context = "dfs")
-public class ContainerMetrics {
+public class ContainerMetrics implements Closeable {
public static final String STORAGE_CONTAINER_METRICS =
"StorageContainerMetrics";
@Metric private MutableCounterLong numOps;
@@ -109,6 +111,11 @@ public static void remove() {
ms.unregisterSource(STORAGE_CONTAINER_METRICS);
}
+ @Override
+ public void close() {
+ opsLatQuantiles.values().forEach(MetricUtil::stop);
+ }
+
public void incContainerOpsMetrics(ContainerProtos.Type type) {
numOps.incr();
numOpsArray.get(type).incr();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index 15cc6245ddb..8dd35064e6b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -23,8 +23,12 @@
import com.google.common.collect.ImmutableMap;
import com.google.protobuf.Message;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State;
+
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.utils.db.InMemoryTestTable;
+import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.utils.ContainerLogger;
@@ -65,10 +69,24 @@ public class ContainerSet implements Iterable> {
new ConcurrentSkipListMap<>();
private Clock clock;
private long recoveringTimeout;
+ private final Table containerIdsTable;
+ @VisibleForTesting
public ContainerSet(long recoveringTimeout) {
+ this(new InMemoryTestTable<>(), recoveringTimeout);
+ }
+
+ public ContainerSet(Table continerIdsTable, long recoveringTimeout) {
+ this(continerIdsTable, recoveringTimeout, false);
+ }
+
+ public ContainerSet(Table continerIdsTable, long recoveringTimeout, boolean readOnly) {
this.clock = Clock.system(ZoneOffset.UTC);
+ this.containerIdsTable = continerIdsTable;
this.recoveringTimeout = recoveringTimeout;
+ if (!readOnly && containerIdsTable == null) {
+ throw new IllegalArgumentException("Container table cannot be null when container set is not read only");
+ }
}
public long getCurrentTime() {
@@ -85,22 +103,64 @@ public void setRecoveringTimeout(long recoveringTimeout) {
this.recoveringTimeout = recoveringTimeout;
}
+ /**
+ * Add Container to container map. This would fail if the container is already present or has been marked as missing.
+ * @param container container to be added
+ * @return If container is added to containerMap returns true, otherwise
+ * false
+ */
+ public boolean addContainer(Container> container) throws StorageContainerException {
+ return addContainer(container, false);
+ }
+
+ /**
+ * Add Container to container map. This would overwrite the container even if it is missing. But would fail if the
+ * container is already present.
+ * @param container container to be added
+ * @return If container is added to containerMap returns true, otherwise
+ * false
+ */
+ public boolean addContainerByOverwriteMissingContainer(Container> container) throws StorageContainerException {
+ return addContainer(container, true);
+ }
+
+ public void ensureContainerNotMissing(long containerId, State state) throws StorageContainerException {
+ if (missingContainerSet.contains(containerId)) {
+ throw new StorageContainerException(String.format("Container with container Id %d with state : %s is missing in" +
+ " the DN.", containerId, state),
+ ContainerProtos.Result.CONTAINER_MISSING);
+ }
+ }
+
/**
* Add Container to container map.
* @param container container to be added
+ * @param overwrite if true should overwrite the container if the container was missing.
* @return If container is added to containerMap returns true, otherwise
* false
*/
- public boolean addContainer(Container> container) throws
+ private boolean addContainer(Container> container, boolean overwrite) throws
StorageContainerException {
Preconditions.checkNotNull(container, "container cannot be null");
long containerId = container.getContainerData().getContainerID();
+ State containerState = container.getContainerData().getState();
+ if (!overwrite) {
+ ensureContainerNotMissing(containerId, containerState);
+ }
if (containerMap.putIfAbsent(containerId, container) == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Container with container Id {} is added to containerMap",
containerId);
}
+ try {
+ if (containerIdsTable != null) {
+ containerIdsTable.put(containerId, containerState.toString());
+ }
+ } catch (IOException e) {
+ throw new StorageContainerException(e, ContainerProtos.Result.IO_EXCEPTION);
+ }
+ missingContainerSet.remove(containerId);
// wish we could have done this from ContainerData.setState
container.getContainerData().commitSpace();
if (container.getContainerData().getState() == RECOVERING) {
@@ -122,21 +182,69 @@ public boolean addContainer(Container> container) throws
* @return Container
*/
public Container> getContainer(long containerId) {
- Preconditions.checkState(containerId >= 0,
- "Container Id cannot be negative.");
+ Preconditions.checkState(containerId >= 0, "Container Id cannot be negative.");
return containerMap.get(containerId);
}
+ /**
+ * Removes container from both memory and database. This should be used when the containerData on disk has been
+ * removed completely from the node.
+ * @param containerId
+ * @return True if container is removed from containerMap.
+ * @throws StorageContainerException
+ */
+ public boolean removeContainer(long containerId) throws StorageContainerException {
+ return removeContainer(containerId, false, true);
+ }
+
+ /**
+ * Removes containerId from memory. This needs to be used when the container is still present on disk, and the
+ * inmemory state of the container needs to be updated.
+ * @param containerId
+ * @return True if container is removed from containerMap.
+ * @throws StorageContainerException
+ */
+ public boolean removeContainerOnlyFromMemory(long containerId) throws StorageContainerException {
+ return removeContainer(containerId, false, false);
+ }
+
+ /**
+ * Marks a container to be missing, thus it removes the container from inmemory containerMap and marks the
+ * container as missing.
+ * @param containerId
+ * @return True if container is removed from containerMap.
+ * @throws StorageContainerException
+ */
+ public boolean removeMissingContainer(long containerId) throws StorageContainerException {
+ return removeContainer(containerId, true, false);
+ }
+
/**
* Removes the Container matching with specified containerId.
* @param containerId ID of the container to remove
* @return If container is removed from containerMap returns true, otherwise
* false
*/
- public boolean removeContainer(long containerId) {
+ private boolean removeContainer(long containerId, boolean markMissing, boolean removeFromDB)
+ throws StorageContainerException {
Preconditions.checkState(containerId >= 0,
"Container Id cannot be negative.");
+ //We need to add to missing container set before removing containerMap since there could be write chunk operation
+ // that could recreate the container in another volume if we remove it from the map before adding to missing
+ // container.
+ if (markMissing) {
+ missingContainerSet.add(containerId);
+ }
Container> removed = containerMap.remove(containerId);
+ if (removeFromDB) {
+ try {
+ if (containerIdsTable != null) {
+ containerIdsTable.delete(containerId);
+ }
+ } catch (IOException e) {
+ throw new StorageContainerException(e, ContainerProtos.Result.IO_EXCEPTION);
+ }
+ }
if (removed == null) {
LOG.debug("Container with containerId {} is not present in " +
"containerMap", containerId);
@@ -190,20 +298,20 @@ public int containerCount() {
*
* @param context StateContext
*/
- public void handleVolumeFailures(StateContext context) {
+ public void handleVolumeFailures(StateContext context) throws StorageContainerException {
AtomicBoolean failedVolume = new AtomicBoolean(false);
AtomicInteger containerCount = new AtomicInteger(0);
- containerMap.values().forEach(c -> {
+ for (Container> c : containerMap.values()) {
ContainerData data = c.getContainerData();
if (data.getVolume().isFailed()) {
- removeContainer(data.getContainerID());
+ removeMissingContainer(data.getContainerID());
LOG.debug("Removing Container {} as the Volume {} " +
- "has failed", data.getContainerID(), data.getVolume());
+ "has failed", data.getContainerID(), data.getVolume());
failedVolume.set(true);
containerCount.incrementAndGet();
ContainerLogger.logLost(data, "Volume failure");
}
- });
+ }
if (failedVolume.get()) {
try {
@@ -251,6 +359,21 @@ public Iterator> getContainerIterator(HddsVolume volume) {
.iterator();
}
+ /**
+ * Get the number of containers based on the given volume.
+ *
+ * @param volume hdds volume.
+ * @return number of containers
+ */
+ public long containerCount(HddsVolume volume) {
+ Preconditions.checkNotNull(volume);
+ Preconditions.checkNotNull(volume.getStorageID());
+ String volumeUuid = volume.getStorageID();
+ return containerMap.values().stream()
+ .filter(x -> volumeUuid.equals(x.getContainerData().getVolume()
+ .getStorageID())).count();
+ }
+
/**
* Return an containerMap iterator over {@link ContainerSet#containerMap}.
* @return containerMap Iterator
@@ -347,6 +470,10 @@ public Set getMissingContainerSet() {
return missingContainerSet;
}
+ public Table getContainerIdsTable() {
+ return containerIdsTable;
+ }
+
/**
* Builds the missing container set by taking a diff between total no
* containers actually found and number of containers which actually
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 28aa3d8588f..cd99b909231 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -177,7 +177,8 @@ private boolean canIgnoreException(Result result) {
case CONTAINER_UNHEALTHY:
case CLOSED_CONTAINER_IO:
case DELETE_ON_OPEN_CONTAINER:
- case UNSUPPORTED_REQUEST: // Blame client for sending unsupported request.
+ case UNSUPPORTED_REQUEST:// Blame client for sending unsupported request.
+ case CONTAINER_MISSING:
return true;
default:
return false;
@@ -278,7 +279,8 @@ private ContainerCommandResponseProto dispatchRequest(
getMissingContainerSet().remove(containerID);
}
}
- if (getMissingContainerSet().contains(containerID)) {
+ if (cmdType != Type.CreateContainer && !HddsUtils.isReadOnly(msg)
+ && getMissingContainerSet().contains(containerID)) {
StorageContainerException sce = new StorageContainerException(
"ContainerID " + containerID
+ " has been lost and cannot be recreated on this DataNode",
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
index 77a4d97878d..fb9dc49071b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -23,6 +23,7 @@
import java.io.OutputStream;
import java.util.Set;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -96,7 +97,8 @@ public abstract StateMachine.DataChannel getStreamDataChannel(
*
* @return datanode Id
*/
- protected String getDatanodeId() {
+ @VisibleForTesting
+ public String getDatanodeId() {
return datanodeId;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
index a6c3b11de92..b3854e7ecd2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
@@ -234,12 +234,17 @@ public void logIfNeeded(Exception ex) {
}
if (missCounter == 0) {
+ long missedDurationSeconds = TimeUnit.MILLISECONDS.toSeconds(
+ this.getMissedCount() * getScmHeartbeatInterval(this.conf)
+ );
LOG.warn(
- "Unable to communicate to {} server at {} for past {} seconds.",
- serverName,
- getAddress().getHostString() + ":" + getAddress().getPort(),
- TimeUnit.MILLISECONDS.toSeconds(this.getMissedCount() *
- getScmHeartbeatInterval(this.conf)), ex);
+ "Unable to communicate to {} server at {}:{} for past {} seconds.",
+ serverName,
+ address.getAddress(),
+ address.getPort(),
+ missedDurationSeconds,
+ ex
+ );
}
if (LOG.isTraceEnabled()) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
index bc703ac6a55..cd032d4b275 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
@@ -31,6 +31,8 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager;
@@ -58,7 +60,7 @@ public class CloseContainerCommandHandler implements CommandHandler {
private final AtomicLong invocationCount = new AtomicLong(0);
private final AtomicInteger queuedCount = new AtomicInteger(0);
private final ThreadPoolExecutor executor;
- private long totalTime;
+ private final MutableRate opsLatencyMs;
/**
* Constructs a close container command handler.
@@ -72,6 +74,9 @@ public CloseContainerCommandHandler(
new ThreadFactoryBuilder()
.setNameFormat(threadNamePrefix + "CloseContainerThread-%d")
.build());
+ MetricsRegistry registry = new MetricsRegistry(
+ CloseContainerCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(SCMCommandProto.Type.closeContainerCommand + "Ms");
}
/**
@@ -155,7 +160,7 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
LOG.error("Can't close container #{}", containerId, e);
} finally {
long endTime = Time.monotonicNow();
- totalTime += endTime - startTime;
+ this.opsLatencyMs.add(endTime - startTime);
}
}, executor).whenComplete((v, e) -> queuedCount.decrementAndGet());
}
@@ -204,15 +209,12 @@ public int getInvocationCount() {
*/
@Override
public long getAverageRunTime() {
- if (invocationCount.get() > 0) {
- return totalTime / invocationCount.get();
- }
- return 0;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime;
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java
index 241abb6f4ae..be39277fdfa 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java
@@ -24,6 +24,8 @@
import org.apache.hadoop.hdds.ratis.RatisHelper;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@@ -60,9 +62,9 @@ public class ClosePipelineCommandHandler implements CommandHandler {
private final AtomicLong invocationCount = new AtomicLong(0);
private final AtomicInteger queuedCount = new AtomicInteger(0);
- private long totalTime;
private final Executor executor;
private final BiFunction newRaftClient;
+ private final MutableRate opsLatencyMs;
/**
* Constructs a closePipelineCommand handler.
@@ -80,6 +82,9 @@ public ClosePipelineCommandHandler(
Executor executor) {
this.newRaftClient = newRaftClient;
this.executor = executor;
+ MetricsRegistry registry = new MetricsRegistry(
+ ClosePipelineCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(SCMCommandProto.Type.closePipelineCommand + "Ms");
}
/**
@@ -155,7 +160,7 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
}
} finally {
long endTime = Time.monotonicNow();
- totalTime += endTime - startTime;
+ this.opsLatencyMs.add(endTime - startTime);
}
}, executor).whenComplete((v, e) -> queuedCount.decrementAndGet());
}
@@ -187,15 +192,12 @@ public int getInvocationCount() {
*/
@Override
public long getAverageRunTime() {
- if (invocationCount.get() > 0) {
- return totalTime / invocationCount.get();
- }
- return 0;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime;
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
index 4a36a1987de..62fc8a919d8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
@@ -30,6 +30,8 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.ratis.RatisHelper;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
@@ -59,8 +61,8 @@ public class CreatePipelineCommandHandler implements CommandHandler {
private final AtomicInteger queuedCount = new AtomicInteger(0);
private final BiFunction newRaftClient;
- private long totalTime;
private final Executor executor;
+ private final MutableRate opsLatencyMs;
/**
* Constructs a createPipelineCommand handler.
@@ -75,6 +77,9 @@ public CreatePipelineCommandHandler(ConfigurationSource conf,
Executor executor) {
this.newRaftClient = newRaftClient;
this.executor = executor;
+ MetricsRegistry registry = new MetricsRegistry(
+ CreatePipelineCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(SCMCommandProto.Type.createPipelineCommand + "Ms");
}
/**
@@ -135,7 +140,7 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
}
} finally {
long endTime = Time.monotonicNow();
- totalTime += endTime - startTime;
+ this.opsLatencyMs.add(endTime - startTime);
}
}, executor).whenComplete((v, e) -> queuedCount.decrementAndGet());
}
@@ -167,15 +172,12 @@ public int getInvocationCount() {
*/
@Override
public long getAverageRunTime() {
- if (invocationCount.get() > 0) {
- return totalTime / invocationCount.get();
- }
- return 0;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime;
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index bd7431c6145..136c5805821 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -32,6 +32,8 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.common.helpers.BlockDeletingServiceMetrics;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfoList;
@@ -91,7 +93,6 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
private final ContainerSet containerSet;
private final ConfigurationSource conf;
private int invocationCount;
- private long totalTime;
private final ThreadPoolExecutor executor;
private final LinkedBlockingQueue deleteCommandQueues;
private final Daemon handlerThread;
@@ -99,6 +100,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
private final BlockDeletingServiceMetrics blockDeleteMetrics;
private final long tryLockTimeoutMs;
private final Map schemaHandlers;
+ private final MutableRate opsLatencyMs;
public DeleteBlocksCommandHandler(OzoneContainer container,
ConfigurationSource conf, DatanodeConfiguration dnConf,
@@ -121,6 +123,9 @@ public DeleteBlocksCommandHandler(OzoneContainer container,
dnConf.getBlockDeleteThreads(), threadFactory);
this.deleteCommandQueues =
new LinkedBlockingQueue<>(dnConf.getBlockDeleteQueueLimit());
+ MetricsRegistry registry = new MetricsRegistry(
+ DeleteBlocksCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(SCMCommandProto.Type.deleteBlocksCommand + "Ms");
long interval = dnConf.getBlockDeleteCommandWorkerInterval().toMillis();
handlerThread = new Daemon(new DeleteCmdWorker(interval));
handlerThread.start();
@@ -354,10 +359,11 @@ private void processCmd(DeleteCmdInfo cmd) {
DeletedContainerBlocksSummary summary =
DeletedContainerBlocksSummary.getFrom(containerBlocks);
LOG.info("Summary of deleting container blocks, numOfTransactions={}, "
- + "numOfContainers={}, numOfBlocks={}",
+ + "numOfContainers={}, numOfBlocks={}, commandId={}.",
summary.getNumOfTxs(),
summary.getNumOfContainers(),
- summary.getNumOfBlocks());
+ summary.getNumOfBlocks(),
+ cmd.getCmd().getId());
if (LOG.isDebugEnabled()) {
LOG.debug("Start to delete container blocks, TXIDs={}",
summary.getTxIDSummary());
@@ -384,7 +390,8 @@ private void processCmd(DeleteCmdInfo cmd) {
LOG.debug("Sending following block deletion ACK to SCM");
for (DeleteBlockTransactionResult result : blockDeletionACK
.getResultsList()) {
- LOG.debug("{} : {}", result.getTxID(), result.getSuccess());
+ LOG.debug("TxId = {} : ContainerId = {} : {}",
+ result.getTxID(), result.getContainerID(), result.getSuccess());
}
}
}
@@ -403,7 +410,7 @@ private void processCmd(DeleteCmdInfo cmd) {
};
updateCommandStatus(cmd.getContext(), cmd.getCmd(), statusUpdater, LOG);
long endTime = Time.monotonicNow();
- totalTime += endTime - startTime;
+ this.opsLatencyMs.add(endTime - startTime);
invocationCount++;
}
}
@@ -666,15 +673,12 @@ public int getInvocationCount() {
@Override
public long getAverageRunTime() {
- if (invocationCount > 0) {
- return totalTime / invocationCount;
- }
- return 0;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime;
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
index b76e306e1c0..59aaacc1c80 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
@@ -22,6 +22,8 @@
import java.util.concurrent.RejectedExecutionException;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@@ -39,7 +41,6 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
/**
* Handler to process the DeleteContainerCommand from SCM.
@@ -51,10 +52,10 @@ public class DeleteContainerCommandHandler implements CommandHandler {
private final AtomicInteger invocationCount = new AtomicInteger(0);
private final AtomicInteger timeoutCount = new AtomicInteger(0);
- private final AtomicLong totalTime = new AtomicLong(0);
private final ThreadPoolExecutor executor;
private final Clock clock;
private int maxQueueSize;
+ private final MutableRate opsLatencyMs;
public DeleteContainerCommandHandler(
int threadPoolSize, Clock clock, int queueSize, String threadNamePrefix) {
@@ -73,6 +74,9 @@ protected DeleteContainerCommandHandler(Clock clock,
this.executor = executor;
this.clock = clock;
maxQueueSize = queueSize;
+ MetricsRegistry registry = new MetricsRegistry(
+ DeleteContainerCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(SCMCommandProto.Type.deleteContainerCommand + "Ms");
}
@Override
public void handle(final SCMCommand command,
@@ -124,7 +128,7 @@ private void handleInternal(SCMCommand command, StateContext context,
} catch (IOException e) {
LOG.error("Exception occurred while deleting the container.", e);
} finally {
- totalTime.getAndAdd(Time.monotonicNow() - startTime);
+ this.opsLatencyMs.add(Time.monotonicNow() - startTime);
}
}
@@ -149,14 +153,12 @@ public int getTimeoutCount() {
@Override
public long getAverageRunTime() {
- final int invocations = invocationCount.get();
- return invocations == 0 ?
- 0 : totalTime.get() / invocations;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime.get();
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/FinalizeNewLayoutVersionCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/FinalizeNewLayoutVersionCommandHandler.java
index bd7ec5710d9..77e152447b9 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/FinalizeNewLayoutVersionCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/FinalizeNewLayoutVersionCommandHandler.java
@@ -20,6 +20,8 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.FinalizeNewLayoutVersionCommandProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager;
@@ -42,12 +44,15 @@ public class FinalizeNewLayoutVersionCommandHandler implements CommandHandler {
LoggerFactory.getLogger(FinalizeNewLayoutVersionCommandHandler.class);
private AtomicLong invocationCount = new AtomicLong(0);
- private long totalTime;
+ private final MutableRate opsLatencyMs;
/**
* Constructs a FinalizeNewLayoutVersionCommandHandler.
*/
public FinalizeNewLayoutVersionCommandHandler() {
+ MetricsRegistry registry = new MetricsRegistry(
+ FinalizeNewLayoutVersionCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(SCMCommandProto.Type.finalizeNewLayoutVersionCommand + "Ms");
}
/**
@@ -82,7 +87,7 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
LOG.error("Exception during finalization.", e);
} finally {
long endTime = Time.monotonicNow();
- totalTime += endTime - startTime;
+ this.opsLatencyMs.add(endTime - startTime);
}
}
@@ -113,15 +118,12 @@ public int getInvocationCount() {
*/
@Override
public long getAverageRunTime() {
- if (invocationCount.get() > 0) {
- return totalTime / invocationCount.get();
- }
- return 0;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime;
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java
index 602687d7a00..030d169e9b8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconstructECContainersCommandHandler.java
@@ -36,6 +36,7 @@ public class ReconstructECContainersCommandHandler implements CommandHandler {
private final ReplicationSupervisor supervisor;
private final ECReconstructionCoordinator coordinator;
private final ConfigurationSource conf;
+ private String metricsName;
public ReconstructECContainersCommandHandler(ConfigurationSource conf,
ReplicationSupervisor supervisor,
@@ -52,8 +53,16 @@ public void handle(SCMCommand command, OzoneContainer container,
(ReconstructECContainersCommand) command;
ECReconstructionCommandInfo reconstructionCommandInfo =
new ECReconstructionCommandInfo(ecContainersCommand);
- this.supervisor.addTask(new ECReconstructionCoordinatorTask(
- coordinator, reconstructionCommandInfo));
+ ECReconstructionCoordinatorTask task = new ECReconstructionCoordinatorTask(
+ coordinator, reconstructionCommandInfo);
+ if (this.metricsName == null) {
+ this.metricsName = task.getMetricName();
+ }
+ this.supervisor.addTask(task);
+ }
+
+ public String getMetricsName() {
+ return this.metricsName;
}
@Override
@@ -63,23 +72,26 @@ public Type getCommandType() {
@Override
public int getInvocationCount() {
- return 0;
+ return this.metricsName == null ? 0 : (int) this.supervisor
+ .getReplicationRequestCount(metricsName);
}
@Override
public long getAverageRunTime() {
- return 0;
+ return this.metricsName == null ? 0 : (int) this.supervisor
+ .getReplicationRequestAvgTime(metricsName);
}
@Override
public long getTotalRunTime() {
- return 0;
+ return this.metricsName == null ? 0 : this.supervisor
+ .getReplicationRequestTotalTime(metricsName);
}
@Override
public int getQueuedCount() {
- return supervisor
- .getInFlightReplications(ECReconstructionCoordinatorTask.class);
+ return this.metricsName == null ? 0 : (int) this.supervisor
+ .getReplicationQueuedCount(metricsName);
}
public ConfigurationSource getConf() {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/RefreshVolumeUsageCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/RefreshVolumeUsageCommandHandler.java
index 3c14b2fb161..1ab31ba1c41 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/RefreshVolumeUsageCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/RefreshVolumeUsageCommandHandler.java
@@ -18,6 +18,8 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@@ -27,7 +29,6 @@
import org.slf4j.LoggerFactory;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
/**
* Command handler to refresh usage info of all volumes.
@@ -38,9 +39,12 @@ public class RefreshVolumeUsageCommandHandler implements CommandHandler {
LoggerFactory.getLogger(RefreshVolumeUsageCommandHandler.class);
private final AtomicInteger invocationCount = new AtomicInteger(0);
- private final AtomicLong totalTime = new AtomicLong(0);
+ private final MutableRate opsLatencyMs;
public RefreshVolumeUsageCommandHandler() {
+ MetricsRegistry registry = new MetricsRegistry(
+ RefreshVolumeUsageCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(Type.refreshVolumeUsageInfo + "Ms");
}
@Override
@@ -50,7 +54,7 @@ public void handle(SCMCommand command, OzoneContainer container,
invocationCount.incrementAndGet();
final long startTime = Time.monotonicNow();
container.getVolumeSet().refreshAllVolumeUsage();
- totalTime.getAndAdd(Time.monotonicNow() - startTime);
+ this.opsLatencyMs.add(Time.monotonicNow() - startTime);
}
@Override
@@ -66,14 +70,12 @@ public int getInvocationCount() {
@Override
public long getAverageRunTime() {
- final int invocations = invocationCount.get();
- return invocations == 0 ?
- 0 : totalTime.get() / invocations;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime.get();
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java
index 21b26339e23..242a4eb74be 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReplicateContainerCommandHandler.java
@@ -43,29 +43,28 @@ public class ReplicateContainerCommandHandler implements CommandHandler {
static final Logger LOG =
LoggerFactory.getLogger(ReplicateContainerCommandHandler.class);
- private int invocationCount;
-
- private long totalTime;
-
- private ConfigurationSource conf;
-
private ReplicationSupervisor supervisor;
private ContainerReplicator downloadReplicator;
private ContainerReplicator pushReplicator;
+ private String metricsName;
+
public ReplicateContainerCommandHandler(
ConfigurationSource conf,
ReplicationSupervisor supervisor,
ContainerReplicator downloadReplicator,
ContainerReplicator pushReplicator) {
- this.conf = conf;
this.supervisor = supervisor;
this.downloadReplicator = downloadReplicator;
this.pushReplicator = pushReplicator;
}
+ public String getMetricsName() {
+ return this.metricsName;
+ }
+
@Override
public void handle(SCMCommand command, OzoneContainer container,
StateContext context, SCMConnectionManager connectionManager) {
@@ -86,12 +85,16 @@ public void handle(SCMCommand command, OzoneContainer container,
downloadReplicator : pushReplicator;
ReplicationTask task = new ReplicationTask(replicateCommand, replicator);
+ if (metricsName == null) {
+ metricsName = task.getMetricName();
+ }
supervisor.addTask(task);
}
@Override
public int getQueuedCount() {
- return supervisor.getInFlightReplications(ReplicationTask.class);
+ return this.metricsName == null ? 0 : (int) this.supervisor
+ .getReplicationQueuedCount(metricsName);
}
@Override
@@ -101,19 +104,19 @@ public SCMCommandProto.Type getCommandType() {
@Override
public int getInvocationCount() {
- return this.invocationCount;
+ return this.metricsName == null ? 0 : (int) this.supervisor
+ .getReplicationRequestCount(metricsName);
}
@Override
public long getAverageRunTime() {
- if (invocationCount > 0) {
- return totalTime / invocationCount;
- }
- return 0;
+ return this.metricsName == null ? 0 : (int) this.supervisor
+ .getReplicationRequestAvgTime(metricsName);
}
@Override
public long getTotalRunTime() {
- return totalTime;
+ return this.metricsName == null ? 0 : this.supervisor
+ .getReplicationRequestTotalTime(metricsName);
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java
index 6f7f4414eeb..33563624795 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java
@@ -21,8 +21,10 @@
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SetNodeOperationalStateCommandProto;
import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@@ -39,7 +41,6 @@
import java.io.File;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
@@ -54,7 +55,7 @@ public class SetNodeOperationalStateCommandHandler implements CommandHandler {
private final ConfigurationSource conf;
private final Consumer replicationSupervisor;
private final AtomicInteger invocationCount = new AtomicInteger(0);
- private final AtomicLong totalTime = new AtomicLong(0);
+ private final MutableRate opsLatencyMs;
/**
* Set Node State command handler.
@@ -65,6 +66,9 @@ public SetNodeOperationalStateCommandHandler(ConfigurationSource conf,
Consumer replicationSupervisor) {
this.conf = conf;
this.replicationSupervisor = replicationSupervisor;
+ MetricsRegistry registry = new MetricsRegistry(
+ SetNodeOperationalStateCommandHandler.class.getSimpleName());
+ this.opsLatencyMs = registry.newRate(Type.setNodeOperationalStateCommand + "Ms");
}
/**
@@ -80,9 +84,6 @@ public void handle(SCMCommand command, OzoneContainer container,
StateContext context, SCMConnectionManager connectionManager) {
long startTime = Time.monotonicNow();
invocationCount.incrementAndGet();
- StorageContainerDatanodeProtocolProtos.SetNodeOperationalStateCommandProto
- setNodeCmdProto = null;
-
if (command.getType() != Type.setNodeOperationalStateCommand) {
LOG.warn("Skipping handling command, expected command "
+ "type {} but found {}",
@@ -91,7 +92,7 @@ public void handle(SCMCommand command, OzoneContainer container,
}
SetNodeOperationalStateCommand setNodeCmd =
(SetNodeOperationalStateCommand) command;
- setNodeCmdProto = setNodeCmd.getProto();
+ SetNodeOperationalStateCommandProto setNodeCmdProto = setNodeCmd.getProto();
DatanodeDetails dni = context.getParent().getDatanodeDetails();
HddsProtos.NodeOperationalState state =
setNodeCmdProto.getNodeOperationalState();
@@ -106,7 +107,7 @@ public void handle(SCMCommand command, OzoneContainer container,
// handler interface.
}
replicationSupervisor.accept(state);
- totalTime.addAndGet(Time.monotonicNow() - startTime);
+ this.opsLatencyMs.add(Time.monotonicNow() - startTime);
}
// TODO - this duplicates code in HddsDatanodeService and InitDatanodeState
@@ -125,8 +126,7 @@ private void persistDatanodeDetails(DatanodeDetails dnDetails)
* @return Type
*/
@Override
- public StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type
- getCommandType() {
+ public Type getCommandType() {
return Type.setNodeOperationalStateCommand;
}
@@ -147,14 +147,12 @@ public int getInvocationCount() {
*/
@Override
public long getAverageRunTime() {
- final int invocations = invocationCount.get();
- return invocations == 0 ?
- 0 : totalTime.get() / invocations;
+ return (long) this.opsLatencyMs.lastStat().mean();
}
@Override
public long getTotalRunTime() {
- return totalTime.get();
+ return (long) this.opsLatencyMs.lastStat().total();
}
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
index e702b1e6e15..968c9b9a6e6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -17,6 +17,7 @@
package org.apache.hadoop.ozone.container.common.states.endpoint;
import java.io.IOException;
+import java.net.BindException;
import java.util.concurrent.Callable;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -104,7 +105,7 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
LOG.debug("Cannot execute GetVersion task as endpoint state machine " +
"is in {} state", rpcEndPoint.getState());
}
- } catch (DiskOutOfSpaceException ex) {
+ } catch (DiskOutOfSpaceException | BindException ex) {
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
} catch (IOException ex) {
rpcEndPoint.logIfNeeded(ex);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
index 9c3f29d0f0c..5f1914402d0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
@@ -20,7 +20,6 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc;
-import org.apache.hadoop.hdds.utils.IOUtils;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.ratis.grpc.util.ZeroCopyMessageMarshaller;
import org.apache.ratis.thirdparty.com.google.protobuf.MessageLite;
@@ -31,7 +30,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.InputStream;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc.getSendMethod;
@@ -45,28 +43,20 @@ public class GrpcXceiverService extends
LOG = LoggerFactory.getLogger(GrpcXceiverService.class);
private final ContainerDispatcher dispatcher;
- private final boolean zeroCopyEnabled;
private final ZeroCopyMessageMarshaller
zeroCopyMessageMarshaller = new ZeroCopyMessageMarshaller<>(
ContainerCommandRequestProto.getDefaultInstance());
- public GrpcXceiverService(ContainerDispatcher dispatcher,
- boolean zeroCopyEnabled) {
+ public GrpcXceiverService(ContainerDispatcher dispatcher) {
this.dispatcher = dispatcher;
- this.zeroCopyEnabled = zeroCopyEnabled;
}
/**
- * Bind service with zerocopy marshaller equipped for the `send` API if
- * zerocopy is enabled.
+ * Bind service with zerocopy marshaller equipped for the `send` API.
* @return service definition.
*/
public ServerServiceDefinition bindServiceWithZeroCopy() {
ServerServiceDefinition orig = super.bindService();
- if (!zeroCopyEnabled) {
- LOG.info("Zerocopy is not enabled.");
- return orig;
- }
ServerServiceDefinition.Builder builder =
ServerServiceDefinition.builder(orig.getServiceDescriptor().getName());
@@ -117,10 +107,7 @@ public void onNext(ContainerCommandRequestProto request) {
isClosed.set(true);
responseObserver.onError(e);
} finally {
- InputStream popStream = zeroCopyMessageMarshaller.popStream(request);
- if (popStream != null) {
- IOUtils.close(LOG, popStream);
- }
+ zeroCopyMessageMarshaller.release(request);
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
index ad9c5c9d9ca..0d95ac25eda 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.ozone.container.common.transport.server;
import java.io.IOException;
+import java.net.BindException;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
@@ -29,7 +30,6 @@
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -66,9 +66,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED_DEFAULT;
-
/**
* Creates a Grpc server endpoint that acts as the communication layer for
* Ozone containers.
@@ -134,13 +131,9 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails,
eventLoopGroup = new NioEventLoopGroup(poolSize / 10, factory);
channelType = NioServerSocketChannel.class;
}
- final boolean zeroCopyEnabled = conf.getBoolean(
- OZONE_EC_GRPC_ZERO_COPY_ENABLED,
- OZONE_EC_GRPC_ZERO_COPY_ENABLED_DEFAULT);
LOG.info("GrpcServer channel type {}", channelType.getSimpleName());
- GrpcXceiverService xceiverService = new GrpcXceiverService(dispatcher,
- zeroCopyEnabled);
+ GrpcXceiverService xceiverService = new GrpcXceiverService(dispatcher);
NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forPort(port)
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
.bossEventLoopGroup(eventLoopGroup)
@@ -185,7 +178,16 @@ public HddsProtos.ReplicationType getServerType() {
@Override
public void start() throws IOException {
if (!isStarted) {
- server.start();
+ try {
+ server.start();
+ } catch (IOException e) {
+ LOG.error("Error while starting the server", e);
+ if (e.getMessage().contains("Failed to bind to address")) {
+ throw new BindException(e.getMessage());
+ } else {
+ throw e;
+ }
+ }
int realPort = server.getPort();
if (port == 0) {
@@ -195,9 +197,7 @@ public void start() throws IOException {
}
//register the real port to the datanode details.
- datanodeDetails.setPort(DatanodeDetails
- .newPort(Name.STANDALONE,
- realPort));
+ datanodeDetails.setPort(DatanodeDetails.newStandalonePort(realPort));
isStarted = true;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index be566f84fc9..23be4138b60 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -41,8 +41,9 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
-import java.util.stream.Collectors;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -64,26 +65,25 @@
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.common.utils.BufferUtils;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.util.Time;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.proto.RaftProtos.StateMachineEntryProto;
import org.apache.ratis.proto.RaftProtos.LogEntryProto;
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
+import org.apache.ratis.proto.RaftProtos.StateMachineEntryProto;
import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
import org.apache.ratis.protocol.Message;
import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftGroup;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftGroupMemberId;
-import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.protocol.exceptions.StateMachineException;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.protocol.TermIndex;
@@ -98,10 +98,10 @@
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat;
+import org.apache.ratis.util.JavaUtils;
import org.apache.ratis.util.LifeCycle;
import org.apache.ratis.util.TaskQueue;
import org.apache.ratis.util.function.CheckedSupplier;
-import org.apache.ratis.util.JavaUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -185,7 +185,6 @@ long getStartTime() {
private final SimpleStateMachineStorage storage =
new SimpleStateMachineStorage();
- private final RaftGroupId gid;
private final ContainerDispatcher dispatcher;
private final ContainerController containerController;
private final XceiverServerRatis ratisServer;
@@ -205,6 +204,7 @@ long getStartTime() {
private final boolean waitOnBothFollowers;
private final HddsDatanodeService datanodeService;
private static Semaphore semaphore = new Semaphore(1);
+ private final AtomicBoolean peersValidated;
/**
* CSM metrics.
@@ -220,7 +220,6 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI
ConfigurationSource conf,
String threadNamePrefix) {
this.datanodeService = hddsDatanodeService;
- this.gid = gid;
this.dispatcher = dispatcher;
this.containerController = containerController;
this.ratisServer = ratisServer;
@@ -256,6 +255,7 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI
HDDS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT);
applyTransactionSemaphore = new Semaphore(maxPendingApplyTransactions);
stateMachineHealthy = new AtomicBoolean(true);
+ this.peersValidated = new AtomicBoolean(false);
ThreadFactory threadFactory = new ThreadFactoryBuilder()
.setNameFormat(
@@ -269,6 +269,19 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI
}
+ private void validatePeers() throws IOException {
+ if (this.peersValidated.get()) {
+ return;
+ }
+ final RaftGroup group = ratisServer.getServerDivision(getGroupId()).getGroup();
+ final RaftPeerId selfId = ratisServer.getServer().getId();
+ if (group.getPeer(selfId) == null) {
+ throw new StorageContainerException("Current datanode " + selfId + " is not a member of " + group,
+ ContainerProtos.Result.INVALID_CONFIG);
+ }
+ peersValidated.set(true);
+ }
+
@Override
public StateMachineStorage getStateMachineStorage() {
return storage;
@@ -284,8 +297,9 @@ public void initialize(
throws IOException {
super.initialize(server, id, raftStorage);
storage.init(raftStorage);
- ratisServer.notifyGroupAdd(gid);
+ ratisServer.notifyGroupAdd(id);
+ LOG.info("{}: initialize {}", server.getId(), id);
loadSnapshot(storage.getLatestSnapshot());
}
@@ -294,7 +308,7 @@ private long loadSnapshot(SingleFileSnapshotInfo snapshot)
if (snapshot == null) {
TermIndex empty = TermIndex.valueOf(0, RaftLog.INVALID_LOG_INDEX);
LOG.info("{}: The snapshot info is null. Setting the last applied index " +
- "to:{}", gid, empty);
+ "to:{}", getGroupId(), empty);
setLastAppliedTermIndex(empty);
return empty.getIndex();
}
@@ -302,7 +316,7 @@ private long loadSnapshot(SingleFileSnapshotInfo snapshot)
final File snapshotFile = snapshot.getFile().getPath().toFile();
final TermIndex last =
SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
- LOG.info("{}: Setting the last applied index to {}", gid, last);
+ LOG.info("{}: Setting the last applied index to {}", getGroupId(), last);
setLastAppliedTermIndex(last);
// initialize the dispatcher with snapshot so that it build the missing
@@ -352,7 +366,7 @@ public long takeSnapshot() throws IOException {
long startTime = Time.monotonicNow();
if (!isStateMachineHealthy()) {
String msg =
- "Failed to take snapshot " + " for " + gid + " as the stateMachine"
+ "Failed to take snapshot " + " for " + getGroupId() + " as the stateMachine"
+ " is unhealthy. The last applied index is at " + ti;
StateMachineException sme = new StateMachineException(msg);
LOG.error(msg);
@@ -361,19 +375,19 @@ public long takeSnapshot() throws IOException {
if (ti != null && ti.getIndex() != RaftLog.INVALID_LOG_INDEX) {
final File snapshotFile =
storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
- LOG.info("{}: Taking a snapshot at:{} file {}", gid, ti, snapshotFile);
+ LOG.info("{}: Taking a snapshot at:{} file {}", getGroupId(), ti, snapshotFile);
try (FileOutputStream fos = new FileOutputStream(snapshotFile)) {
persistContainerSet(fos);
fos.flush();
// make sure the snapshot file is synced
fos.getFD().sync();
} catch (IOException ioe) {
- LOG.error("{}: Failed to write snapshot at:{} file {}", gid, ti,
+ LOG.error("{}: Failed to write snapshot at:{} file {}", getGroupId(), ti,
snapshotFile);
throw ioe;
}
LOG.info("{}: Finished taking a snapshot at:{} file:{} took: {} ms",
- gid, ti, snapshotFile, (Time.monotonicNow() - startTime));
+ getGroupId(), ti, snapshotFile, (Time.monotonicNow() - startTime));
return ti.getIndex();
}
return -1;
@@ -387,7 +401,7 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftPeerRole rol
final StateMachineLogEntryProto stateMachineLogEntry = entry.getStateMachineLogEntry();
final ContainerCommandRequestProto logProto;
try {
- logProto = getContainerCommandRequestProto(gid, stateMachineLogEntry.getLogData());
+ logProto = getContainerCommandRequestProto(getGroupId(), stateMachineLogEntry.getLogData());
} catch (InvalidProtocolBufferException e) {
trx.setException(e);
return trx;
@@ -414,7 +428,7 @@ public TransactionContext startTransaction(RaftClientRequest request)
long startTime = Time.monotonicNowNanos();
final ContainerCommandRequestProto proto =
message2ContainerCommandRequestProto(request.getMessage());
- Preconditions.checkArgument(request.getRaftGroupId().equals(gid));
+ Preconditions.checkArgument(request.getRaftGroupId().equals(getGroupId()));
final TransactionContext.Builder builder = TransactionContext.newBuilder()
.setClientRequest(request)
@@ -450,7 +464,7 @@ public TransactionContext startTransaction(RaftClientRequest request)
final WriteChunkRequestProto.Builder commitWriteChunkProto = WriteChunkRequestProto.newBuilder(write)
.clearData();
protoBuilder.setWriteChunk(commitWriteChunkProto)
- .setPipelineID(gid.getUuid().toString())
+ .setPipelineID(getGroupId().getUuid().toString())
.setTraceID(proto.getTraceID());
builder.setStateMachineData(write.getData());
@@ -492,20 +506,20 @@ private static ContainerCommandRequestProto getContainerCommandRequestProto(
private ContainerCommandRequestProto message2ContainerCommandRequestProto(
Message message) throws InvalidProtocolBufferException {
- return ContainerCommandRequestMessage.toProto(message.getContent(), gid);
+ return ContainerCommandRequestMessage.toProto(message.getContent(), getGroupId());
}
private ContainerCommandResponseProto dispatchCommand(
ContainerCommandRequestProto requestProto, DispatcherContext context) {
if (LOG.isTraceEnabled()) {
- LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid,
+ LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", getGroupId(),
requestProto.getCmdType(), requestProto.getContainerID(),
requestProto.getPipelineID(), requestProto.getTraceID());
}
ContainerCommandResponseProto response =
dispatcher.dispatch(requestProto, context);
if (LOG.isTraceEnabled()) {
- LOG.trace("{}: response {}", gid, response);
+ LOG.trace("{}: response {}", getGroupId(), response);
}
return response;
}
@@ -532,7 +546,7 @@ private CompletableFuture writeStateMachineData(
RaftServer server = ratisServer.getServer();
Preconditions.checkArgument(!write.getData().isEmpty());
try {
- if (server.getDivision(gid).getInfo().isLeader()) {
+ if (server.getDivision(getGroupId()).getInfo().isLeader()) {
stateMachineDataCache.put(entryIndex, write.getData());
}
} catch (InterruptedException ioe) {
@@ -560,7 +574,7 @@ private CompletableFuture writeStateMachineData(
return dispatchCommand(requestProto, context);
} catch (Exception e) {
LOG.error("{}: writeChunk writeStateMachineData failed: blockId" +
- "{} logIndex {} chunkName {}", gid, write.getBlockID(),
+ "{} logIndex {} chunkName {}", getGroupId(), write.getBlockID(),
entryIndex, write.getChunkData().getChunkName(), e);
metrics.incNumWriteDataFails();
// write chunks go in parallel. It's possible that one write chunk
@@ -574,7 +588,7 @@ private CompletableFuture writeStateMachineData(
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
if (LOG.isDebugEnabled()) {
LOG.debug("{}: writeChunk writeStateMachineData : blockId" +
- "{} logIndex {} chunkName {}", gid, write.getBlockID(),
+ "{} logIndex {} chunkName {}", getGroupId(), write.getBlockID(),
entryIndex, write.getChunkData().getChunkName());
}
// Remove the future once it finishes execution from the
@@ -588,7 +602,7 @@ private CompletableFuture writeStateMachineData(
&& r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) {
StorageContainerException sce =
new StorageContainerException(r.getMessage(), r.getResult());
- LOG.error(gid + ": writeChunk writeStateMachineData failed: blockId" +
+ LOG.error(getGroupId() + ": writeChunk writeStateMachineData failed: blockId" +
write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
write.getChunkData().getChunkName() + " Error message: " +
r.getMessage() + " Container Result: " + r.getResult());
@@ -602,7 +616,7 @@ private CompletableFuture writeStateMachineData(
metrics.incNumBytesWrittenCount(
requestProto.getWriteChunk().getChunkData().getLen());
if (LOG.isDebugEnabled()) {
- LOG.debug(gid +
+ LOG.debug(getGroupId() +
": writeChunk writeStateMachineData completed: blockId" +
write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
write.getChunkData().getChunkName());
@@ -623,7 +637,7 @@ private StateMachine.DataChannel getStreamDataChannel(
DispatcherContext context) throws StorageContainerException {
if (LOG.isDebugEnabled()) {
LOG.debug("{}: getStreamDataChannel {} containerID={} pipelineID={} " +
- "traceID={}", gid, requestProto.getCmdType(),
+ "traceID={}", getGroupId(), requestProto.getCmdType(),
requestProto.getContainerID(), requestProto.getPipelineID(),
requestProto.getTraceID());
}
@@ -782,7 +796,7 @@ private ByteString readStateMachineData(
new StorageContainerException(response.getMessage(),
response.getResult());
LOG.error("gid {} : ReadStateMachine failed. cmd {} logIndex {} msg : "
- + "{} Container Result: {}", gid, response.getCmdType(), index,
+ + "{} Container Result: {}", getGroupId(), response.getCmdType(), index,
response.getMessage(), response.getResult());
stateMachineHealthy.set(false);
throw sce;
@@ -818,11 +832,9 @@ private ByteString readStateMachineData(
*/
@Override
public CompletableFuture flush(long index) {
- List> futureList =
- writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index)
- .map(Map.Entry::getValue).collect(Collectors.toList());
return CompletableFuture.allOf(
- futureList.toArray(new CompletableFuture[futureList.size()]));
+ writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index)
+ .map(Map.Entry::getValue).toArray(CompletableFuture[]::new));
}
/**
@@ -859,7 +871,7 @@ public CompletableFuture read(LogEntryProto entry, TransactionContex
.map(TransactionContext::getStateMachineContext)
.orElse(null);
final ContainerCommandRequestProto requestProto = context != null ? context.getLogProto()
- : getContainerCommandRequestProto(gid, entry.getStateMachineLogEntry().getLogData());
+ : getContainerCommandRequestProto(getGroupId(), entry.getStateMachineLogEntry().getLogData());
if (requestProto.getCmdType() != Type.WriteChunk) {
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
@@ -877,7 +889,7 @@ public CompletableFuture read(LogEntryProto entry, TransactionContex
return future;
} catch (Exception e) {
metrics.incNumReadStateMachineFails();
- LOG.error("{} unable to read stateMachineData:", gid, e);
+ LOG.error("{} unable to read stateMachineData:", getGroupId(), e);
return completeExceptionally(e);
}
}
@@ -923,7 +935,7 @@ public void notifyServerShutdown(RaftProtos.RoleInfoProto roleInfo, boolean allS
// from `HddsDatanodeService.stop()`, otherwise, it indicates this `close` originates from ratis.
if (allServer) {
if (datanodeService != null && !datanodeService.isStopped()) {
- LOG.info("{} is closed by ratis", gid);
+ LOG.info("{} is closed by ratis", getGroupId());
if (semaphore.tryAcquire()) {
// run with a different thread, so this raft group can be closed
Runnable runnable = () -> {
@@ -955,7 +967,7 @@ public void notifyServerShutdown(RaftProtos.RoleInfoProto roleInfo, boolean allS
CompletableFuture.runAsync(runnable);
}
} else {
- LOG.info("{} is closed by HddsDatanodeService", gid);
+ LOG.info("{} is closed by HddsDatanodeService", getGroupId());
}
}
}
@@ -967,6 +979,11 @@ private CompletableFuture applyTransaction(
final CheckedSupplier task
= () -> {
try {
+ try {
+ this.validatePeers();
+ } catch (StorageContainerException e) {
+ return ContainerUtils.logAndReturnError(LOG, e, request);
+ }
long timeNow = Time.monotonicNowNanos();
long queueingDelay = timeNow - context.getStartTime();
metrics.recordQueueingDelay(request.getCmdType(), queueingDelay);
@@ -986,14 +1003,17 @@ private CompletableFuture applyTransaction(
private void removeStateMachineDataIfNeeded(long index) {
if (waitOnBothFollowers) {
try {
- RaftServer.Division division = ratisServer.getServer().getDivision(gid);
+ RaftServer.Division division = ratisServer.getServer().getDivision(getGroupId());
if (division.getInfo().isLeader()) {
- long minIndex = Arrays.stream(division.getInfo()
- .getFollowerNextIndices()).min().getAsLong();
- LOG.debug("Removing data corresponding to log index {} min index {} "
- + "from cache", index, minIndex);
- removeCacheDataUpTo(Math.min(minIndex, index));
+ Arrays.stream(division.getInfo()
+ .getFollowerNextIndices()).min().ifPresent(minIndex -> {
+ removeCacheDataUpTo(Math.min(minIndex, index));
+ LOG.debug("Removing data corresponding to log index {} min index {} "
+ + "from cache", index, minIndex);
+ });
}
+ } catch (RuntimeException e) {
+ throw e;
} catch (Exception e) {
throw new RuntimeException(e);
}
@@ -1044,7 +1064,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
CompletableFuture applyTransactionFuture =
new CompletableFuture<>();
final Consumer exceptionHandler = e -> {
- LOG.error(gid + ": failed to applyTransaction at logIndex " + index
+ LOG.error(getGroupId() + ": failed to applyTransaction at logIndex " + index
+ " for " + requestProto.getCmdType(), e);
stateMachineHealthy.compareAndSet(true, false);
metrics.incNumApplyTransactionsFails();
@@ -1072,7 +1092,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
new StorageContainerException(r.getMessage(), r.getResult());
LOG.error(
"gid {} : ApplyTransaction failed. cmd {} logIndex {} msg : "
- + "{} Container Result: {}", gid, r.getCmdType(), index,
+ + "{} Container Result: {}", getGroupId(), r.getCmdType(), index,
r.getMessage(), r.getResult());
metrics.incNumApplyTransactionsFails();
// Since the applyTransaction now is completed exceptionally,
@@ -1081,12 +1101,12 @@ public CompletableFuture applyTransaction(TransactionContext trx) {
// shutdown.
applyTransactionFuture.completeExceptionally(sce);
stateMachineHealthy.compareAndSet(true, false);
- ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole());
+ ratisServer.handleApplyTransactionFailure(getGroupId(), trx.getServerRole());
} else {
if (LOG.isDebugEnabled()) {
LOG.debug(
"gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
- + "{} Container Result: {}", gid, r.getCmdType(), index,
+ + "{} Container Result: {}", getGroupId(), r.getCmdType(), index,
r.getMessage(), r.getResult());
}
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
@@ -1164,25 +1184,25 @@ public void evictStateMachineCache() {
@Override
public void notifyFollowerSlowness(RoleInfoProto roleInfoProto, RaftPeer follower) {
- ratisServer.handleFollowerSlowness(gid, roleInfoProto, follower);
+ ratisServer.handleFollowerSlowness(getGroupId(), roleInfoProto, follower);
}
@Override
public void notifyExtendedNoLeader(RoleInfoProto roleInfoProto) {
- ratisServer.handleNoLeader(gid, roleInfoProto);
+ ratisServer.handleNoLeader(getGroupId(), roleInfoProto);
}
@Override
public void notifyLogFailed(Throwable t, LogEntryProto failedEntry) {
- LOG.error("{}: {} {}", gid, TermIndex.valueOf(failedEntry),
+ LOG.error("{}: {} {}", getGroupId(), TermIndex.valueOf(failedEntry),
toStateMachineLogEntryString(failedEntry.getStateMachineLogEntry()), t);
- ratisServer.handleNodeLogFailure(gid, t);
+ ratisServer.handleNodeLogFailure(getGroupId(), t);
}
@Override
public CompletableFuture notifyInstallSnapshotFromLeader(
RoleInfoProto roleInfoProto, TermIndex firstTermIndexInLog) {
- ratisServer.handleInstallSnapshotFromLeader(gid, roleInfoProto,
+ ratisServer.handleInstallSnapshotFromLeader(getGroupId(), roleInfoProto,
firstTermIndexInLog);
final CompletableFuture future = new CompletableFuture<>();
future.complete(firstTermIndexInLog);
@@ -1191,7 +1211,7 @@ public CompletableFuture notifyInstallSnapshotFromLeader(
@Override
public void notifyGroupRemove() {
- ratisServer.notifyGroupRemove(gid);
+ ratisServer.notifyGroupRemove(getGroupId());
// Make best effort to quasi-close all the containers on group removal.
// Containers already in terminal state like CLOSED or UNHEALTHY will not
// be affected.
@@ -1199,7 +1219,7 @@ public void notifyGroupRemove() {
try {
containerController.markContainerForClose(cid);
containerController.quasiCloseContainer(cid,
- "Ratis group removed. Group id: " + gid);
+ "Ratis group removed. Group id: " + getGroupId());
} catch (IOException e) {
LOG.debug("Failed to quasi-close container {}", cid);
}
@@ -1221,7 +1241,7 @@ public void notifyLeaderChanged(RaftGroupMemberId groupMemberId,
@Override
public String toStateMachineLogEntryString(StateMachineLogEntryProto proto) {
- return smProtoToString(gid, containerController, proto);
+ return smProtoToString(getGroupId(), containerController, proto);
}
public static String smProtoToString(RaftGroupId gid,
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
index c58aab2e5ba..5fced0e39b3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
@@ -29,6 +29,7 @@
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.annotation.InterfaceStability;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
@@ -36,6 +37,7 @@
import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
import org.apache.hadoop.ozone.container.common.utils.RawDB;
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3;
import org.apache.hadoop.util.Time;
@@ -44,6 +46,7 @@
import jakarta.annotation.Nullable;
+import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_DATANODE_IO_METRICS_PERCENTILES_INTERVALS_SECONDS_KEY;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_NAME;
import static org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil.initPerDiskDBStore;
@@ -80,6 +83,8 @@ public class HddsVolume extends StorageVolume {
private final VolumeIOStats volumeIOStats;
private final VolumeInfoMetrics volumeInfoMetrics;
+ private ContainerController controller;
+
private final AtomicLong committedBytes = new AtomicLong(); // till Open containers become full
// Mentions the type of volume
@@ -119,8 +124,10 @@ private HddsVolume(Builder b) throws IOException {
if (!b.getFailedVolume() && getVolumeInfo().isPresent()) {
this.setState(VolumeState.NOT_INITIALIZED);
+ ConfigurationSource conf = getConf();
+ int[] intervals = conf.getInts(OZONE_DATANODE_IO_METRICS_PERCENTILES_INTERVALS_SECONDS_KEY);
this.volumeIOStats = new VolumeIOStats(b.getVolumeRootStr(),
- this.getStorageDir().toString());
+ this.getStorageDir().toString(), intervals);
this.volumeInfoMetrics =
new VolumeInfoMetrics(b.getVolumeRootStr(), this);
@@ -382,6 +389,17 @@ public void loadDbStore(boolean readOnly) throws IOException {
getStorageID());
}
+ public void setController(ContainerController controller) {
+ this.controller = controller;
+ }
+
+ public long getContainers() {
+ if (controller != null) {
+ return controller.getContainerCount(this);
+ }
+ return 0;
+ }
+
/**
* Pick a DbVolume for HddsVolume and init db instance.
* Use the HddsVolume directly if no DbVolume found.
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
index e195b127d49..9afea8e6b0c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
@@ -44,6 +44,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
+import org.apache.ratis.util.function.CheckedRunnable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -84,7 +85,7 @@ public class MutableVolumeSet implements VolumeSet {
private String clusterID;
private final StorageVolumeChecker volumeChecker;
- private Runnable failedVolumeListener;
+ private CheckedRunnable failedVolumeListener;
private StateContext context;
private final StorageVolumeFactory volumeFactory;
private final StorageVolume.VolumeType volumeType;
@@ -132,7 +133,7 @@ public MutableVolumeSet(String dnUuid, String clusterID,
initializeVolumeSet();
}
- public void setFailedVolumeListener(Runnable runnable) {
+ public void setFailedVolumeListener(CheckedRunnable runnable) {
failedVolumeListener = runnable;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
index e22addd354f..2ce19c3bf19 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
@@ -21,7 +21,10 @@
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
+import org.apache.hadoop.metrics2.lib.MutableRate;
/**
* This class is used to track Volume IO stats for each HDDS Volume.
@@ -29,12 +32,23 @@
public class VolumeIOStats {
private String metricsSourceName = VolumeIOStats.class.getSimpleName();
private String storageDirectory;
- private @Metric MutableCounterLong readBytes;
- private @Metric MutableCounterLong readOpCount;
- private @Metric MutableCounterLong writeBytes;
- private @Metric MutableCounterLong writeOpCount;
- private @Metric MutableCounterLong readTime;
- private @Metric MutableCounterLong writeTime;
+ private final MetricsRegistry registry = new MetricsRegistry("VolumeIOStats");
+ @Metric
+ private MutableCounterLong readBytes;
+ @Metric
+ private MutableCounterLong readOpCount;
+ @Metric
+ private MutableCounterLong writeBytes;
+ @Metric
+ private MutableCounterLong writeOpCount;
+ @Metric
+ private MutableRate readTime;
+ @Metric
+ private MutableQuantiles[] readLatencyQuantiles;
+ @Metric
+ private MutableRate writeTime;
+ @Metric
+ private MutableQuantiles[] writeLatencyQuantiles;
@Deprecated
public VolumeIOStats() {
@@ -44,9 +58,24 @@ public VolumeIOStats() {
/**
* @param identifier Typically, path to volume root. e.g. /data/hdds
*/
- public VolumeIOStats(String identifier, String storageDirectory) {
+ public VolumeIOStats(String identifier, String storageDirectory, int[] intervals) {
this.metricsSourceName += '-' + identifier;
this.storageDirectory = storageDirectory;
+
+ // Try initializing `readLatencyQuantiles` and `writeLatencyQuantiles`
+ if (intervals != null && intervals.length > 0) {
+ final int length = intervals.length;
+ readLatencyQuantiles = new MutableQuantiles[intervals.length];
+ writeLatencyQuantiles = new MutableQuantiles[intervals.length];
+ for (int i = 0; i < length; i++) {
+ readLatencyQuantiles[i] = registry.newQuantiles(
+ "readLatency" + intervals[i] + "s",
+ "Read Data File Io Latency in ms", "ops", "latency", intervals[i]);
+ writeLatencyQuantiles[i] = registry.newQuantiles(
+ "writeLatency" + intervals[i] + "s",
+ "Write Data File Io Latency in ms", "ops", "latency", intervals[i]);
+ }
+ }
init();
}
@@ -99,7 +128,10 @@ public void incWriteOpCount() {
* @param time
*/
public void incReadTime(long time) {
- readTime.incr(time);
+ readTime.add(time);
+ for (MutableQuantiles q : readLatencyQuantiles) {
+ q.add(time);
+ }
}
/**
@@ -107,7 +139,10 @@ public void incReadTime(long time) {
* @param time
*/
public void incWriteTime(long time) {
- writeTime.incr(time);
+ writeTime.add(time);
+ for (MutableQuantiles q : writeLatencyQuantiles) {
+ q.add(time);
+ }
}
/**
@@ -147,7 +182,7 @@ public long getWriteOpCount() {
* @return long
*/
public long getReadTime() {
- return readTime.value();
+ return (long) readTime.lastStat().total();
}
/**
@@ -155,7 +190,7 @@ public long getReadTime() {
* @return long
*/
public long getWriteTime() {
- return writeTime.value();
+ return (long) writeTime.lastStat().total();
}
@Metric
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfoMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfoMetrics.java
index 68140600db9..cd31b8063d3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfoMetrics.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfoMetrics.java
@@ -37,6 +37,7 @@ public class VolumeInfoMetrics {
private final HddsVolume volume;
@Metric("Returns the RocksDB compact times of the Volume")
private MutableRate dbCompactLatency;
+ private long containers;
/**
* @param identifier Typically, path to volume root. E.g. /data/hdds
@@ -153,4 +154,11 @@ public void dbCompactTimesNanoSecondsIncr(long time) {
dbCompactLatency.add(time);
}
+ /**
+ * Return the Container Count of the Volume.
+ */
+ @Metric("Returns the Container Count of the Volume")
+ public long getContainers() {
+ return volume.getContainers();
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
index 733dc7964f1..34ba66c91bb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.ozone.container.common.volume;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.conf.ConfigurationException;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.StorageSize;
import org.apache.hadoop.hdds.conf.StorageUnit;
@@ -220,9 +221,8 @@ private static long getReserved(ConfigurationSource conf, String rootDir,
for (String reserve : reserveList) {
String[] words = reserve.split(":");
if (words.length < 2) {
- LOG.error("Reserved space should be configured in a pair, but current value is {}",
- reserve);
- continue;
+ throw new ConfigurationException("hdds.datanode.dir.du.reserved - " +
+ "Reserved space should be configured in a pair, but current value is " + reserve);
}
try {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java
index 487e6d37b28..95b7d06167f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java
@@ -18,11 +18,13 @@
package org.apache.hadoop.ozone.container.ec.reconstruction;
import com.google.common.collect.ImmutableList;
+import jakarta.annotation.Nonnull;
import org.apache.commons.collections.map.SingletonMap;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.client.ClientTrustManager;
@@ -34,8 +36,6 @@
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State;
-import jakarta.annotation.Nonnull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -44,7 +44,6 @@
import java.util.List;
import java.util.Objects;
import java.util.Set;
-import java.util.stream.Collectors;
/**
* This class wraps necessary container-level rpc calls
@@ -93,14 +92,11 @@ public BlockData[] listBlock(long containerId, DatanodeDetails dn,
try {
return BlockData.getFromProtoBuf(i);
} catch (IOException e) {
- LOG.debug("Failed while converting to protobuf BlockData. Returning"
- + " null for listBlock from DN: " + dn,
- e);
+ LOG.debug("Failed while converting to protobuf BlockData. Returning null for listBlock from DN: {}", dn, e);
// TODO: revisit here.
return null;
}
- }).collect(Collectors.toList())
- .toArray(new BlockData[blockDataList.size()]);
+ }).toArray(BlockData[]::new);
} finally {
this.xceiverClientManager.releaseClient(xceiverClient, false);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index d587748e6f8..716eb440530 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -103,6 +103,7 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.CLOSED;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.QUASI_CLOSED;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.UNHEALTHY;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.RECOVERING;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CLOSED_CONTAINER_IO;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_ALREADY_EXISTS;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_INTERNAL_ERROR;
@@ -110,6 +111,7 @@
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.DELETE_ON_NON_EMPTY_CONTAINER;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.DELETE_ON_OPEN_CONTAINER;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.GET_SMALL_FILE_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.INVALID_ARGUMENT;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.INVALID_CONTAINER_STATE;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.IO_EXCEPTION;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.PUT_SMALL_FILE_ERROR;
@@ -132,12 +134,8 @@
import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.putBlockResponseSuccess;
import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest;
import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
- .ContainerDataProto.State.RECOVERING;
-
-import org.apache.hadoop.ozone.container.common.interfaces.ScanResult;
-import static org.apache.hadoop.ozone.ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST;
import static org.apache.hadoop.ozone.OzoneConsts.INCREMENTAL_CHUNK_LIST;
+import org.apache.hadoop.ozone.container.common.interfaces.ScanResult;
import org.apache.hadoop.util.Time;
import org.apache.ratis.statemachine.StateMachine;
@@ -260,6 +258,15 @@ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler,
ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
Type cmdType = request.getCmdType();
+ // Validate the request has been made to the correct datanode with the node id matching.
+ if (kvContainer != null) {
+ try {
+ handler.validateRequestDatanodeId(kvContainer.getContainerData().getReplicaIndex(),
+ request.getDatanodeUuid());
+ } catch (StorageContainerException e) {
+ return ContainerUtils.logAndReturnError(LOG, e, request);
+ }
+ }
switch (cmdType) {
case CreateContainer:
@@ -378,7 +385,23 @@ ContainerCommandResponseProto handleCreateContainer(
" already exists", null, CONTAINER_ALREADY_EXISTS), request);
}
+ try {
+ this.validateRequestDatanodeId(request.getCreateContainer().hasReplicaIndex() ?
+ request.getCreateContainer().getReplicaIndex() : null, request.getDatanodeUuid());
+ } catch (StorageContainerException e) {
+ return ContainerUtils.logAndReturnError(LOG, e, request);
+ }
+
long containerID = request.getContainerID();
+ State containerState = request.getCreateContainer().getState();
+
+ if (containerState != RECOVERING) {
+ try {
+ containerSet.ensureContainerNotMissing(containerID, containerState);
+ } catch (StorageContainerException ex) {
+ return ContainerUtils.logAndReturnError(LOG, ex, request);
+ }
+ }
ContainerLayoutVersion layoutVersion =
ContainerLayoutVersion.getConfiguredVersion(conf);
@@ -403,7 +426,11 @@ ContainerCommandResponseProto handleCreateContainer(
try {
if (containerSet.getContainer(containerID) == null) {
newContainer.create(volumeSet, volumeChoosingPolicy, clusterId);
- created = containerSet.addContainer(newContainer);
+ if (RECOVERING == newContainer.getContainerState()) {
+ created = containerSet.addContainerByOverwriteMissingContainer(newContainer);
+ } else {
+ created = containerSet.addContainer(newContainer);
+ }
} else {
// The create container request for an already existing container can
// arrive in case the ContainerStateMachine reapplies the transaction
@@ -608,6 +635,8 @@ ContainerCommandResponseProto handlePutBlock(
endOfBlock = true;
}
+ // Note: checksum held inside blockData. But no extra checksum validation here with handlePutBlock.
+
long bcsId =
dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
blockData.setBlockCommitSequenceId(bcsId);
@@ -723,15 +752,6 @@ ContainerCommandResponseProto handleGetContainerChecksumInfo(
return getGetContainerMerkleTreeResponse(request, checksumTree);
}
- /**
- * Checks if a replicaIndex needs to be checked based on the client version for a request.
- * @param request ContainerCommandRequest object.
- * @return true if the validation is required for the client version else false.
- */
- private boolean replicaIndexCheckRequired(ContainerCommandRequestProto request) {
- return request.hasVersion() && request.getVersion() >= EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue();
- }
-
/**
* Handle Get Block operation. Calls BlockManager to process the request.
*/
@@ -750,9 +770,7 @@ ContainerCommandResponseProto handleGetBlock(
try {
BlockID blockID = BlockID.getFromProtobuf(
request.getGetBlock().getBlockID());
- if (replicaIndexCheckRequired(request)) {
- BlockUtils.verifyReplicaIdx(kvContainer, blockID);
- }
+ BlockUtils.verifyReplicaIdx(kvContainer, blockID);
responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage();
final long numBytes = responseData.getSerializedSize();
metrics.incContainerBytesStats(Type.GetBlock, numBytes);
@@ -875,9 +893,7 @@ ContainerCommandResponseProto handleReadChunk(
ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk()
.getChunkData());
Preconditions.checkNotNull(chunkInfo);
- if (replicaIndexCheckRequired(request)) {
- BlockUtils.verifyReplicaIdx(kvContainer, blockID);
- }
+ BlockUtils.verifyReplicaIdx(kvContainer, blockID);
BlockUtils.verifyBCSId(kvContainer, blockID);
if (dispatcherContext == null) {
@@ -977,6 +993,7 @@ ContainerCommandResponseProto handleWriteChunk(
if (isWrite) {
data =
ChunkBuffer.wrap(writeChunk.getData().asReadOnlyByteBufferList());
+ // TODO: Can improve checksum validation here. Make this one-shot after protocol change.
validateChunkChecksumData(data, chunkInfo);
}
chunkManager
@@ -1225,7 +1242,7 @@ private void checkContainerOpen(KeyValueContainer kvContainer)
* might already be in closing state here.
*/
if (containerState == State.OPEN || containerState == State.CLOSING
- || containerState == State.RECOVERING) {
+ || containerState == RECOVERING) {
return;
}
@@ -1706,4 +1723,22 @@ public static FaultInjector getInjector() {
public static void setInjector(FaultInjector instance) {
injector = instance;
}
+
+ /**
+ * Verify if request's replicaIndex matches with containerData. This validates only for EC containers i.e.
+ * containerReplicaIdx should be > 0.
+ *
+ * @param containerReplicaIdx replicaIndex for the container command.
+ * @param requestDatanodeUUID requested block info
+ * @throws StorageContainerException if replicaIndex mismatches.
+ */
+ private boolean validateRequestDatanodeId(Integer containerReplicaIdx, String requestDatanodeUUID)
+ throws StorageContainerException {
+ if (containerReplicaIdx != null && containerReplicaIdx > 0 && !requestDatanodeUUID.equals(this.getDatanodeId())) {
+ throw new StorageContainerException(
+ String.format("Request is trying to write to node with uuid : %s but the current nodeId is: %s .",
+ requestDatanodeUUID, this.getDatanodeId()), INVALID_ARGUMENT);
+ }
+ return true;
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
index 945efbcf6ea..8bbc2478004 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
@@ -247,7 +247,9 @@ public static void verifyBCSId(Container container, BlockID blockID)
public static void verifyReplicaIdx(Container container, BlockID blockID)
throws IOException {
Integer containerReplicaIndex = container.getContainerData().getReplicaIndex();
- if (containerReplicaIndex > 0 && !containerReplicaIndex.equals(blockID.getReplicaIndex())) {
+ Integer blockReplicaIndex = blockID.getReplicaIndex();
+ if (containerReplicaIndex > 0 && blockReplicaIndex != null && blockReplicaIndex != 0 &&
+ !containerReplicaIndex.equals(blockReplicaIndex)) {
throw new StorageContainerException(
"Unable to find the Container with replicaIdx " + blockID.getReplicaIndex() + ". Container "
+ container.getContainerData().getContainerID() + " replicaIdx is "
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java
index 88aeb3c174d..d9edd6d4cb0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java
@@ -17,27 +17,22 @@
*/
package org.apache.hadoop.ozone.container.metadata;
-import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.StringUtils;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
import org.apache.hadoop.hdds.utils.db.DBProfile;
import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.hdds.utils.db.TableIterator;
-import org.apache.hadoop.hdds.utils.db.managed.ManagedColumnFamilyOptions;
import org.apache.hadoop.hdds.utils.db.managed.ManagedDBOptions;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfoList;
import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
-import org.apache.hadoop.ozone.container.common.utils.db.DatanodeDBProfile;
-import org.rocksdb.InfoLogLevel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,14 +40,11 @@
import java.io.IOException;
import java.util.NoSuchElementException;
-import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DB_PROFILE;
-import static org.apache.hadoop.hdds.utils.db.DBStoreBuilder.HDDS_DEFAULT_DB_PROFILE;
-
/**
* Implementation of the {@link DatanodeStore} interface that contains
* functionality common to all more derived datanode store implementations.
*/
-public abstract class AbstractDatanodeStore implements DatanodeStore {
+public class AbstractDatanodeStore extends AbstractRDBStore implements DatanodeStore {
private Table metadataTable;
@@ -68,12 +60,6 @@ public abstract class AbstractDatanodeStore implements DatanodeStore {
public static final Logger LOG =
LoggerFactory.getLogger(AbstractDatanodeStore.class);
- private volatile DBStore store;
- private final AbstractDatanodeDBDefinition dbDef;
- private final ManagedColumnFamilyOptions cfOptions;
-
- private static DatanodeDBProfile dbProfile;
- private final boolean openReadOnly;
/**
* Constructs the metadata store and starts the DB services.
@@ -84,114 +70,64 @@ public abstract class AbstractDatanodeStore implements DatanodeStore {
protected AbstractDatanodeStore(ConfigurationSource config,
AbstractDatanodeDBDefinition dbDef, boolean openReadOnly)
throws IOException {
-
- dbProfile = DatanodeDBProfile
- .getProfile(config.getEnum(HDDS_DB_PROFILE, HDDS_DEFAULT_DB_PROFILE));
-
- // The same config instance is used on each datanode, so we can share the
- // corresponding column family options, providing a single shared cache
- // for all containers on a datanode.
- cfOptions = dbProfile.getColumnFamilyOptions(config);
-
- this.dbDef = dbDef;
- this.openReadOnly = openReadOnly;
- start(config);
+ super(dbDef, config, openReadOnly);
}
@Override
- public void start(ConfigurationSource config)
+ protected DBStore initDBStore(DBStoreBuilder dbStoreBuilder, ManagedDBOptions options, ConfigurationSource config)
throws IOException {
- if (this.store == null) {
- ManagedDBOptions options = dbProfile.getDBOptions();
- options.setCreateIfMissing(true);
- options.setCreateMissingColumnFamilies(true);
-
- if (this.dbDef instanceof DatanodeSchemaOneDBDefinition ||
- this.dbDef instanceof DatanodeSchemaTwoDBDefinition) {
- long maxWalSize = DBProfile.toLong(StorageUnit.MB.toBytes(2));
- options.setMaxTotalWalSize(maxWalSize);
- }
-
- DatanodeConfiguration dc =
- config.getObject(DatanodeConfiguration.class);
- // Config user log files
- InfoLogLevel level = InfoLogLevel.valueOf(
- dc.getRocksdbLogLevel() + "_LEVEL");
- options.setInfoLogLevel(level);
- options.setMaxLogFileSize(dc.getRocksdbLogMaxFileSize());
- options.setKeepLogFileNum(dc.getRocksdbLogMaxFileNum());
-
- if (this.dbDef instanceof DatanodeSchemaThreeDBDefinition) {
- options.setDeleteObsoleteFilesPeriodMicros(
- dc.getRocksdbDeleteObsoleteFilesPeriod());
-
- // For V3, all Rocksdb dir has the same "container.db" name. So use
- // parentDirName(storage UUID)-dbDirName as db metrics name
- this.store = DBStoreBuilder.newBuilder(config, dbDef)
- .setDBOptions(options)
- .setDefaultCFOptions(cfOptions)
- .setOpenReadOnly(openReadOnly)
- .setDBJmxBeanNameName(dbDef.getDBLocation(config).getName() + "-" +
- dbDef.getName())
- .build();
- } else {
- this.store = DBStoreBuilder.newBuilder(config, dbDef)
- .setDBOptions(options)
- .setDefaultCFOptions(cfOptions)
- .setOpenReadOnly(openReadOnly)
- .build();
- }
+ AbstractDatanodeDBDefinition dbDefinition = this.getDbDef();
+ if (dbDefinition instanceof DatanodeSchemaOneDBDefinition ||
+ dbDefinition instanceof DatanodeSchemaTwoDBDefinition) {
+ long maxWalSize = DBProfile.toLong(StorageUnit.MB.toBytes(2));
+ options.setMaxTotalWalSize(maxWalSize);
+ }
+ DatanodeConfiguration dc =
+ config.getObject(DatanodeConfiguration.class);
- // Use the DatanodeTable wrapper to disable the table iterator on
- // existing Table implementations retrieved from the DBDefinition.
- // See the DatanodeTable's Javadoc for an explanation of why this is
- // necessary.
- metadataTable = new DatanodeTable<>(
- dbDef.getMetadataColumnFamily().getTable(this.store));
- checkTableStatus(metadataTable, metadataTable.getName());
-
- // The block iterator this class returns will need to use the table
- // iterator internally, so construct a block data table instance
- // that does not have the iterator disabled by DatanodeTable.
- blockDataTableWithIterator =
- dbDef.getBlockDataColumnFamily().getTable(this.store);
-
- blockDataTable = new DatanodeTable<>(blockDataTableWithIterator);
- checkTableStatus(blockDataTable, blockDataTable.getName());
-
- if (dbDef.getFinalizeBlocksColumnFamily() != null) {
- finalizeBlocksTableWithIterator =
- dbDef.getFinalizeBlocksColumnFamily().getTable(this.store);
-
- finalizeBlocksTable = new DatanodeTable<>(
- finalizeBlocksTableWithIterator);
- checkTableStatus(finalizeBlocksTable, finalizeBlocksTable.getName());
- }
+ if (dbDefinition instanceof DatanodeSchemaThreeDBDefinition) {
+ options.setDeleteObsoleteFilesPeriodMicros(
+ dc.getRocksdbDeleteObsoleteFilesPeriod());
- if (dbDef.getLastChunkInfoColumnFamily() != null) {
- lastChunkInfoTable = new DatanodeTable<>(
- dbDef.getLastChunkInfoColumnFamily().getTable(this.store));
- checkTableStatus(lastChunkInfoTable, lastChunkInfoTable.getName());
- }
+ // For V3, all Rocksdb dir has the same "container.db" name. So use
+ // parentDirName(storage UUID)-dbDirName as db metrics name
+ dbStoreBuilder.setDBJmxBeanNameName(dbDefinition.getDBLocation(config).getName() + "-" +
+ dbDefinition.getName());
}
- }
-
- @Override
- public synchronized void stop() throws Exception {
- if (store != null) {
- store.close();
- store = null;
+ DBStore dbStore = dbStoreBuilder.setDBOptions(options).build();
+
+ // Use the DatanodeTable wrapper to disable the table iterator on
+ // existing Table implementations retrieved from the DBDefinition.
+ // See the DatanodeTable's Javadoc for an explanation of why this is
+ // necessary.
+ metadataTable = new DatanodeTable<>(
+ dbDefinition.getMetadataColumnFamily().getTable(dbStore));
+ checkTableStatus(metadataTable, metadataTable.getName());
+
+ // The block iterator this class returns will need to use the table
+ // iterator internally, so construct a block data table instance
+ // that does not have the iterator disabled by DatanodeTable.
+ blockDataTableWithIterator =
+ dbDefinition.getBlockDataColumnFamily().getTable(dbStore);
+
+ blockDataTable = new DatanodeTable<>(blockDataTableWithIterator);
+ checkTableStatus(blockDataTable, blockDataTable.getName());
+
+ if (dbDefinition.getFinalizeBlocksColumnFamily() != null) {
+ finalizeBlocksTableWithIterator =
+ dbDefinition.getFinalizeBlocksColumnFamily().getTable(dbStore);
+
+ finalizeBlocksTable = new DatanodeTable<>(
+ finalizeBlocksTableWithIterator);
+ checkTableStatus(finalizeBlocksTable, finalizeBlocksTable.getName());
}
- }
- @Override
- public DBStore getStore() {
- return this.store;
- }
-
- @Override
- public BatchOperationHandler getBatchHandler() {
- return this.store;
+ if (dbDefinition.getLastChunkInfoColumnFamily() != null) {
+ lastChunkInfoTable = new DatanodeTable<>(
+ dbDefinition.getLastChunkInfoColumnFamily().getTable(dbStore));
+ checkTableStatus(lastChunkInfoTable, lastChunkInfoTable.getName());
+ }
+ return dbStore;
}
@Override
@@ -240,44 +176,6 @@ public BlockIterator getFinalizeBlockIterator(long containerID,
finalizeBlocksTableWithIterator.iterator(), filter);
}
- @Override
- public synchronized boolean isClosed() {
- if (this.store == null) {
- return true;
- }
- return this.store.isClosed();
- }
-
- @Override
- public void close() throws IOException {
- this.store.close();
- this.cfOptions.close();
- }
-
- @Override
- public void flushDB() throws IOException {
- store.flushDB();
- }
-
- @Override
- public void flushLog(boolean sync) throws IOException {
- store.flushLog(sync);
- }
-
- @Override
- public void compactDB() throws IOException {
- store.compactDB();
- }
-
- @VisibleForTesting
- public DatanodeDBProfile getDbProfile() {
- return dbProfile;
- }
-
- protected AbstractDatanodeDBDefinition getDbDef() {
- return this.dbDef;
- }
-
protected Table getBlockDataTableWithIterator() {
return this.blockDataTableWithIterator;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractRDBStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractRDBStore.java
new file mode 100644
index 00000000000..5ce1a85b388
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractRDBStore.java
@@ -0,0 +1,135 @@
+package org.apache.hadoop.ozone.container.metadata;
+
+/*
+ * 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.
+ *
+ */
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedColumnFamilyOptions;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedDBOptions;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.db.DatanodeDBProfile;
+import org.rocksdb.InfoLogLevel;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DB_PROFILE;
+import static org.apache.hadoop.hdds.utils.db.DBStoreBuilder.HDDS_DEFAULT_DB_PROFILE;
+
+/**
+ * Abstract Interface defining the way to interact with any rocksDB in the datanode.
+ * @param Generic parameter defining the schema for the DB.
+ */
+public abstract class AbstractRDBStore implements DBStoreManager {
+ private final DEF dbDef;
+ private final ManagedColumnFamilyOptions cfOptions;
+ private static DatanodeDBProfile dbProfile;
+ private final boolean openReadOnly;
+ private volatile DBStore store;
+
+ protected AbstractRDBStore(DEF dbDef, ConfigurationSource config, boolean openReadOnly) throws IOException {
+ dbProfile = DatanodeDBProfile.getProfile(config.getEnum(HDDS_DB_PROFILE, HDDS_DEFAULT_DB_PROFILE));
+
+ // The same config instance is used on each datanode, so we can share the
+ // corresponding column family options, providing a single shared cache
+ // for all containers on a datanode.
+ cfOptions = dbProfile.getColumnFamilyOptions(config);
+ this.dbDef = dbDef;
+ this.openReadOnly = openReadOnly;
+ start(config);
+ }
+
+ public void start(ConfigurationSource config)
+ throws IOException {
+ if (this.store == null) {
+ ManagedDBOptions options = dbProfile.getDBOptions();
+ options.setCreateIfMissing(true);
+ options.setCreateMissingColumnFamilies(true);
+
+ DatanodeConfiguration dc =
+ config.getObject(DatanodeConfiguration.class);
+ // Config user log files
+ InfoLogLevel level = InfoLogLevel.valueOf(
+ dc.getRocksdbLogLevel() + "_LEVEL");
+ options.setInfoLogLevel(level);
+ options.setMaxLogFileSize(dc.getRocksdbLogMaxFileSize());
+ options.setKeepLogFileNum(dc.getRocksdbLogMaxFileNum());
+ this.store = initDBStore(DBStoreBuilder.newBuilder(config, dbDef)
+ .setDBOptions(options)
+ .setDefaultCFOptions(cfOptions)
+ .setOpenReadOnly(openReadOnly), options, config);
+ }
+ }
+
+ protected abstract DBStore initDBStore(DBStoreBuilder dbStoreBuilder, ManagedDBOptions options,
+ ConfigurationSource config) throws IOException;
+
+ public synchronized void stop() throws Exception {
+ if (store != null) {
+ store.close();
+ store = null;
+ }
+ }
+
+ public DBStore getStore() {
+ return this.store;
+ }
+
+ public synchronized boolean isClosed() {
+ if (this.store == null) {
+ return true;
+ }
+ return this.store.isClosed();
+ }
+
+ public BatchOperationHandler getBatchHandler() {
+ return this.store;
+ }
+
+ public void close() throws IOException {
+ this.store.close();
+ this.cfOptions.close();
+ }
+
+ public void flushDB() throws IOException {
+ store.flushDB();
+ }
+
+ public void flushLog(boolean sync) throws IOException {
+ store.flushLog(sync);
+ }
+
+ public void compactDB() throws IOException {
+ store.compactDB();
+ }
+
+ @VisibleForTesting
+ public DatanodeDBProfile getDbProfile() {
+ return dbProfile;
+ }
+
+ protected DEF getDbDef() {
+ return this.dbDef;
+ }
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DBStoreManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DBStoreManager.java
new file mode 100644
index 00000000000..ec9849950a0
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DBStoreManager.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.metadata;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Interface for interacting with datanode databases.
+ */
+public interface DBStoreManager extends Closeable {
+
+ /**
+ * Start datanode manager.
+ *
+ * @param configuration - Configuration
+ * @throws IOException - Unable to start datanode store.
+ */
+ void start(ConfigurationSource configuration) throws IOException;
+
+ /**
+ * Stop datanode manager.
+ */
+ void stop() throws Exception;
+
+ /**
+ * Get datanode store.
+ *
+ * @return datanode store.
+ */
+ DBStore getStore();
+
+ /**
+ * Helper to create and write batch transactions.
+ */
+ BatchOperationHandler getBatchHandler();
+
+ void flushLog(boolean sync) throws IOException;
+
+ void flushDB() throws IOException;
+
+ void compactDB() throws IOException;
+
+ /**
+ * Returns if the underlying DB is closed. This call is thread safe.
+ * @return true if the DB is closed.
+ */
+ boolean isClosed();
+
+ default void compactionIfNeeded() throws Exception {
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java
index 4f54e85da2b..bd1c0fb368a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java
@@ -51,27 +51,21 @@ public class DatanodeSchemaOneDBDefinition
BLOCK_DATA =
new DBColumnFamilyDefinition<>(
StringUtils.bytes2String(DEFAULT_COLUMN_FAMILY),
- String.class,
SchemaOneKeyCodec.get(),
- BlockData.class,
BlockData.getCodec());
public static final DBColumnFamilyDefinition
METADATA =
new DBColumnFamilyDefinition<>(
StringUtils.bytes2String(DEFAULT_COLUMN_FAMILY),
- String.class,
SchemaOneKeyCodec.get(),
- Long.class,
LongCodec.get());
public static final DBColumnFamilyDefinition
DELETED_BLOCKS =
new DBColumnFamilyDefinition<>(
StringUtils.bytes2String(DEFAULT_COLUMN_FAMILY),
- String.class,
SchemaOneKeyCodec.get(),
- ChunkInfoList.class,
SchemaOneChunkInfoListCodec.get());
private static final Map>>
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java
index d47446d49b0..10537ca6f2d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java
@@ -59,45 +59,35 @@ public class DatanodeSchemaThreeDBDefinition
BLOCK_DATA =
new DBColumnFamilyDefinition<>(
"block_data",
- String.class,
FixedLengthStringCodec.get(),
- BlockData.class,
BlockData.getCodec());
public static final DBColumnFamilyDefinition
METADATA =
new DBColumnFamilyDefinition<>(
"metadata",
- String.class,
FixedLengthStringCodec.get(),
- Long.class,
LongCodec.get());
public static final DBColumnFamilyDefinition
DELETE_TRANSACTION =
new DBColumnFamilyDefinition<>(
"delete_txns",
- String.class,
FixedLengthStringCodec.get(),
- DeletedBlocksTransaction.class,
Proto2Codec.get(DeletedBlocksTransaction.getDefaultInstance()));
public static final DBColumnFamilyDefinition
FINALIZE_BLOCKS =
new DBColumnFamilyDefinition<>(
"finalize_blocks",
- String.class,
FixedLengthStringCodec.get(),
- Long.class,
LongCodec.get());
public static final DBColumnFamilyDefinition
LAST_CHUNK_INFO =
new DBColumnFamilyDefinition<>(
"last_chunk_info",
- String.class,
FixedLengthStringCodec.get(),
- BlockData.class,
BlockData.getCodec());
private static String separator = "";
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java
index b9e7ec7bd5b..bf6b1d0a29c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.container.metadata;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
import org.apache.hadoop.hdds.utils.db.DBDefinition;
import org.apache.hadoop.hdds.utils.db.FixedLengthStringCodec;
@@ -44,45 +43,35 @@ public class DatanodeSchemaTwoDBDefinition
BLOCK_DATA =
new DBColumnFamilyDefinition<>(
"block_data",
- String.class,
StringCodec.get(),
- BlockData.class,
BlockData.getCodec());
public static final DBColumnFamilyDefinition
METADATA =
new DBColumnFamilyDefinition<>(
"metadata",
- String.class,
StringCodec.get(),
- Long.class,
LongCodec.get());
public static final DBColumnFamilyDefinition
DELETE_TRANSACTION =
new DBColumnFamilyDefinition<>(
"delete_txns",
- Long.class,
LongCodec.get(),
- StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction.class,
Proto2Codec.get(DeletedBlocksTransaction.getDefaultInstance()));
public static final DBColumnFamilyDefinition
FINALIZE_BLOCKS =
new DBColumnFamilyDefinition<>(
"finalize_blocks",
- String.class,
FixedLengthStringCodec.get(),
- Long.class,
LongCodec.get());
public static final DBColumnFamilyDefinition
LAST_CHUNK_INFO =
new DBColumnFamilyDefinition<>(
"last_chunk_info",
- String.class,
FixedLengthStringCodec.get(),
- BlockData.class,
BlockData.getCodec());
public DatanodeSchemaTwoDBDefinition(String dbPath,
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java
index d791d9bbeab..3ebdc3f6295 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java
@@ -17,22 +17,16 @@
*/
package org.apache.hadoop.ozone.container.metadata;
-import com.google.common.annotations.VisibleForTesting;
-
import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.hdds.utils.db.BatchOperation;
-import org.apache.hadoop.hdds.utils.db.BatchOperationHandler;
-import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfoList;
import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import java.io.Closeable;
import java.io.IOException;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK;
@@ -40,31 +34,10 @@
/**
* Interface for interacting with datanode databases.
*/
-public interface DatanodeStore extends Closeable {
+public interface DatanodeStore extends DBStoreManager {
String NO_SUCH_BLOCK_ERR_MSG =
"Unable to find the block.";
- /**
- * Start datanode manager.
- *
- * @param configuration - Configuration
- * @throws IOException - Unable to start datanode store.
- */
- void start(ConfigurationSource configuration) throws IOException;
-
- /**
- * Stop datanode manager.
- */
- void stop() throws Exception;
-
- /**
- * Get datanode store.
- *
- * @return datanode store.
- */
- @VisibleForTesting
- DBStore getStore();
-
/**
* A Table that keeps the block data.
*
@@ -100,17 +73,6 @@ public interface DatanodeStore extends Closeable {
*/
Table getLastChunkInfoTable();
- /**
- * Helper to create and write batch transactions.
- */
- BatchOperationHandler getBatchHandler();
-
- void flushLog(boolean sync) throws IOException;
-
- void flushDB() throws IOException;
-
- void compactDB() throws IOException;
-
BlockIterator getBlockIterator(long containerID)
throws IOException;
@@ -120,15 +82,6 @@ BlockIterator getBlockIterator(long containerID,
BlockIterator getFinalizeBlockIterator(long containerID,
KeyPrefixFilter filter) throws IOException;
- /**
- * Returns if the underlying DB is closed. This call is thread safe.
- * @return true if the DB is closed.
- */
- boolean isClosed();
-
- default void compactionIfNeeded() throws Exception {
- }
-
default BlockData getBlockByID(BlockID blockID,
String blockKey) throws IOException {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneChunkInfoListCodec.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneChunkInfoListCodec.java
index 1be5a3819c8..25a49eaabe4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneChunkInfoListCodec.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneChunkInfoListCodec.java
@@ -57,6 +57,11 @@ private SchemaOneChunkInfoListCodec() {
// singleton
}
+ @Override
+ public Class getTypeClass() {
+ return ChunkInfoList.class;
+ }
+
@Override
public byte[] toPersistedFormat(ChunkInfoList chunkList) {
return chunkList.getProtoBufMessage().toByteArray();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneKeyCodec.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneKeyCodec.java
index 2f1660f4d2e..add24874a31 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneKeyCodec.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/SchemaOneKeyCodec.java
@@ -48,6 +48,11 @@ private SchemaOneKeyCodec() {
// singleton
}
+ @Override
+ public Class getTypeClass() {
+ return String.class;
+ }
+
@Override
public byte[] toPersistedFormat(String stringObject) throws IOException {
try {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerDBDefinition.java
new file mode 100644
index 00000000000..a15ab27a69d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerDBDefinition.java
@@ -0,0 +1,71 @@
+package org.apache.hadoop.ozone.container.metadata;
+
+/*
+ * 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.
+ *
+ */
+
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.LongCodec;
+import org.apache.hadoop.hdds.utils.db.StringCodec;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+import java.util.Map;
+
+/**
+ * Class for defining the schema for master volume in a datanode.
+ */
+public final class WitnessedContainerDBDefinition extends DBDefinition.WithMap {
+
+ private static final String CONTAINER_IDS_TABLE_NAME = "containerIds";
+
+ public static final DBColumnFamilyDefinition
+ CONTAINER_IDS_TABLE = new DBColumnFamilyDefinition<>(
+ CONTAINER_IDS_TABLE_NAME,
+ LongCodec.get(),
+ StringCodec.get());
+
+ private static final Map>
+ COLUMN_FAMILIES = DBColumnFamilyDefinition.newUnmodifiableMap(
+ CONTAINER_IDS_TABLE);
+
+ private static final WitnessedContainerDBDefinition INSTANCE = new WitnessedContainerDBDefinition();
+
+ public static WitnessedContainerDBDefinition get() {
+ return INSTANCE;
+ }
+
+ private WitnessedContainerDBDefinition() {
+ super(COLUMN_FAMILIES);
+ }
+
+ @Override
+ public String getName() {
+ return OzoneConsts.WITNESSED_CONTAINER_DB_NAME;
+ }
+
+ @Override
+ public String getLocationConfigKey() {
+ return ScmConfigKeys.OZONE_SCM_DATANODE_ID_DIR;
+ }
+
+ public DBColumnFamilyDefinition getContainerIdsTable() {
+ return CONTAINER_IDS_TABLE;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerMetadataStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerMetadataStore.java
new file mode 100644
index 00000000000..b16c7b981ce
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerMetadataStore.java
@@ -0,0 +1,34 @@
+package org.apache.hadoop.ozone.container.metadata;
+
+/*
+ * 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.
+ *
+ */
+
+import org.apache.hadoop.hdds.utils.db.Table;
+
+/**
+ * Interface for interacting with database in the master volume of a datanode.
+ */
+public interface WitnessedContainerMetadataStore extends DBStoreManager {
+ /**
+ * A Table that keeps the containerIds in a datanode.
+ *
+ * @return Table
+ */
+ Table getContainerIdsTable();
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerMetadataStoreImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerMetadataStoreImpl.java
new file mode 100644
index 00000000000..270daf815b2
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/WitnessedContainerMetadataStoreImpl.java
@@ -0,0 +1,78 @@
+package org.apache.hadoop.ozone.container.metadata;
+
+/*
+ * 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.
+ *
+ */
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedDBOptions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Class for interacting with database in the master volume of a datanode.
+ */
+public final class WitnessedContainerMetadataStoreImpl extends AbstractRDBStore
+ implements WitnessedContainerMetadataStore {
+
+ private Table containerIdsTable;
+ private static final ConcurrentMap INSTANCES =
+ new ConcurrentHashMap<>();
+
+ public static WitnessedContainerMetadataStore get(ConfigurationSource conf)
+ throws IOException {
+ String dbDirPath = DBStoreBuilder.getDBDirPath(WitnessedContainerDBDefinition.get(), conf).getAbsolutePath();
+ try {
+ return INSTANCES.compute(dbDirPath, (k, v) -> {
+ if (v == null || v.isClosed()) {
+ try {
+ return new WitnessedContainerMetadataStoreImpl(conf, false);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+ return v;
+ });
+ } catch (UncheckedIOException e) {
+ throw e.getCause();
+ }
+ }
+
+ private WitnessedContainerMetadataStoreImpl(ConfigurationSource config, boolean openReadOnly) throws IOException {
+ super(WitnessedContainerDBDefinition.get(), config, openReadOnly);
+ }
+
+ @Override
+ protected DBStore initDBStore(DBStoreBuilder dbStoreBuilder, ManagedDBOptions options, ConfigurationSource config)
+ throws IOException {
+ DBStore dbStore = dbStoreBuilder.build();
+ this.containerIdsTable = this.getDbDef().getContainerIdsTable().getTable(dbStore);
+ return dbStore;
+ }
+
+ @Override
+ public Table getContainerIdsTable() {
+ return containerIdsTable;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/BackgroundContainerDataScanner.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/BackgroundContainerDataScanner.java
index 1a4f0bf6460..af810c62842 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/BackgroundContainerDataScanner.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/BackgroundContainerDataScanner.java
@@ -62,6 +62,7 @@ public BackgroundContainerDataScanner(ContainerScannerConfiguration conf,
throttler = new HddsDataTransferThrottler(conf.getBandwidthPerVolume());
canceler = new Canceler();
this.metrics = ContainerDataScannerMetrics.create(volume.toString());
+ this.metrics.setStorageDirectory(volume.toString());
this.minScanGap = conf.getContainerScanMinGap();
this.checksumManager = checksumManager;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java
index 84ddba759fe..94841c9d2ea 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java
@@ -249,6 +249,16 @@ public Iterator> getContainers(HddsVolume volume) {
return containerSet.getContainerIterator(volume);
}
+ /**
+ * Get the number of containers based on the given volume.
+ *
+ * @param volume hdds volume.
+ * @return number of containers.
+ */
+ public long getContainerCount(HddsVolume volume) {
+ return containerSet.containerCount(volume);
+ }
+
void updateDataScanTimestamp(long containerId, Instant timestamp)
throws IOException {
Container container = containerSet.getContainer(containerId);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScannerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScannerMetrics.java
index a3f71d34ba1..76e71312aed 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScannerMetrics.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerDataScannerMetrics.java
@@ -37,6 +37,8 @@ public final class ContainerDataScannerMetrics
@Metric("disk bandwidth used by the container data scanner per volume")
private MutableRate numBytesScanned;
+ private String storageDirectory;
+
public double getNumBytesScannedMean() {
return numBytesScanned.lastStat().mean();
}
@@ -66,4 +68,13 @@ public static ContainerDataScannerMetrics create(final String volumeName) {
return ms.register(name, null, new ContainerDataScannerMetrics(name, ms));
}
+
+ @Metric("Returns the Directory name for the volume")
+ public String getStorageDirectory() {
+ return storageDirectory;
+ }
+
+ public void setStorageDirectory(final String volumeName) {
+ this.storageDirectory = volumeName;
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index 1685d1c5fe2..027fbff89c8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -320,8 +320,7 @@ private void resolveDuplicate(KeyValueContainer existing,
private void swapAndRemoveContainer(KeyValueContainer existing,
KeyValueContainer toAdd) throws IOException {
- containerSet.removeContainer(
- existing.getContainerData().getContainerID());
+ containerSet.removeContainerOnlyFromMemory(existing.getContainerData().getContainerID());
containerSet.addContainer(toAdd);
KeyValueContainerUtil.removeContainer(existing.getContainerData(),
hddsVolume.getConf());
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OnDemandContainerDataScanner.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OnDemandContainerDataScanner.java
index eb0f3eedb03..df5050266bd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OnDemandContainerDataScanner.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OnDemandContainerDataScanner.java
@@ -80,6 +80,9 @@ public static synchronized void init(
}
private static boolean shouldScan(Container> container) {
+ if (container == null) {
+ return false;
+ }
long containerID = container.getContainerData().getContainerID();
if (instance == null) {
LOG.debug("Skipping on demand scan for container {} since scanner was " +
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 8ae838a7e53..8316d687b8e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -29,11 +29,15 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.security.SecurityConfig;
import org.apache.hadoop.hdds.security.symmetric.SecretKeyVerifierClient;
import org.apache.hadoop.hdds.security.token.TokenVerifier;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.hdds.utils.IOUtils;
import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
@@ -58,6 +62,8 @@
import org.apache.hadoop.ozone.container.common.volume.StorageVolume.VolumeType;
import org.apache.hadoop.ozone.container.common.volume.StorageVolumeChecker;
import org.apache.hadoop.ozone.container.keyvalue.statemachine.background.StaleRecoveringContainerScrubbingService;
+import org.apache.hadoop.ozone.container.metadata.WitnessedContainerMetadataStore;
+import org.apache.hadoop.ozone.container.metadata.WitnessedContainerMetadataStoreImpl;
import org.apache.hadoop.ozone.container.replication.ContainerImporter;
import org.apache.hadoop.ozone.container.replication.ReplicationServer;
import org.apache.hadoop.ozone.container.replication.ReplicationServer.ReplicationConfig;
@@ -71,6 +77,7 @@
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@@ -131,6 +138,7 @@ public class OzoneContainer {
private ScheduledExecutorService dbCompactionExecutorService;
private final ContainerMetrics metrics;
+ private WitnessedContainerMetadataStore witnessedContainerMetadataStore;
enum InitializingStatus {
UNINITIALIZED, INITIALIZING, INITIALIZED
@@ -181,12 +189,11 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService,
TimeUnit.MINUTES);
}
}
-
long recoveringContainerTimeout = config.getTimeDuration(
OZONE_RECOVERING_CONTAINER_TIMEOUT,
OZONE_RECOVERING_CONTAINER_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
-
- containerSet = new ContainerSet(recoveringContainerTimeout);
+ this.witnessedContainerMetadataStore = WitnessedContainerMetadataStoreImpl.get(conf);
+ containerSet = new ContainerSet(witnessedContainerMetadataStore.getContainerIdsTable(), recoveringContainerTimeout);
metadataScanner = null;
metrics = ContainerMetrics.create(conf);
@@ -309,7 +316,7 @@ public GrpcTlsConfig getTlsClientConfig() {
* Build's container map after volume format.
*/
@VisibleForTesting
- public void buildContainerSet() {
+ public void buildContainerSet() throws IOException {
Iterator volumeSetIterator = volumeSet.getVolumesList()
.iterator();
ArrayList volumeThreads = new ArrayList<>();
@@ -337,6 +344,14 @@ public void buildContainerSet() {
for (int i = 0; i < volumeThreads.size(); i++) {
volumeThreads.get(i).join();
}
+ try (TableIterator> itr =
+ containerSet.getContainerIdsTable().iterator()) {
+ Map containerIds = new HashMap<>();
+ while (itr.hasNext()) {
+ containerIds.put(itr.next().getKey(), 0L);
+ }
+ containerSet.buildMissingContainerSetAndValidate(containerIds);
+ }
} catch (InterruptedException ex) {
LOG.error("Volume Threads Interrupted exception", ex);
Thread.currentThread().interrupt();
@@ -392,6 +407,18 @@ private void initContainerScanner(ContainerScannerConfiguration c) {
}
}
+ /**
+ * We need to inject the containerController into the hddsVolume.
+ * because we need to obtain the container count
+ * for each disk based on the container controller.
+ */
+ private void initHddsVolumeContainer() {
+ for (StorageVolume v : volumeSet.getVolumesList()) {
+ HddsVolume hddsVolume = (HddsVolume) v;
+ hddsVolume.setController(controller);
+ }
+ }
+
private void initMetadataScanner(ContainerScannerConfiguration c) {
if (this.metadataScanner == null) {
this.metadataScanner =
@@ -490,6 +517,8 @@ public void start(String clusterId) throws IOException {
blockDeletingService.start();
recoveringContainerScrubbingService.start();
+ initHddsVolumeContainer();
+
// mark OzoneContainer as INITIALIZED.
initializingStatus.set(InitializingStatus.INITIALIZED);
}
@@ -517,11 +546,21 @@ public void stop() {
}
blockDeletingService.shutdown();
recoveringContainerScrubbingService.shutdown();
+ IOUtils.closeQuietly(metrics);
ContainerMetrics.remove();
checksumTreeManager.stop();
+ if (this.witnessedContainerMetadataStore != null) {
+ try {
+ this.witnessedContainerMetadataStore.stop();
+ } catch (Exception e) {
+ LOG.error("Error while stopping witnessedContainerMetadataStore. Status of store: {}",
+ witnessedContainerMetadataStore.isClosed(), e);
+ }
+ this.witnessedContainerMetadataStore = null;
+ }
}
- public void handleVolumeFailures() {
+ public void handleVolumeFailures() throws StorageContainerException {
if (containerSet != null) {
containerSet.handleVolumeFailures(context);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java
index 9e5b5dbdabd..db86882bfb8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java
@@ -128,7 +128,7 @@ public void importContainer(long containerID, Path tarFilePath,
try (FileInputStream input = new FileInputStream(tarFilePath.toFile())) {
Container container = controller.importContainer(
containerData, input, packer);
- containerSet.addContainer(container);
+ containerSet.addContainerByOverwriteMissingContainer(container);
}
} finally {
importContainerProgress.remove(containerID);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java
index 6bc237207b3..26cd0d82a99 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java
@@ -19,7 +19,6 @@
package org.apache.hadoop.ozone.container.replication;
import java.io.IOException;
-import java.io.InputStream;
import java.io.OutputStream;
import java.util.HashSet;
import java.util.Set;
@@ -59,37 +58,24 @@ public class GrpcReplicationService extends
private final ContainerReplicationSource source;
private final ContainerImporter importer;
- private final boolean zeroCopyEnabled;
-
private final ZeroCopyMessageMarshaller
sendContainerZeroCopyMessageMarshaller;
private final ZeroCopyMessageMarshaller
copyContainerZeroCopyMessageMarshaller;
- public GrpcReplicationService(ContainerReplicationSource source,
- ContainerImporter importer, boolean zeroCopyEnabled) {
+ public GrpcReplicationService(ContainerReplicationSource source, ContainerImporter importer) {
this.source = source;
this.importer = importer;
- this.zeroCopyEnabled = zeroCopyEnabled;
-
- if (zeroCopyEnabled) {
- sendContainerZeroCopyMessageMarshaller = new ZeroCopyMessageMarshaller<>(
- SendContainerRequest.getDefaultInstance());
- copyContainerZeroCopyMessageMarshaller = new ZeroCopyMessageMarshaller<>(
- CopyContainerRequestProto.getDefaultInstance());
- } else {
- sendContainerZeroCopyMessageMarshaller = null;
- copyContainerZeroCopyMessageMarshaller = null;
- }
+
+ sendContainerZeroCopyMessageMarshaller = new ZeroCopyMessageMarshaller<>(
+ SendContainerRequest.getDefaultInstance());
+ copyContainerZeroCopyMessageMarshaller = new ZeroCopyMessageMarshaller<>(
+ CopyContainerRequestProto.getDefaultInstance());
}
public ServerServiceDefinition bindServiceWithZeroCopy() {
ServerServiceDefinition orig = super.bindService();
- if (!zeroCopyEnabled) {
- LOG.info("Zerocopy is not enabled.");
- return orig;
- }
Set methodNames = new HashSet<>();
ServerServiceDefinition.Builder builder =
@@ -155,14 +141,7 @@ public void download(CopyContainerRequestProto request,
} finally {
// output may have already been closed, ignore such errors
IOUtils.cleanupWithLogger(LOG, outputStream);
-
- if (copyContainerZeroCopyMessageMarshaller != null) {
- InputStream popStream =
- copyContainerZeroCopyMessageMarshaller.popStream(request);
- if (popStream != null) {
- IOUtils.cleanupWithLogger(LOG, popStream);
- }
- }
+ copyContainerZeroCopyMessageMarshaller.release(request);
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationServer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationServer.java
index b4e92a4a60a..6ca474bdd8a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationServer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationServer.java
@@ -99,13 +99,12 @@ public ReplicationServer(ContainerController controller,
new LinkedBlockingQueue<>(replicationQueueLimit),
threadFactory);
- init(replicationConfig.isZeroCopyEnable());
+ init();
}
- public void init(boolean enableZeroCopy) {
+ public void init() {
GrpcReplicationService grpcReplicationService = new GrpcReplicationService(
- new OnDemandContainerReplicationSource(controller), importer,
- enableZeroCopy);
+ new OnDemandContainerReplicationSource(controller), importer);
NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forPort(port)
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
.addService(ServerInterceptors.intercept(
@@ -203,11 +202,6 @@ public static final class ReplicationConfig {
static final String REPLICATION_OUTOFSERVICE_FACTOR_KEY =
PREFIX + "." + OUTOFSERVICE_FACTOR_KEY;
- public static final String ZEROCOPY_ENABLE_KEY = "zerocopy.enabled";
- private static final boolean ZEROCOPY_ENABLE_DEFAULT = true;
- private static final String ZEROCOPY_ENABLE_DEFAULT_VALUE =
- "true";
-
/**
* The maximum number of replication commands a single datanode can execute
* simultaneously.
@@ -249,15 +243,6 @@ public static final class ReplicationConfig {
)
private double outOfServiceFactor = OUTOFSERVICE_FACTOR_DEFAULT;
- @Config(key = ZEROCOPY_ENABLE_KEY,
- type = ConfigType.BOOLEAN,
- defaultValue = ZEROCOPY_ENABLE_DEFAULT_VALUE,
- tags = {DATANODE, SCM},
- description = "Specify if zero-copy should be enabled for " +
- "replication protocol."
- )
- private boolean zeroCopyEnable = ZEROCOPY_ENABLE_DEFAULT;
-
public double getOutOfServiceFactor() {
return outOfServiceFactor;
}
@@ -291,14 +276,6 @@ public void setReplicationQueueLimit(int limit) {
this.replicationQueueLimit = limit;
}
- public boolean isZeroCopyEnable() {
- return zeroCopyEnable;
- }
-
- public void setZeroCopyEnable(boolean zeroCopyEnable) {
- this.zeroCopyEnable = zeroCopyEnable;
- }
-
@PostConstruct
public void validate() {
if (replicationMaxStreams < 1) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisor.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisor.java
index 92ff4b6d8d6..9513cac84ef 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisor.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisor.java
@@ -43,6 +43,8 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicationCommandPriority;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.replication.ReplicationServer.ReplicationConfig;
@@ -50,6 +52,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -77,6 +80,10 @@ public final class ReplicationSupervisor {
private final Map failureCounter = new ConcurrentHashMap<>();
private final Map timeoutCounter = new ConcurrentHashMap<>();
private final Map skippedCounter = new ConcurrentHashMap<>();
+ private final Map queuedCounter = new ConcurrentHashMap<>();
+
+ private final MetricsRegistry registry;
+ private final Map opsLatencyMs = new ConcurrentHashMap<>();
private static final Map METRICS_MAP;
@@ -218,6 +225,7 @@ private ReplicationSupervisor(StateContext context, ExecutorService executor,
nodeStateUpdated(dn.getPersistedOpState());
}
}
+ registry = new MetricsRegistry(ReplicationSupervisor.class.getSimpleName());
}
/**
@@ -240,6 +248,9 @@ public void addTask(AbstractReplicationTask task) {
failureCounter.put(task.getMetricName(), new AtomicLong(0));
timeoutCounter.put(task.getMetricName(), new AtomicLong(0));
skippedCounter.put(task.getMetricName(), new AtomicLong(0));
+ queuedCounter.put(task.getMetricName(), new AtomicLong(0));
+ opsLatencyMs.put(task.getMetricName(), registry.newRate(
+ task.getClass().getSimpleName() + "Ms"));
METRICS_MAP.put(task.getMetricName(), task.getMetricDescriptionSegment());
}
}
@@ -253,6 +264,7 @@ public void addTask(AbstractReplicationTask task) {
taskCounter.computeIfAbsent(task.getClass(),
k -> new AtomicInteger()).incrementAndGet();
}
+ queuedCounter.get(task.getMetricName()).incrementAndGet();
executor.execute(new TaskRunner(task));
}
}
@@ -353,6 +365,7 @@ public TaskRunner(AbstractReplicationTask task) {
@Override
public void run() {
+ final long startTime = Time.monotonicNow();
try {
requestCounter.get(task.getMetricName()).incrementAndGet();
@@ -401,6 +414,8 @@ public void run() {
LOG.warn("Failed {}", this, e);
failureCounter.get(task.getMetricName()).incrementAndGet();
} finally {
+ queuedCounter.get(task.getMetricName()).decrementAndGet();
+ opsLatencyMs.get(task.getMetricName()).add(Time.monotonicNow() - startTime);
inFlight.remove(task);
decrementTaskCounter(task);
}
@@ -511,4 +526,22 @@ public long getReplicationSkippedCount(String metricsName) {
return counter != null ? counter.get() : 0;
}
+ public long getReplicationQueuedCount() {
+ return getCount(queuedCounter);
+ }
+
+ public long getReplicationQueuedCount(String metricsName) {
+ AtomicLong counter = queuedCounter.get(metricsName);
+ return counter != null ? counter.get() : 0;
+ }
+
+ public long getReplicationRequestAvgTime(String metricsName) {
+ MutableRate rate = opsLatencyMs.get(metricsName);
+ return rate != null ? (long) rate.lastStat().mean() : 0;
+ }
+
+ public long getReplicationRequestTotalTime(String metricsName) {
+ MutableRate rate = opsLatencyMs.get(metricsName);
+ return rate != null ? (long) rate.lastStat().total() : 0;
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisorMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisorMetrics.java
index a1763976af9..cd1103a0c46 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisorMetrics.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationSupervisorMetrics.java
@@ -67,7 +67,7 @@ public void getMetrics(MetricsCollector collector, boolean all) {
supervisor.getTotalInFlightReplications())
.addGauge(Interns.info("numQueuedReplications",
"Number of replications in queue"),
- supervisor.getQueueSize())
+ supervisor.getReplicationQueuedCount())
.addGauge(Interns.info("numRequestedReplications",
"Number of requested replications"),
supervisor.getReplicationRequestCount())
@@ -107,7 +107,10 @@ public void getMetrics(MetricsCollector collector, boolean all) {
.addGauge(Interns.info("numSkipped" + metricsName,
"Number of " + descriptionSegment + " skipped as the container is "
+ "already present"),
- supervisor.getReplicationSkippedCount(metricsName));
+ supervisor.getReplicationSkippedCount(metricsName))
+ .addGauge(Interns.info("numQueued" + metricsName,
+ "Number of " + descriptionSegment + " in queue"),
+ supervisor.getReplicationQueuedCount(metricsName));
}
});
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java
index 506a96fe051..40b4dec3493 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java
@@ -30,7 +30,6 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.io.InputStream;
import java.io.OutputStream;
import java.nio.file.Files;
import java.nio.file.Path;
@@ -105,10 +104,7 @@ public void onNext(SendContainerRequest req) {
onError(t);
} finally {
if (marshaller != null) {
- InputStream popStream = marshaller.popStream(req);
- if (popStream != null) {
- IOUtils.cleanupWithLogger(LOG, popStream);
- }
+ marshaller.release(req);
}
}
}
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html
index fd3d7407d23..4f51b423e8a 100644
--- a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html
@@ -22,8 +22,32 @@
+HeartBeat Information
+
+
+
+ Address |
+ Last Successful HeartBeat |
+ Missed Count |
+ State |
+ Type |
+ Version Number |
+
+
+
+
+ {{scm.addressString}} |
+ {{scm.lastSuccessfulHeartbeat}} |
+ {{scm.missedCount}} |
+ {{scm.state}} |
+ {{scm.type}} |
+ {{scm.versionNumber}} |
+
+
+
+
Volume Information
-
+
Directory |
@@ -33,6 +57,7 @@ Volume Information
Available Space |
Reserved |
Total Capacity |
+ Containers |
State |
@@ -45,6 +70,7 @@ Volume Information
{{volumeInfo.Available}} |
{{volumeInfo.Reserved}} |
{{volumeInfo.TotalCapacity}} |
+ {{volumeInfo.Containers}} |
{{volumeInfo["tag.VolumeState"]}} |
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-scanner.html b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-scanner.html
new file mode 100644
index 00000000000..5c54a2aa0a7
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-scanner.html
@@ -0,0 +1,47 @@
+
+
+
+
+ DataNode Scanner Status
+
+
+ DataNode Scanner Information
+
+
+
+ Directory |
+ NumBytesScannedNumOps |
+ NumBytesScannedAvgTime |
+ NumContainersScanned |
+ NumScanIterations |
+ NumUnHealthyContainers |
+
+
+
+
+ {{scanner["tag.StorageDirectory"]}} |
+ {{scanner.NumBytesScannedNumOps}} |
+ {{scanner.NumBytesScannedAvgTime | millisecondsToMinutes}} |
+ {{scanner.NumContainersScanned}} |
+ {{scanner.NumScanIterations}} |
+ {{scanner.NumUnHealthyContainers}} |
+
+
+
+
+
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js
index adc507acce9..547e566ef8a 100644
--- a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js
@@ -36,20 +36,104 @@
volume.TotalCapacity = transform(volume.TotalCapacity);
})
});
+
+ $http.get("jmx?qry=Hadoop:service=HddsDatanode,name=SCMConnectionManager")
+ .then(function (result) {
+ ctrl.heartbeatmetrics = result.data.beans;
+ ctrl.heartbeatmetrics.forEach(scm => {
+ var scmServers = scm.SCMServers;
+ scmServers.forEach(scmServer => {
+ scmServer.lastSuccessfulHeartbeat = convertTimestampToDate(scmServer.lastSuccessfulHeartbeat)
+ })
+ })
+ });
}
});
- function transform(v) {
- var UNITS = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'ZB'];
- var prev = 0, i = 0;
- while (Math.floor(v) > 0 && i < UNITS.length) {
+
+ // Register ioStatus Controller
+ angular.module('ozone').config(function ($routeProvider) {
+ $routeProvider.when('/iostatus', {
+ templateUrl: 'iostatus.html',
+ controller: 'IOStatusController as ioStatusCtrl',
+ });
+ });
+
+ angular.module('ozone')
+ .controller('IOStatusController', function ($http) {
+ var ctrl = this;
+ $http.get("jmx?qry=Hadoop:service=HddsDatanode,name=VolumeIOStats*")
+ .then(function (result) {
+ ctrl.dniostatus = result.data.beans;
+ });
+ });
+
+ // Register Scanner Controller
+ angular.module('ozone').config(function ($routeProvider) {
+ $routeProvider.when('/dn-scanner', {
+ templateUrl: 'dn-scanner.html',
+ controller: 'DNScannerController as scannerStatusCtrl',
+ });
+ });
+
+ angular.module('ozone')
+ .controller('DNScannerController', function ($http) {
+ var ctrl = this;
+ $http.get("jmx?qry=Hadoop:service=HddsDatanode,name=ContainerDataScannerMetrics*")
+ .then(function (result) {
+ ctrl.dnscanner = result.data.beans;
+ });
+ });
+
+ angular.module('ozone')
+ .filter('millisecondsToMinutes', function() {
+ return function(milliseconds) {
+ if (isNaN(milliseconds)) {
+ return 'Invalid input';
+ }
+ var minutes = Math.floor(milliseconds / 60000); // 1 minute = 60000 milliseconds
+ var seconds = Math.floor((milliseconds % 60000) / 1000);
+ return minutes + ' mins ' + seconds + ' secs';
+ };
+ });
+
+ angular.module('ozone')
+ .filter('twoDecimalPlaces', function() {
+ return function(input) {
+ if (isNaN(input)) {
+ return 'Invalid input';
+ }
+ return parseFloat(input).toFixed(2);
+ };
+ });
+
+ function transform(v) {
+ var UNITS = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'ZB'];
+ var prev = 0, i = 0;
+ while (Math.floor(v) > 0 && i < UNITS.length) {
prev = v;
v /= 1024;
i += 1;
- }
- if (i > 0 && i < UNITS.length) {
+ }
+ if (i > 0 && i < UNITS.length) {
v = prev;
i -= 1;
- }
- return Math.round(v * 100) / 100 + ' ' + UNITS[i];
}
+ return Math.round(v * 100) / 100 + ' ' + UNITS[i];
+ }
+
+ function convertTimestampToDate(timestamp) {
+ if (!timestamp) return '';
+ var milliseconds = timestamp * 1000;
+
+ var date = new Date(milliseconds);
+
+ var year = date.getFullYear();
+ var month = date.getMonth() + 1;
+ var day = date.getDate();
+ var hours = date.getHours();
+ var minutes = date.getMinutes();
+ var seconds = date.getSeconds();
+
+ return `${year}-${month.toString().padStart(2, '0')}-${day.toString().padStart(2, '0')} ${hours.toString().padStart(2, '0')}:${minutes.toString().padStart(2, '0')}:${seconds.toString().padStart(2, '0')}`;
+ }
})();
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html
index 1c32fe64e0e..0e1cbf21a00 100644
--- a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html
@@ -49,11 +49,10 @@
HDDS Datanode Service
-
-
-
-
+
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/iostatus.html b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/iostatus.html
new file mode 100644
index 00000000000..94916821bd8
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/iostatus.html
@@ -0,0 +1,76 @@
+
+
+
+
+ DataNode IO Status
+
+
+
+ Read Performance
+
+
+
+ Directory |
+ ReadBytes |
+ ReadOpCount |
+ ReadAvgTime |
+ ReadLatency60s(P90) |
+ ReadLatency60s(P95) |
+ ReadLatency60s(P99) |
+
+
+
+
+ {{volumeInfo["tag.StorageDirectory"]}} |
+ {{volumeInfo.ReadBytes}} |
+ {{volumeInfo.ReadOpCount}} |
+ {{volumeInfo.ReadTimeAvgTime | twoDecimalPlaces}} ms |
+ {{volumeInfo.ReadLatency60s90thPercentileLatency | twoDecimalPlaces}} ms |
+ {{volumeInfo.ReadLatency60s95thPercentileLatency | twoDecimalPlaces}} ms |
+ {{volumeInfo.ReadLatency60s99thPercentileLatency | twoDecimalPlaces}} ms |
+
+
+
+
+ Write Performance
+
+
+
+ Directory |
+ WriteBytes |
+ WriteOpCount |
+ WriteAvgTime |
+ WriteLatency60s(P90) |
+ WriteLatency60s(P95) |
+ WriteLatency60s(P99) |
+
+
+
+
+ {{volumeInfo["tag.StorageDirectory"]}} |
+ {{volumeInfo.WriteBytes}} |
+ {{volumeInfo.WriteOpCount}} |
+ {{volumeInfo.WriteTimeAvgTime | twoDecimalPlaces}} ms |
+ {{volumeInfo.WriteLatency60s90thPercentileLatency | twoDecimalPlaces}} ms |
+ {{volumeInfo.WriteLatency60s95thPercentileLatency | twoDecimalPlaces}} ms |
+ {{volumeInfo.WriteLatency60s99thPercentileLatency | twoDecimalPlaces}} ms |
+
+
+
+
+
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
index 3b9c4a93ec5..e52328bafd0 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
@@ -163,11 +163,11 @@ public static DatanodeDetails createDatanodeDetails() {
.nextInt(256) + "." + random.nextInt(256);
DatanodeDetails.Port containerPort =
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 0);
+ DatanodeDetails.newStandalonePort(0);
DatanodeDetails.Port ratisPort =
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.RATIS, 0);
+ DatanodeDetails.newRatisPort(0);
DatanodeDetails.Port restPort =
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.REST, 0);
+ DatanodeDetails.newRestPort(0);
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
builder.setUuid(UUID.randomUUID())
.setHostName("localhost")
@@ -414,7 +414,7 @@ public static ContainerController getEmptyContainerController() {
public static XceiverServerRatis newXceiverServerRatis(
DatanodeDetails dn, OzoneConfiguration conf) throws IOException {
conf.setInt(OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_PORT,
- dn.getPort(DatanodeDetails.Port.Name.RATIS).getValue());
+ dn.getRatisPort().getValue());
return XceiverServerRatis.newXceiverServerRatis(null, dn, conf,
getNoopContainerDispatcher(), getEmptyContainerController(),
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
index e1e1ee9172a..41be7acbb14 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
@@ -198,8 +198,7 @@ public void testDatanodeStateContext() throws IOException,
OzoneConsts.OZONE_SCM_DATANODE_ID_FILE_DEFAULT);
idPath.delete();
DatanodeDetails datanodeDetails = getNewDatanodeDetails();
- DatanodeDetails.Port port = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE,
+ DatanodeDetails.Port port = DatanodeDetails.newStandalonePort(
OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT);
datanodeDetails.setPort(port);
ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath, conf);
@@ -325,8 +324,7 @@ public void testDatanodeStateMachineWithIdWriteFail() throws Exception {
OzoneConsts.OZONE_SCM_DATANODE_ID_FILE_DEFAULT);
idPath.delete();
DatanodeDetails datanodeDetails = getNewDatanodeDetails();
- DatanodeDetails.Port port = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE,
+ DatanodeDetails.Port port = DatanodeDetails.newStandalonePort(
OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT);
datanodeDetails.setPort(port);
@@ -408,12 +406,9 @@ public void testDatanodeStateMachineWithInvalidConfiguration()
}
private DatanodeDetails getNewDatanodeDetails() {
- DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE, 0);
- DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.RATIS, 0);
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.REST, 0);
+ DatanodeDetails.Port containerPort = DatanodeDetails.newStandalonePort(0);
+ DatanodeDetails.Port ratisPort = DatanodeDetails.newRatisPort(0);
+ DatanodeDetails.Port restPort = DatanodeDetails.newRestPort(0);
DatanodeDetails.Port streamPort = DatanodeDetails.newPort(
DatanodeDetails.Port.Name.RATIS_DATASTREAM, 0);
return DatanodeDetails.newBuilder()
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestContainerUtils.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestContainerUtils.java
index 4f33e833a3c..f825be46882 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestContainerUtils.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestContainerUtils.java
@@ -38,7 +38,6 @@
import java.nio.charset.StandardCharsets;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port;
import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.ReadChunk;
import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getReadChunkResponse;
@@ -92,7 +91,7 @@ public void testTarName() throws IOException {
public void testDatanodeIDPersistent(@TempDir File tempDir) throws Exception {
// Generate IDs for testing
DatanodeDetails id1 = randomDatanodeDetails();
- id1.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 1));
+ id1.setPort(DatanodeDetails.newStandalonePort(1));
assertWriteRead(tempDir, id1);
// Add certificate serial id.
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeIdYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeIdYaml.java
index 8a272868146..1a0401de7e8 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeIdYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeIdYaml.java
@@ -73,7 +73,7 @@ void testWriteReadBeforeRatisDatastreamPortLayoutVersion(@TempDir File dir)
// if no separate admin/server/datastream port, return single Ratis one for
// compat
assertEquals(read.getPort(DatanodeDetails.Port.Name.RATIS_DATASTREAM),
- read.getPort(DatanodeDetails.Port.Name.RATIS));
+ read.getRatisPort());
}
@Test
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java
index 05bebdd1b90..fc107414d40 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java
@@ -510,12 +510,9 @@ static HddsDispatcher createDispatcher(DatanodeDetails dd, UUID scmId,
// This method has to be removed once we move scm/TestUtils.java
// from server-scm project to container-service or to common project.
private static DatanodeDetails randomDatanodeDetails() {
- DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE, 0);
- DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.RATIS, 0);
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.REST, 0);
+ DatanodeDetails.Port containerPort = DatanodeDetails.newStandalonePort(0);
+ DatanodeDetails.Port ratisPort = DatanodeDetails.newRatisPort(0);
+ DatanodeDetails.Port restPort = DatanodeDetails.newRestPort(0);
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
builder.setUuid(UUID.randomUUID())
.setHostName("localhost")
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerCommandHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerCommandHandler.java
index a3b60aa36da..2e1e0eafd01 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerCommandHandler.java
@@ -273,12 +273,9 @@ private CloseContainerCommand forceCloseWithoutPipeline() {
*/
private static DatanodeDetails randomDatanodeDetails() {
String ipAddress = "127.0.0.1";
- DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE, 0);
- DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.RATIS, 0);
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.REST, 0);
+ DatanodeDetails.Port containerPort = DatanodeDetails.newStandalonePort(0);
+ DatanodeDetails.Port ratisPort = DatanodeDetails.newRatisPort(0);
+ DatanodeDetails.Port restPort = DatanodeDetails.newRestPort(0);
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
builder.setUuid(UUID.randomUUID())
.setHostName("localhost")
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconstructECContainersCommandHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconstructECContainersCommandHandler.java
new file mode 100644
index 00000000000..7e6c7608180
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconstructECContainersCommandHandler.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Proto2Utils;
+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.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
+import org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionCoordinator;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.replication.ReplicationSupervisor;
+import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test cases to verify {@link ReconstructECContainersCommandHandler}.
+ */
+public class TestReconstructECContainersCommandHandler {
+ private OzoneConfiguration conf;
+ private ReplicationSupervisor supervisor;
+ private ECReconstructionCoordinator coordinator;
+ private OzoneContainer ozoneContainer;
+ private StateContext stateContext;
+ private SCMConnectionManager connectionManager;
+
+ @BeforeEach
+ public void setUp() {
+ supervisor = mock(ReplicationSupervisor.class);
+ coordinator = mock(ECReconstructionCoordinator.class);
+ conf = new OzoneConfiguration();
+ ozoneContainer = mock(OzoneContainer.class);
+ connectionManager = mock(SCMConnectionManager.class);
+ stateContext = mock(StateContext.class);
+ }
+
+ @Test
+ public void testMetrics() {
+ ReconstructECContainersCommandHandler commandHandler =
+ new ReconstructECContainersCommandHandler(conf, supervisor, coordinator);
+ doNothing().when(supervisor).addTask(any());
+ Map handlerMap = new HashMap<>();
+ handlerMap.put(commandHandler.getCommandType(), commandHandler);
+ CommandHandlerMetrics metrics = CommandHandlerMetrics.create(handlerMap);
+ try {
+ byte[] missingIndexes = {1, 2};
+ ByteString missingContainerIndexes = Proto2Utils.unsafeByteString(missingIndexes);
+ ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2);
+ List dnDetails = getDNDetails(5);
+ List sources =
+ dnDetails.stream().map(a -> new ReconstructECContainersCommand
+ .DatanodeDetailsAndReplicaIndex(a, dnDetails.indexOf(a)))
+ .collect(Collectors.toList());
+ List targets = getDNDetails(2);
+ ReconstructECContainersCommand reconstructECContainersCommand =
+ new ReconstructECContainersCommand(1L, sources, targets,
+ missingContainerIndexes, ecReplicationConfig);
+
+ commandHandler.handle(reconstructECContainersCommand, ozoneContainer,
+ stateContext, connectionManager);
+ String metricsName = "ECReconstructions";
+ assertEquals(commandHandler.getMetricsName(), metricsName);
+ when(supervisor.getReplicationRequestCount(metricsName)).thenReturn(1L);
+ assertEquals(commandHandler.getInvocationCount(), 1);
+
+ commandHandler.handle(new ReconstructECContainersCommand(2L, sources,
+ targets, missingContainerIndexes, ecReplicationConfig), ozoneContainer,
+ stateContext, connectionManager);
+ commandHandler.handle(new ReconstructECContainersCommand(3L, sources,
+ targets, missingContainerIndexes, ecReplicationConfig), ozoneContainer,
+ stateContext, connectionManager);
+ commandHandler.handle(new ReconstructECContainersCommand(4L, sources,
+ targets, missingContainerIndexes, ecReplicationConfig), ozoneContainer,
+ stateContext, connectionManager);
+ commandHandler.handle(new ReconstructECContainersCommand(5L, sources,
+ targets, missingContainerIndexes, ecReplicationConfig), ozoneContainer,
+ stateContext, connectionManager);
+ commandHandler.handle(new ReconstructECContainersCommand(6L, sources,
+ targets, missingContainerIndexes, ecReplicationConfig), ozoneContainer,
+ stateContext, connectionManager);
+
+ when(supervisor.getReplicationRequestCount(metricsName)).thenReturn(5L);
+ when(supervisor.getReplicationRequestTotalTime(metricsName)).thenReturn(10L);
+ when(supervisor.getReplicationRequestAvgTime(metricsName)).thenReturn(2L);
+ when(supervisor.getReplicationQueuedCount(metricsName)).thenReturn(1L);
+ assertEquals(commandHandler.getInvocationCount(), 5);
+ assertEquals(commandHandler.getQueuedCount(), 1);
+ assertEquals(commandHandler.getTotalRunTime(), 10);
+ assertEquals(commandHandler.getAverageRunTime(), 2);
+
+ MetricsCollectorImpl metricsCollector = new MetricsCollectorImpl();
+ metrics.getMetrics(metricsCollector, true);
+ assertEquals(1, metricsCollector.getRecords().size());
+ } finally {
+ metrics.unRegister();
+ }
+ }
+
+ private List getDNDetails(int numDns) {
+ List dns = new ArrayList<>();
+ for (int i = 0; i < numDns; i++) {
+ dns.add(MockDatanodeDetails.randomDatanodeDetails());
+ }
+ return dns;
+ }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerCommandHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerCommandHandler.java
new file mode 100644
index 00000000000..9de00877e5b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerCommandHandler.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
+
+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.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
+import org.apache.hadoop.ozone.container.common.helpers.CommandHandlerMetrics;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.replication.ContainerReplicator;
+import org.apache.hadoop.ozone.container.replication.ReplicationSupervisor;
+import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.doNothing;
+
+/**
+ * Test cases to verify {@link ReplicateContainerCommandHandler}.
+ */
+public class TestReplicateContainerCommandHandler {
+ private OzoneConfiguration conf;
+ private ReplicationSupervisor supervisor;
+ private ContainerReplicator downloadReplicator;
+ private ContainerReplicator pushReplicator;
+ private OzoneContainer ozoneContainer;
+ private StateContext stateContext;
+ private SCMConnectionManager connectionManager;
+
+ @BeforeEach
+ public void setUp() {
+ conf = new OzoneConfiguration();
+ supervisor = mock(ReplicationSupervisor.class);
+ downloadReplicator = mock(ContainerReplicator.class);
+ pushReplicator = mock(ContainerReplicator.class);
+ ozoneContainer = mock(OzoneContainer.class);
+ connectionManager = mock(SCMConnectionManager.class);
+ stateContext = mock(StateContext.class);
+ }
+
+ @Test
+ public void testMetrics() {
+ ReplicateContainerCommandHandler commandHandler =
+ new ReplicateContainerCommandHandler(conf, supervisor,
+ downloadReplicator, pushReplicator);
+ Map handlerMap = new HashMap<>();
+ handlerMap.put(commandHandler.getCommandType(), commandHandler);
+ CommandHandlerMetrics metrics = CommandHandlerMetrics.create(handlerMap);
+ try {
+ doNothing().when(supervisor).addTask(any());
+ DatanodeDetails source = MockDatanodeDetails.randomDatanodeDetails();
+ DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails();
+ List sourceList = new ArrayList<>();
+ sourceList.add(source);
+
+ ReplicateContainerCommand command = ReplicateContainerCommand.fromSources(
+ 1, sourceList);
+ commandHandler.handle(command, ozoneContainer, stateContext, connectionManager);
+ String metricsName = "ContainerReplications";
+ assertEquals(commandHandler.getMetricsName(), metricsName);
+ when(supervisor.getReplicationRequestCount(metricsName)).thenReturn(1L);
+ assertEquals(commandHandler.getInvocationCount(), 1);
+
+ commandHandler.handle(ReplicateContainerCommand.fromSources(2, sourceList),
+ ozoneContainer, stateContext, connectionManager);
+ commandHandler.handle(ReplicateContainerCommand.fromSources(3, sourceList),
+ ozoneContainer, stateContext, connectionManager);
+ commandHandler.handle(ReplicateContainerCommand.toTarget(4, target),
+ ozoneContainer, stateContext, connectionManager);
+ commandHandler.handle(ReplicateContainerCommand.toTarget(5, target),
+ ozoneContainer, stateContext, connectionManager);
+ commandHandler.handle(ReplicateContainerCommand.fromSources(6, sourceList),
+ ozoneContainer, stateContext, connectionManager);
+
+ when(supervisor.getReplicationRequestCount(metricsName)).thenReturn(5L);
+ when(supervisor.getReplicationRequestTotalTime(metricsName)).thenReturn(10L);
+ when(supervisor.getReplicationRequestAvgTime(metricsName)).thenReturn(3L);
+ when(supervisor.getReplicationQueuedCount(metricsName)).thenReturn(1L);
+ assertEquals(commandHandler.getInvocationCount(), 5);
+ assertEquals(commandHandler.getQueuedCount(), 1);
+ assertEquals(commandHandler.getTotalRunTime(), 10);
+ assertEquals(commandHandler.getAverageRunTime(), 3);
+
+ MetricsCollectorImpl metricsCollector = new MetricsCollectorImpl();
+ metrics.getMetrics(metricsCollector, true);
+ assertEquals(1, metricsCollector.getRecords().size());
+ } finally {
+ metrics.unRegister();
+ }
+ }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestReservedVolumeSpace.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestReservedVolumeSpace.java
index 5e2dd0c75c9..5e0a31944f7 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestReservedVolumeSpace.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestReservedVolumeSpace.java
@@ -20,6 +20,7 @@
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationException;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.fs.MockSpaceUsageCheckFactory;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -37,6 +38,7 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_DU_RESERVED_PERCENT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_DU_RESERVED_PERCENT_DEFAULT;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
/**
@@ -166,6 +168,16 @@ public void testInvalidConfig() throws Exception {
assertEquals(getExpectedDefaultReserved(hddsVolume2), reservedFromVolume2);
}
+ @Test
+ public void testInvalidConfigThrowsException() {
+ OzoneConfiguration conf = new OzoneConfiguration();
+ conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_DU_RESERVED, "15GB");
+
+ assertThrows(ConfigurationException.class,
+ () -> volumeBuilder.conf(conf).build(),
+ "Reserved space should be configured in a pair");
+ }
+
@Test
public void testPathsCanonicalized() throws Exception {
OzoneConfiguration conf = new OzoneConfiguration();
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeIOStatsWithPrometheusSink.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeIOStatsWithPrometheusSink.java
index c8934bab416..1df886098ab 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeIOStatsWithPrometheusSink.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeIOStatsWithPrometheusSink.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.ozone.container.common.volume;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.server.http.PrometheusMetricsSink;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -30,6 +31,7 @@
import static org.assertj.core.api.Assertions.assertThat;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_DATANODE_IO_METRICS_PERCENTILES_INTERVALS_SECONDS_KEY;
/**
* Test PrometheusMetricSink regarding VolumeIOStats.
@@ -54,11 +56,14 @@ public void tearDown() {
@Test
public void testMultipleVolumeIOMetricsExist() throws IOException {
+ OzoneConfiguration conf = new OzoneConfiguration();
+ int[] intervals = conf.getInts(OZONE_DATANODE_IO_METRICS_PERCENTILES_INTERVALS_SECONDS_KEY);
+
//GIVEN
VolumeIOStats volumeIOStats1 = new VolumeIOStats("VolumeIOStat1",
- "vol1/dir");
+ "vol1/dir", intervals);
VolumeIOStats volumeIOStat2 = new VolumeIOStats("VolumeIOStat2",
- "vol2/dir");
+ "vol2/dir", intervals);
//WHEN
String writtenMetrics = publishMetricsAndGetOutput();
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java
index 55df5f43b6b..0b24161aadb 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java
@@ -340,6 +340,7 @@ public void testVolumeFailure() throws IOException {
conSet.handleVolumeFailures(stateContext);
// ContainerID1 should be removed belonging to failed volume
assertNull(conSet.getContainer(containerID1));
+ assertTrue(conSet.getMissingContainerSet().contains(containerID1));
// ContainerID should exist belonging to normal volume
assertNotNull(conSet.getContainer(containerID));
expectedReportCount.put(
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index e1a3de30ddf..584db675d93 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -689,7 +689,7 @@ public void testContainerRocksDB(ContainerTestVersionInfo versionInfo)
try (DBHandle db = BlockUtils.getDB(keyValueContainerData, CONF)) {
RDBStore store = (RDBStore) db.getStore().getStore();
- long defaultCacheSize = 64 * OzoneConsts.MB;
+ long defaultCacheSize = OzoneConsts.GB;
long cacheSize = Long.parseLong(store
.getProperty("rocksdb.block-cache-capacity"));
assertEquals(defaultCacheSize, cacheSize);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index 30a8a9bcbce..0ff2aaa22b5 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -83,6 +83,7 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.io.TempDir;
+import org.mockito.Mockito;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
@@ -147,7 +148,13 @@ public void testHandlerCommandHandling() throws Exception {
.build();
KeyValueContainer container = mock(KeyValueContainer.class);
-
+ KeyValueContainerData containerData = mock(KeyValueContainerData.class);
+ Mockito.when(container.getContainerData()).thenReturn(containerData);
+ Mockito.when(containerData.getReplicaIndex()).thenReturn(1);
+ ContainerProtos.ContainerCommandResponseProto responseProto = KeyValueHandler.dispatchRequest(handler,
+ createContainerRequest, container, null);
+ assertEquals(ContainerProtos.Result.INVALID_ARGUMENT, responseProto.getResult());
+ Mockito.when(handler.getDatanodeId()).thenReturn(DATANODE_UUID);
KeyValueHandler
.dispatchRequest(handler, createContainerRequest, container, null);
verify(handler, times(0)).handleListBlock(
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java
index af0c430c86d..5f1a93ef2fb 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java
@@ -133,8 +133,7 @@ public void testGetBlockWithReplicaIndexMismatch(ClientVersion clientVersion, in
handler.handleGetBlock(
getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.GetBlock, rid),
container);
- assertEquals((replicaIndex > 0 && rid != replicaIndex && clientVersion.toProtoValue() >=
- ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue()) ?
+ assertEquals((replicaIndex > 0 && rid != 0 && rid != replicaIndex) ?
ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID,
response.getResult());
}
@@ -176,8 +175,7 @@ public void testReadChunkWithReplicaIndexMismatch(ClientVersion clientVersion, i
ContainerProtos.ContainerCommandResponseProto response =
handler.handleReadChunk(getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.ReadChunk, rid),
container, null);
- assertEquals((replicaIndex > 0 && rid != replicaIndex &&
- clientVersion.toProtoValue() >= ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue()) ?
+ assertEquals((replicaIndex > 0 && rid != 0 && rid != replicaIndex) ?
ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID,
response.getResult());
}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 07804c2a20b..2f2cbc81e90 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -20,6 +20,7 @@
import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.client.BlockID;
@@ -51,7 +52,9 @@
import java.io.File;
import java.nio.file.Files;
import java.nio.file.Path;
+import java.util.HashSet;
import java.util.Random;
+import java.util.Set;
import java.util.UUID;
import java.util.HashMap;
import java.util.List;
@@ -122,7 +125,7 @@ public void testBuildContainerMap(ContainerTestVersionInfo versionInfo)
volume.format(clusterId);
commitSpaceMap.put(getVolumeKey(volume), Long.valueOf(0));
}
-
+ List containerDatas = new ArrayList<>();
// Add containers to disk
int numTestContainers = 10;
for (int i = 0; i < numTestContainers; i++) {
@@ -136,6 +139,7 @@ public void testBuildContainerMap(ContainerTestVersionInfo versionInfo)
layout,
maxCap, UUID.randomUUID().toString(),
datanodeDetails.getUuidString());
+ containerDatas.add(keyValueContainerData);
keyValueContainer = new KeyValueContainer(
keyValueContainerData, conf);
keyValueContainer.create(volumeSet, volumeChoosingPolicy, clusterId);
@@ -156,8 +160,22 @@ public void testBuildContainerMap(ContainerTestVersionInfo versionInfo)
ozoneContainer.buildContainerSet();
ContainerSet containerset = ozoneContainer.getContainerSet();
assertEquals(numTestContainers, containerset.containerCount());
-
verifyCommittedSpace(ozoneContainer);
+ Set missingContainers = new HashSet<>();
+ for (int i = 0; i < numTestContainers; i++) {
+ if (i % 2 == 0) {
+ missingContainers.add(containerDatas.get(i).getContainerID());
+ FileUtils.deleteDirectory(new File(containerDatas.get(i).getContainerPath()));
+ }
+ }
+ ozoneContainer.stop();
+ ozoneContainer = ContainerTestUtils.getOzoneContainer(datanodeDetails, conf);
+ ozoneContainer.buildContainerSet();
+ containerset = ozoneContainer.getContainerSet();
+ assertEquals(numTestContainers / 2, containerset.containerCount());
+ assertEquals(numTestContainers / 2 + numTestContainers % 2, containerset.getMissingContainerSet().size());
+ assertEquals(missingContainers, containerset.getMissingContainerSet());
+ ozoneContainer.stop();
}
@ContainerTestVersionInfo.ContainerTest
@@ -300,12 +318,9 @@ private DatanodeDetails createDatanodeDetails() {
random.nextInt(256) + "." + random.nextInt(256) + "." + random
.nextInt(256) + "." + random.nextInt(256);
- DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE, 0);
- DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.RATIS, 0);
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.REST, 0);
+ DatanodeDetails.Port containerPort = DatanodeDetails.newStandalonePort(0);
+ DatanodeDetails.Port ratisPort = DatanodeDetails.newRatisPort(0);
+ DatanodeDetails.Port restPort = DatanodeDetails.newRestPort(0);
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
builder.setUuid(UUID.randomUUID())
.setHostName("localhost")
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java
index b8c43460ba3..c1cf59f0fdd 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java
@@ -82,17 +82,15 @@ class TestGrpcReplicationService {
@BeforeEach
public void setUp() throws Exception {
- init(false);
+ init();
}
- public void init(boolean isZeroCopy) throws Exception {
+ public void init() throws Exception {
conf = new OzoneConfiguration();
ReplicationServer.ReplicationConfig replicationConfig =
conf.getObject(ReplicationServer.ReplicationConfig.class);
- replicationConfig.setZeroCopyEnable(isZeroCopy);
-
SecurityConfig secConf = new SecurityConfig(conf);
ContainerSet containerSet = new ContainerSet(1000);
@@ -103,11 +101,9 @@ public void init(boolean isZeroCopy) throws Exception {
.setPersistedOpState(HddsProtos.NodeOperationalState.IN_SERVICE)
.setPersistedOpStateExpiry(0);
DatanodeDetails.Port containerPort =
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE,
- OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT);
+ DatanodeDetails.newStandalonePort(OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT);
DatanodeDetails.Port ratisPort =
- DatanodeDetails.newPort(DatanodeDetails.Port.Name.RATIS,
- OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_PORT_DEFAULT);
+ DatanodeDetails.newRatisPort(OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_PORT_DEFAULT);
DatanodeDetails.Port replicationPort =
DatanodeDetails.newPort(DatanodeDetails.Port.Name.REPLICATION,
replicationConfig.getPort());
@@ -226,7 +222,7 @@ public void copyData(long containerId, OutputStream destination,
};
ContainerImporter importer = mock(ContainerImporter.class);
GrpcReplicationService subject =
- new GrpcReplicationService(source, importer, false);
+ new GrpcReplicationService(source, importer);
CopyContainerRequestProto request = CopyContainerRequestProto.newBuilder()
.setContainerID(1)
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java
index ef37c226653..315e0c0253b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java
@@ -87,6 +87,7 @@
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicationCommandPriority.LOW;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicationCommandPriority.NORMAL;
import static org.mockito.Mockito.any;
@@ -488,6 +489,15 @@ public void testMultipleReplication(ContainerLayoutVersion layout,
assertEquals(0, ecReconstructionSupervisor.getReplicationRequestCount(
task1.getMetricName()));
+ assertTrue(replicationSupervisor.getReplicationRequestTotalTime(
+ task1.getMetricName()) > 0);
+ assertTrue(ecReconstructionSupervisor.getReplicationRequestTotalTime(
+ task2.getMetricName()) > 0);
+ assertTrue(replicationSupervisor.getReplicationRequestAvgTime(
+ task1.getMetricName()) > 0);
+ assertTrue(ecReconstructionSupervisor.getReplicationRequestAvgTime(
+ task2.getMetricName()) > 0);
+
MetricsCollectorImpl replicationMetricsCollector = new MetricsCollectorImpl();
replicationMetrics.getMetrics(replicationMetricsCollector, true);
assertEquals(1, replicationMetricsCollector.getRecords().size());
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDataNodeStartupSlvLessThanMlv.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDataNodeStartupSlvLessThanMlv.java
index e9fef6ecfd6..55bddf2e99a 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDataNodeStartupSlvLessThanMlv.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDataNodeStartupSlvLessThanMlv.java
@@ -72,12 +72,9 @@ public void testStartupSlvLessThanMlv() throws Exception {
}
private DatanodeDetails getNewDatanodeDetails() {
- DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.STANDALONE, 0);
- DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.RATIS, 0);
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.REST, 0);
+ DatanodeDetails.Port containerPort = DatanodeDetails.newStandalonePort(0);
+ DatanodeDetails.Port ratisPort = DatanodeDetails.newRatisPort(0);
+ DatanodeDetails.Port restPort = DatanodeDetails.newRestPort(0);
return DatanodeDetails.newBuilder()
.setUuid(UUID.randomUUID())
.setHostName("localhost")
diff --git a/hadoop-hdds/crypto-api/pom.xml b/hadoop-hdds/crypto-api/pom.xml
index db19cc4f341..ca54b3de9f2 100644
--- a/hadoop-hdds/crypto-api/pom.xml
+++ b/hadoop-hdds/crypto-api/pom.xml
@@ -19,11 +19,11 @@
org.apache.ozone
hdds
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
hdds-crypto-api
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
Apache Ozone Distributed Data Store cryptographic functions
Apache Ozone HDDS Crypto
diff --git a/hadoop-hdds/crypto-default/pom.xml b/hadoop-hdds/crypto-default/pom.xml
index c586f91712b..6024c3e2ddf 100644
--- a/hadoop-hdds/crypto-default/pom.xml
+++ b/hadoop-hdds/crypto-default/pom.xml
@@ -19,11 +19,11 @@
org.apache.ozone
hdds
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
hdds-crypto-default
- 1.5.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
Default implementation of Apache Ozone Distributed Data Store's cryptographic functions
Apache Ozone HDDS Crypto - Default
diff --git a/hadoop-hdds/docs/content/feature/Quota.md b/hadoop-hdds/docs/content/feature/Quota.md
index 90e413357b5..53c196307fa 100644
--- a/hadoop-hdds/docs/content/feature/Quota.md
+++ b/hadoop-hdds/docs/content/feature/Quota.md
@@ -1,6 +1,6 @@
---
title: "Quota in Ozone"
-date: "2020-October-22"
+date: "2020-10-22"
weight: 4
summary: Quota in Ozone
icon: user
diff --git a/hadoop-hdds/docs/content/feature/Quota.zh.md b/hadoop-hdds/docs/content/feature/Quota.zh.md
index 16e5db26cde..d690947ef06 100644
--- a/hadoop-hdds/docs/content/feature/Quota.zh.md
+++ b/hadoop-hdds/docs/content/feature/Quota.zh.md
@@ -1,6 +1,6 @@
---
title: "Ozone 中的配额"
-date: "2020-October-22"
+date: "2020-10-22"
weight: 4
summary: Ozone中的配额
icon: user
diff --git a/hadoop-hdds/docs/content/feature/SCM-HA.md b/hadoop-hdds/docs/content/feature/SCM-HA.md
index cc42500e0c3..333c908275d 100644
--- a/hadoop-hdds/docs/content/feature/SCM-HA.md
+++ b/hadoop-hdds/docs/content/feature/SCM-HA.md
@@ -96,7 +96,7 @@ Second and third nodes should be *bootstrapped* instead of init. These clusters
ozone scm --bootstrap
```
-Note: both commands perform one-time initialization. SCM still needs to be started by running `ozone scm --daemon start`.
+Note: both commands perform one-time initialization. SCM still needs to be started by running `ozone --daemon start scm`.
## Auto-bootstrap
@@ -121,7 +121,7 @@ Note: SCM still needs to be started after the init/bootstrap process.
```
ozone scm --init
ozone scm --bootstrap
-ozone scm --daemon start
+ozone --daemon start scm
```
For Docker/Kubernetes, use `ozone scm` to start it in the foreground.
diff --git a/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md b/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md
index 92ae64f8a1d..002aba4cc2d 100644
--- a/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md
+++ b/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.md
@@ -25,15 +25,15 @@ summary: Introduction to Ozone Datanode Container Schema V3
In Ozone, user data are separated into blocks and stored in HDDS Containers. Containers are the fundamental replication unit of Ozone/HDDS. Each Container has its metadata and data. Data are saved as files on disk. Metadata is saved in RocksDB.
-Currently there will be one RocksDB for each Container on datanode. With user data continously grow, there will be hundreds of thousands of RocksDB instances on one datanode. It's a big challenge to manage this amount of RocksDB instances in one JVM.
+Earlier, there was one RocksDB for each Container on datanode. With user data continously growing, there will be hundreds of thousands of RocksDB instances on one datanode. It's a big challenge to manage this amount of RocksDB instances in one JVM.
-Unlike the current approach, this "Merge Container RocksDB in DN" feature will use only one RocksDB for each data volume, holding all metadata of Containers in this RocksDB.
+Unlike the previous approach, this "Merge Container RocksDB in DN" feature will use only one RocksDB for each data volume, holding all metadata of Containers in this RocksDB.
## Configuration
-This is mainly a DN feature, which doesn't require much configuration.
+This is mainly a DN feature, which doesn't require much configuration. By default, it is enabled.
-Here is a configuration which disable this feature if the current one RocksDB for each container mode is more preferred. Please be noted that once the feature is enabled, it's strongly suggested not to disable it in later.
+Here is a configuration which disables this feature if the "one RocksDB for each container" mode is more preferred. Please be noted that once the feature is enabled, it's strongly suggested not to disable it in later.
```XML
diff --git a/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.zh.md b/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.zh.md
index cd3eb5fbdc5..65085a99451 100644
--- a/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.zh.md
+++ b/hadoop-hdds/docs/content/feature/dn-merge-rocksdb.zh.md
@@ -25,13 +25,13 @@ summary: Ozone DataNode Container模式简介V3
在 Ozone 中,用户数据被分割成blocks并存储在 HDDS Container中。Container是 Ozone/HDDS 的基本复制单元。每个Container都有自己的元数据和数据, 数据以文件形式保存在磁盘上,元数据保存在RocksDB中。
-目前,数据节点上的每个Container都有一个RocksDB。随着用户数据的不断增长,一个DataNode上将会有成百上千个RocksDB实例。在一个JVM中管理如此多的RocksDB实例是一个巨大的挑战。
+之前,数据节点上每个Container都有一个RocksDB。随着用户数据的不断增长,一个DataNode上将会有成百上千个RocksDB实例。在一个JVM中管理如此多的RocksDB实例是一个巨大的挑战。
-与当前使用方法不同,"Merge Container RocksDB in DN"功能将为每个Volume只使用一个RocksDB,并在此RocksDB中保存所有Container的元数据。
+与以前的用法不同,"Merge Container RocksDB in DN"功能将为每个Volume只使用一个RocksDB,并在此RocksDB中保存所有Container的元数据。
## 配置
-这主要是DataNode的功能,不需要太多配置。
+这主要是DataNode的功能,不需要太多配置。默认情况下,它是启用的。
如果更倾向于为每个Container使用一个RocksDB的模式,那么这下面的配置可以禁用上面所介绍的功能。请注意,一旦启用该功能,强烈建议以后不要再禁用。
diff --git a/hadoop-hdds/docs/content/interface/CSI.md b/hadoop-hdds/docs/content/interface/CSI.md
index 59b24c94d19..84bd89c049e 100644
--- a/hadoop-hdds/docs/content/interface/CSI.md
+++ b/hadoop-hdds/docs/content/interface/CSI.md
@@ -57,7 +57,7 @@ Now, create the CSI related resources by execute the follow command.
kubectl create -f /ozone/kubernetes/examples/ozone/csi
```
-## Crete pv-test and visit the result.
+## Create pv-test and visit the result.
Create pv-test related resources by execute the follow command.
diff --git a/hadoop-hdds/docs/content/interface/S3.md b/hadoop-hdds/docs/content/interface/S3.md
index 1d0c5dcb4cc..c2f0f3af9b9 100644
--- a/hadoop-hdds/docs/content/interface/S3.md
+++ b/hadoop-hdds/docs/content/interface/S3.md
@@ -163,10 +163,3 @@ Or
aws s3 ls --endpoint http://localhost:9878 s3://buckettest
```
-### S3 Fuse driver (goofys)
-
-[Goofys](https://github.com/kahing/goofys) is a S3 FUSE driver. As Ozone S3 gateway is AWS S3 compatible, it can be used to mount any Ozone buckets as an OS level mounted filesystem.
-
-```bash
-goofys --endpoint http://localhost:9878 bucket1 /mount/bucket1
-```
diff --git a/hadoop-hdds/docs/content/interface/S3.zh.md b/hadoop-hdds/docs/content/interface/S3.zh.md
index e3b133a0e16..370098e091e 100644
--- a/hadoop-hdds/docs/content/interface/S3.zh.md
+++ b/hadoop-hdds/docs/content/interface/S3.zh.md
@@ -142,10 +142,3 @@ aws s3api --endpoint http://localhost:9878 create-bucket --bucket buckettest
aws s3 ls --endpoint http://localhost:9878 s3://buckettest
```
-### S3 Fuse 驱动(goofys)
-
-Goofys 是一个 S3 FUSE 驱动,可以将 Ozone 的桶挂载到 POSIX 文件系统。
-
-```bash
-goofys --endpoint http://localhost:9878 bucket1 /mount/bucket1
-```
diff --git a/hadoop-hdds/docs/content/security/GDPR.md b/hadoop-hdds/docs/content/security/GDPR.md
index 25b2f2c4416..409a3ae7be0 100644
--- a/hadoop-hdds/docs/content/security/GDPR.md
+++ b/hadoop-hdds/docs/content/security/GDPR.md
@@ -1,6 +1,6 @@
---
title: "GDPR in Ozone"
-date: "2019-September-17"
+date: "2019-09-17"
weight: 3
icon: user
menu:
diff --git a/hadoop-hdds/docs/content/security/GDPR.zh.md b/hadoop-hdds/docs/content/security/GDPR.zh.md
index a7db4030871..8fd3514138f 100644
--- a/hadoop-hdds/docs/content/security/GDPR.zh.md
+++ b/hadoop-hdds/docs/content/security/GDPR.zh.md
@@ -1,6 +1,6 @@
---
title: "Ozone 中的 GDPR"
-date: "2019-September-17"
+date: "2019-09-17"
weight: 3
summary: Ozone 中的 GDPR
menu:
diff --git a/hadoop-hdds/docs/content/security/SecureOzone.md b/hadoop-hdds/docs/content/security/SecureOzone.md
index 76fd7470109..bbeef79b613 100644
--- a/hadoop-hdds/docs/content/security/SecureOzone.md
+++ b/hadoop-hdds/docs/content/security/SecureOzone.md
@@ -1,6 +1,6 @@
---
title: "Securing Ozone"
-date: "2019-April-03"
+date: "2019-04-03"
summary: Overview of Ozone security concepts and steps to secure Ozone Manager and SCM.
weight: 1
menu:
diff --git a/hadoop-hdds/docs/content/security/SecureOzone.zh.md b/hadoop-hdds/docs/content/security/SecureOzone.zh.md
index a7660233f4d..e74b5d8dfab 100644
--- a/hadoop-hdds/docs/content/security/SecureOzone.zh.md
+++ b/hadoop-hdds/docs/content/security/SecureOzone.zh.md
@@ -1,6 +1,6 @@
---
title: "安全化 Ozone"
-date: "2019-April-03"
+date: "2019-04-03"
summary: 简要介绍 Ozone 中的安全概念以及安全化 OM 和 SCM 的步骤。
weight: 1
menu:
diff --git a/hadoop-hdds/docs/content/security/SecuringDatanodes.md b/hadoop-hdds/docs/content/security/SecuringDatanodes.md
index 717e746cfb9..2254155e1f4 100644
--- a/hadoop-hdds/docs/content/security/SecuringDatanodes.md
+++ b/hadoop-hdds/docs/content/security/SecuringDatanodes.md
@@ -1,6 +1,6 @@
---
title: "Securing Datanodes"
-date: "2019-April-03"
+date: "2019-04-03"
weight: 3
menu:
main:
diff --git a/hadoop-hdds/docs/content/security/SecuringDatanodes.zh.md b/hadoop-hdds/docs/content/security/SecuringDatanodes.zh.md
index 608be16e8a3..8b37fd2f6ee 100644
--- a/hadoop-hdds/docs/content/security/SecuringDatanodes.zh.md
+++ b/hadoop-hdds/docs/content/security/SecuringDatanodes.zh.md
@@ -1,6 +1,6 @@
---
title: "安全化 Datanode"
-date: "2019-April-03"
+date: "2019-04-03"
weight: 3
menu:
main:
diff --git a/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.md b/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.md
index 47c04eb94d9..a8601d7a5e1 100644
--- a/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.md
+++ b/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.md
@@ -1,6 +1,6 @@
---
title: "Securing HTTP"
-date: "2020-June-17"
+date: "2020-06-17"
summary: Secure HTTP web-consoles for Ozone services
weight: 4
menu:
diff --git a/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.zh.md b/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.zh.md
index 07b3f6164f6..5907a7caf9a 100644
--- a/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.zh.md
+++ b/hadoop-hdds/docs/content/security/SecuringOzoneHTTP.zh.md
@@ -1,6 +1,6 @@
---
title: "安全化 HTTP"
-date: "2020-June-17"
+date: "2020-06-17"
summary: 安全化 Ozone 服务的 HTTP 网络控制台
weight: 4
menu:
diff --git a/hadoop-hdds/docs/content/security/SecuringS3.md b/hadoop-hdds/docs/content/security/SecuringS3.md
index e6218b95e91..04ef6921af6 100644
--- a/hadoop-hdds/docs/content/security/SecuringS3.md
+++ b/hadoop-hdds/docs/content/security/SecuringS3.md
@@ -1,6 +1,6 @@
---
title: "Securing S3"
-date: "2019-April-03"
+date: "2019-04-03"
summary: Ozone supports S3 protocol, and uses AWS Signature Version 4 protocol which allows a seamless S3 experience.
weight: 5
menu:
diff --git a/hadoop-hdds/docs/content/security/SecuringS3.zh.md b/hadoop-hdds/docs/content/security/SecuringS3.zh.md
index 218786fd366..395b9303354 100644
--- a/hadoop-hdds/docs/content/security/SecuringS3.zh.md
+++ b/hadoop-hdds/docs/content/security/SecuringS3.zh.md
@@ -1,6 +1,6 @@
---
title: "安全化 S3"
-date: "2019-April-03"
+date: "2019-04-03"
summary: Ozone 支持 S3 协议,并使用 AWS Signature Version 4 protocol which allows a seamless S3
experience.
weight: 5
diff --git a/hadoop-hdds/docs/content/security/SecuringTDE.md b/hadoop-hdds/docs/content/security/SecuringTDE.md
index 3b75bee1bfd..0d04a28aec7 100644
--- a/hadoop-hdds/docs/content/security/SecuringTDE.md
+++ b/hadoop-hdds/docs/content/security/SecuringTDE.md
@@ -1,6 +1,6 @@
---
title: "Transparent Data Encryption"
-date: "2019-April-03"
+date: "2019-04-03"
summary: TDE allows data on the disks to be encrypted-at-rest and automatically decrypted during access.
weight: 2
menu:
diff --git a/hadoop-hdds/docs/content/security/SecuringTDE.zh.md b/hadoop-hdds/docs/content/security/SecuringTDE.zh.md
index ed42519e0b2..d7fa4941e44 100644
--- a/hadoop-hdds/docs/content/security/SecuringTDE.zh.md
+++ b/hadoop-hdds/docs/content/security/SecuringTDE.zh.md
@@ -1,6 +1,6 @@
---
title: "透明数据加密"
-date: "2019-April-03"
+date: "2019-04-03"
summary: 透明数据加密(Transparent Data Encryption,TDE)以密文形式在磁盘上保存数据,但可以在用户访问的时候自动进行解密。
weight: 2
menu:
diff --git a/hadoop-hdds/docs/content/security/SecurityAcls.md b/hadoop-hdds/docs/content/security/SecurityAcls.md
index 9976cbbc4fb..ee48999ed25 100644
--- a/hadoop-hdds/docs/content/security/SecurityAcls.md
+++ b/hadoop-hdds/docs/content/security/SecurityAcls.md
@@ -1,6 +1,6 @@
---
title: "Ozone ACLs"
-date: "2019-April-03"
+date: "2019-04-03"
weight: 6
menu:
main:
diff --git a/hadoop-hdds/docs/content/security/SecurityAcls.zh.md b/hadoop-hdds/docs/content/security/SecurityAcls.zh.md
index 3d95fcf0877..99751cd62da 100644
--- a/hadoop-hdds/docs/content/security/SecurityAcls.zh.md
+++ b/hadoop-hdds/docs/content/security/SecurityAcls.zh.md
@@ -1,6 +1,6 @@
---
title: "Ozone 访问控制列表"
-date: "2019-April-03"
+date: "2019-04-03"
weight: 6
menu:
main:
diff --git a/hadoop-hdds/docs/content/security/SecurityWithRanger.md b/hadoop-hdds/docs/content/security/SecurityWithRanger.md
index bbbd8c19f32..7dc1895ad3d 100644
--- a/hadoop-hdds/docs/content/security/SecurityWithRanger.md
+++ b/hadoop-hdds/docs/content/security/SecurityWithRanger.md
@@ -1,6 +1,6 @@
---
title: "Apache Ranger"
-date: "2019-April-03"
+date: "2019-04-03"
weight: 7
menu:
main:
diff --git a/hadoop-hdds/docs/content/security/SecurityWithRanger.zh.md b/hadoop-hdds/docs/content/security/SecurityWithRanger.zh.md
index b7c7b8721bb..8917c0b84bc 100644
--- a/hadoop-hdds/docs/content/security/SecurityWithRanger.zh.md
+++ b/hadoop-hdds/docs/content/security/SecurityWithRanger.zh.md
@@ -1,6 +1,6 @@
---
title: "Apache Ranger"
-date: "2019-April-03"
+date: "2019-04-03"
weight: 7
menu:
main:
diff --git a/hadoop-hdds/docs/content/tools/Debug.md b/hadoop-hdds/docs/content/tools/Debug.md
new file mode 100644
index 00000000000..79c11f777ef
--- /dev/null
+++ b/hadoop-hdds/docs/content/tools/Debug.md
@@ -0,0 +1,473 @@
+---
+title: "Ozone Debug"
+date: 2024-10-14
+summary: Ozone Debug command can be used for all the debugging related tasks.
+---
+
+
+Ozone Debug command (`ozone debug`) is a collection of developer tools intended to help in debugging and get more information of various components of ozone.
+
+```bash
+Usage: ozone debug [-hV] [--verbose] [-conf=]
+ [-D=]... [COMMAND]
+Developer tools for Ozone Debug operations
+ -conf=
+ path to the ozone configuration file
+ -D, --set=
+ Map of (configuration_key,configuration_value) for any
+ configuration overrides
+ -h, --help Show this help message and exit.
+ -V, --version Print version information and exit.
+ --verbose More verbose output. Show the stack trace of the errors.
+Commands:
+ chunkinfo returns chunk location information about an
+ existing key
+ print-log-dag, pld Create an image of the current compaction log DAG
+ in OM.
+ find-missing-padding, fmp List all keys with any missing padding, optionally
+ limited to a volume/bucket/key URI.
+ recover recover the lease of a specified file. Make sure
+ to specify file system scheme if ofs:// is not
+ the default.
+ prefix Parse prefix contents
+ ldb Parse rocksdb file content
+ read-replicas Reads every replica for all the blocks associated
+ with a given key.
+ container Container replica specific operations to be
+ executed on datanodes only
+ ratislogparser Shell of printing Ratis Log in understandable text
+```
+For more detailed usage see the output of `--help` for each of the subcommands.
+
+
+## ozone debug ldb
+
+Ozone heavily uses RocksDB for storing metadata. This tool helps parse the contents of RocksDB belonging to Ozone Roles.
+Supported DB's : Ozone Manager (om.db) , StorageContainerManager (scm.db), Datanode/Container (container.db)
+Below is the usage:
+
+```bash
+Usage: ozone debug ldb --db= [COMMAND]
+Parse rocksdb file content
+ --db= Database File Path
+Commands:
+ scan Parse specified metadataTable
+ list_column_families, ls list all column families in db.
+ value-schema Schema of value in metadataTable
+```
+
+### list_column_families command
+
+`list_column_families` command lists all the column families in the db provided.
+
+```bash
+$ ozone debug ldb --db=/path/to/scm.db ls
+default
+sequenceId
+revokedCertsV2
+pipelines
+crls
+crlSequenceId
+meta
+containers
+validCerts
+validSCMCerts
+scmTransactionInfos
+deletedBlocks
+statefulServiceConfig
+revokedCerts
+move
+```
+
+### scan command
+
+`scan` command parses a particular column family of a rocksdb provided and prints the records.
+
+```bash
+Usage: ozone debug ldb scan [--compact] [--count] [--with-keys]
+ [--batch-size=] --cf=
+ [--cid=] [-d=]
+ [-e=] [--fields=]
+ [--filter=] [-l=] [-o=