diff --git a/hadoop-hdds/client/pom.xml b/hadoop-hdds/client/pom.xml index 5cd4ead18fb..bf728403cb4 100644 --- a/hadoop-hdds/client/pom.xml +++ b/hadoop-hdds/client/pom.xml @@ -37,10 +37,67 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-common + + org.apache.ozone + hdds-config + org.apache.ozone hdds-erasurecode + + org.apache.ozone + hdds-interface-client + + + + org.apache.commons + commons-lang3 + + + org.apache.ratis + ratis-client + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-grpc + + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-thirdparty-misc + + + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + + + io.opentracing + opentracing-api + + + io.opentracing + opentracing-util + + + + jakarta.annotation + jakarta.annotation-api + diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ErrorInjector.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ErrorInjector.java new file mode 100644 index 00000000000..35806967a5c --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ErrorInjector.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.scm; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.RaftClientReply; + +/** + * Client side error injector allowing simulating receiving errors from server side. + */ +@FunctionalInterface +public interface ErrorInjector { + RaftClientReply getResponse(ContainerProtos.ContainerCommandRequestProto request, ClientId id, Pipeline pipeline); +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java index b130f48776c..873f6f67348 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java @@ -248,21 +248,29 @@ public enum ChecksumCombineMode { private String fsDefaultBucketLayout = "FILE_SYSTEM_OPTIMIZED"; @Config(key = "incremental.chunk.list", - defaultValue = "false", + defaultValue = "true", type = ConfigType.BOOLEAN, description = "Client PutBlock request can choose incremental chunk " + "list rather than full chunk list to optimize performance. " + - "Critical to HBase.", + "Critical to HBase. EC does not support this feature.", tags = ConfigTag.CLIENT) private boolean incrementalChunkList = true; @Config(key = "stream.putblock.piggybacking", - defaultValue = "false", + defaultValue = "true", type = ConfigType.BOOLEAN, description = "Allow PutBlock to be piggybacked in WriteChunk " + "requests if the chunk is small.", tags = ConfigTag.CLIENT) - private boolean enablePutblockPiggybacking = false; + private boolean enablePutblockPiggybacking = true; + + @Config(key = "key.write.concurrency", + defaultValue = "1", + description = "Maximum concurrent writes allowed on each key. " + + "Defaults to 1 which matches the behavior before HDDS-9844. " + + "For unlimited write concurrency, set this to -1 or any negative integer value.", + tags = ConfigTag.CLIENT) + private int maxConcurrentWritePerKey = 1; @PostConstruct public void validate() { @@ -485,4 +493,12 @@ public void setIncrementalChunkList(boolean enable) { public boolean getIncrementalChunkList() { return this.incrementalChunkList; } + + public void setMaxConcurrentWritePerKey(int maxConcurrentWritePerKey) { + this.maxConcurrentWritePerKey = maxConcurrentWritePerKey; + } + + public int getMaxConcurrentWritePerKey() { + return this.maxConcurrentWritePerKey; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java new file mode 100644 index 00000000000..75ae01c1005 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.scm; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.scm.client.ClientTrustManager; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneSecurityUtil; + +import java.io.IOException; + +/** + * Factory for XceiverClientSpi implementations. Client instances are not cached. + */ +public class XceiverClientCreator implements XceiverClientFactory { + private static ErrorInjector errorInjector; + + public static void enableErrorInjection(ErrorInjector injector) { + errorInjector = injector; + } + + private final ConfigurationSource conf; + private final boolean topologyAwareRead; + private final ClientTrustManager trustManager; + private final boolean securityEnabled; + + public XceiverClientCreator(ConfigurationSource conf) { + this(conf, null); + } + + public XceiverClientCreator(ConfigurationSource conf, ClientTrustManager trustManager) { + this.conf = conf; + this.securityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf); + topologyAwareRead = conf.getBoolean( + OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_KEY, + OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_DEFAULT); + this.trustManager = trustManager; + if (securityEnabled) { + Preconditions.checkNotNull(trustManager); + } + } + + public boolean isSecurityEnabled() { + return securityEnabled; + } + + protected XceiverClientSpi newClient(Pipeline pipeline) throws IOException { + XceiverClientSpi client; + switch (pipeline.getType()) { + case RATIS: + client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf, trustManager, errorInjector); + break; + case STAND_ALONE: + client = new XceiverClientGrpc(pipeline, conf, trustManager); + break; + case EC: + client = new ECXceiverClientGrpc(pipeline, conf, trustManager); + break; + case CHAINED: + default: + throw new IOException("not implemented " + pipeline.getType()); + } + try { + client.connect(); + } catch (Exception e) { + throw new IOException(e); + } + return client; + } + + @Override + public XceiverClientSpi acquireClient(Pipeline pipeline) throws IOException { + return acquireClient(pipeline, false); + } + + @Override + public void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClient) { + releaseClient(xceiverClient, invalidateClient, false); + } + + @Override + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline) throws IOException { + return acquireClient(pipeline); + } + + @Override + public void releaseClientForReadData(XceiverClientSpi xceiverClient, boolean invalidateClient) { + releaseClient(xceiverClient, invalidateClient, topologyAwareRead); + } + + @Override + public XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) throws IOException { + return newClient(pipeline); + } + + @Override + public void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClient, boolean topologyAware) { + IOUtils.closeQuietly(xceiverClient); + } + + @Override + public void close() throws Exception { + // clients are not tracked, closing each client is the responsibility of users of this class + } +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java index 36c134b87a4..b7276d645b4 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java @@ -26,16 +26,53 @@ */ public interface XceiverClientFactory extends AutoCloseable { + /** + * Acquires a XceiverClientSpi connected to a container capable of + * storing the specified key. It does not consider the topology + * of the datanodes in the pipeline (e.g. closest datanode to the + * client) + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ XceiverClientSpi acquireClient(Pipeline pipeline) throws IOException; - void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClient); + /** + * Releases a XceiverClientSpi after use. + * + * @param client client to release + * @param invalidateClient if true, invalidates the client in cache + */ + void releaseClient(XceiverClientSpi client, boolean invalidateClient); + /** + * Acquires a XceiverClientSpi connected to a container for read. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ XceiverClientSpi acquireClientForReadData(Pipeline pipeline) throws IOException; - void releaseClientForReadData(XceiverClientSpi xceiverClient, + /** + * Releases a read XceiverClientSpi after use. + * + * @param client client to release + * @param invalidateClient if true, invalidates the client in cache + */ + void releaseClientForReadData(XceiverClientSpi client, boolean invalidateClient); + /** + * Acquires a XceiverClientSpi connected to a container capable of + * storing the specified key. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) throws IOException; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 5cd41edd38d..c02306f8af8 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -55,7 +55,6 @@ import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; -import java.util.concurrent.TimeoutException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -649,14 +648,6 @@ private void reconnect(DatanodeDetails dn) } } - @Override - public XceiverClientReply watchForCommit(long index) - throws InterruptedException, ExecutionException, TimeoutException, - IOException { - // there is no notion of watch for commit index in standalone pipeline - return null; - } - @Override public long getReplicatedMinCommitIndex() { return 0; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java index 2190391d18f..07b70441721 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hdds.scm; import java.io.IOException; -import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hdds.conf.Config; @@ -30,8 +29,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.hadoop.ozone.OzoneConfigKeys; -import org.apache.hadoop.ozone.OzoneSecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import com.google.common.annotations.VisibleForTesting; @@ -61,18 +58,15 @@ * without reestablishing connection. But the connection will be closed if * not being used for a period of time. */ -public class XceiverClientManager implements XceiverClientFactory { +public class XceiverClientManager extends XceiverClientCreator { private static final Logger LOG = LoggerFactory.getLogger(XceiverClientManager.class); - //TODO : change this to SCM configuration class - private final ConfigurationSource conf; + private final Cache clientCache; private final CacheMetrics cacheMetrics; - private ClientTrustManager trustManager; private static XceiverClientMetrics metrics; - private boolean isSecurityEnabled; - private final boolean topologyAwareRead; + /** * Creates a new XceiverClientManager for non secured ozone cluster. * For security enabled ozone cluster, client should use the other constructor @@ -87,15 +81,10 @@ public XceiverClientManager(ConfigurationSource conf) throws IOException { public XceiverClientManager(ConfigurationSource conf, ScmClientConfig clientConf, ClientTrustManager trustManager) throws IOException { + super(conf, trustManager); Preconditions.checkNotNull(clientConf); Preconditions.checkNotNull(conf); long staleThresholdMs = clientConf.getStaleThreshold(MILLISECONDS); - this.conf = conf; - this.isSecurityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf); - if (isSecurityEnabled) { - Preconditions.checkNotNull(trustManager); - this.trustManager = trustManager; - } this.clientCache = CacheBuilder.newBuilder() .recordStats() @@ -114,9 +103,6 @@ public void onRemoval( } } }).build(); - topologyAwareRead = conf.getBoolean( - OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_KEY, - OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_DEFAULT); cacheMetrics = CacheMetrics.create(clientCache, this); } @@ -127,50 +113,10 @@ public Cache getClientCache() { } /** - * Acquires a XceiverClientSpi connected to a container capable of - * storing the specified key. It does not consider the topology - * of the datanodes in the pipeline (e.g. closest datanode to the - * client) + * {@inheritDoc} * * If there is already a cached XceiverClientSpi, simply return * the cached otherwise create a new one. - * - * @param pipeline the container pipeline for the client connection - * @return XceiverClientSpi connected to a container - * @throws IOException if a XceiverClientSpi cannot be acquired - */ - @Override - public XceiverClientSpi acquireClient(Pipeline pipeline) - throws IOException { - return acquireClient(pipeline, false); - } - - /** - * Acquires a XceiverClientSpi connected to a container for read. - * - * If there is already a cached XceiverClientSpi, simply return - * the cached otherwise create a new one. - * - * @param pipeline the container pipeline for the client connection - * @return XceiverClientSpi connected to a container - * @throws IOException if a XceiverClientSpi cannot be acquired - */ - @Override - public XceiverClientSpi acquireClientForReadData(Pipeline pipeline) - throws IOException { - return acquireClient(pipeline, topologyAwareRead); - } - - /** - * Acquires a XceiverClientSpi connected to a container capable of - * storing the specified key. - * - * If there is already a cached XceiverClientSpi, simply return - * the cached otherwise create a new one. - * - * @param pipeline the container pipeline for the client connection - * @return XceiverClientSpi connected to a container - * @throws IOException if a XceiverClientSpi cannot be acquired */ @Override public XceiverClientSpi acquireClient(Pipeline pipeline, @@ -187,29 +133,6 @@ public XceiverClientSpi acquireClient(Pipeline pipeline, } } - /** - * Releases a XceiverClientSpi after use. - * - * @param client client to release - * @param invalidateClient if true, invalidates the client in cache - */ - @Override - public void releaseClient(XceiverClientSpi client, boolean invalidateClient) { - releaseClient(client, invalidateClient, false); - } - - /** - * Releases a read XceiverClientSpi after use. - * - * @param client client to release - * @param invalidateClient if true, invalidates the client in cache - */ - @Override - public void releaseClientForReadData(XceiverClientSpi client, - boolean invalidateClient) { - releaseClient(client, invalidateClient, topologyAwareRead); - } - @Override public void releaseClient(XceiverClientSpi client, boolean invalidateClient, boolean topologyAware) { @@ -227,39 +150,16 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, } } - private XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware) + protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware) throws IOException { - HddsProtos.ReplicationType type = pipeline.getType(); try { // create different client different pipeline node based on // network topology String key = getPipelineCacheKey(pipeline, topologyAware); - return clientCache.get(key, new Callable() { - @Override - public XceiverClientSpi call() throws Exception { - XceiverClientSpi client = null; - switch (type) { - case RATIS: - client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf, - trustManager); - break; - case STAND_ALONE: - client = new XceiverClientGrpc(pipeline, conf, trustManager); - break; - case EC: - client = new ECXceiverClientGrpc(pipeline, conf, trustManager); - break; - case CHAINED: - default: - throw new IOException("not implemented " + pipeline.getType()); - } - client.connect(); - return client; - } - }); + return clientCache.get(key, () -> newClient(pipeline)); } catch (Exception e) { throw new IOException( - "Exception getting XceiverClient: " + e.toString(), e); + "Exception getting XceiverClient: " + e, e); } } @@ -293,7 +193,7 @@ private String getPipelineCacheKey(Pipeline pipeline, } } - if (isSecurityEnabled) { + if (isSecurityEnabled()) { // Append user short name to key to prevent a different user // from using same instance of xceiverClient. try { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java index b67f4a56ec3..b0ef85cfbf7 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java @@ -29,8 +29,6 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Stream; @@ -66,6 +64,7 @@ import org.apache.ratis.rpc.RpcType; import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.ratis.util.JavaUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,12 +79,12 @@ public final class XceiverClientRatis extends XceiverClientSpi { public static XceiverClientRatis newXceiverClientRatis( org.apache.hadoop.hdds.scm.pipeline.Pipeline pipeline, ConfigurationSource ozoneConf) { - return newXceiverClientRatis(pipeline, ozoneConf, null); + return newXceiverClientRatis(pipeline, ozoneConf, null, null); } public static XceiverClientRatis newXceiverClientRatis( org.apache.hadoop.hdds.scm.pipeline.Pipeline pipeline, - ConfigurationSource ozoneConf, ClientTrustManager trustManager) { + ConfigurationSource ozoneConf, ClientTrustManager trustManager, ErrorInjector errorInjector) { final String rpcType = ozoneConf .get(ScmConfigKeys.HDDS_CONTAINER_RATIS_RPC_TYPE_KEY, ScmConfigKeys.HDDS_CONTAINER_RATIS_RPC_TYPE_DEFAULT); @@ -94,7 +93,7 @@ public static XceiverClientRatis newXceiverClientRatis( SecurityConfig(ozoneConf), trustManager); return new XceiverClientRatis(pipeline, SupportedRpcType.valueOfIgnoreCase(rpcType), - retryPolicy, tlsConfig, ozoneConf); + retryPolicy, tlsConfig, ozoneConf, errorInjector); } private final Pipeline pipeline; @@ -111,13 +110,14 @@ public static XceiverClientRatis newXceiverClientRatis( = XceiverClientManager.getXceiverClientMetrics(); private final RaftProtos.ReplicationLevel watchType; private final int majority; + private final ErrorInjector errorInjector; /** * Constructs a client. */ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType, RetryPolicy retryPolicy, GrpcTlsConfig tlsConfig, - ConfigurationSource configuration) { + ConfigurationSource configuration, ErrorInjector errorInjector) { super(); this.pipeline = pipeline; this.majority = (pipeline.getReplicationConfig().getRequiredNodes() / 2) + 1; @@ -138,11 +138,12 @@ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType, throw new IllegalArgumentException(watchType + " is not supported. " + "Currently only ALL_COMMITTED or MAJORITY_COMMITTED are supported"); } - LOG.info("WatchType {}. Majority {}, ", this.watchType, this.majority); + LOG.debug("WatchType {}. Majority {}, ", this.watchType, this.majority); if (LOG.isTraceEnabled()) { LOG.trace("new XceiverClientRatis for pipeline " + pipeline.getId(), new Throwable("TRACE")); } + this.errorInjector = errorInjector; } private long updateCommitInfosMap(RaftClientReply reply, RaftProtos.ReplicationLevel level) { @@ -249,6 +250,12 @@ public ConcurrentMap getCommitInfoMap() { private CompletableFuture sendRequestAsync( ContainerCommandRequestProto request) { + if (errorInjector != null) { + RaftClientReply response = errorInjector.getResponse(request, getClient().getId(), pipeline); + if (response != null) { + return CompletableFuture.completedFuture(response); + } + } return TracingUtil.executeInNewSpan( "XceiverClientRatis." + request.getCmdType().name(), () -> { @@ -295,46 +302,39 @@ private XceiverClientReply newWatchReply( } @Override - public XceiverClientReply watchForCommit(long index) - throws InterruptedException, ExecutionException, TimeoutException, - IOException { + public CompletableFuture watchForCommit(long index) { final long replicatedMin = getReplicatedMinCommitIndex(); if (replicatedMin >= index) { - return newWatchReply(index, "replicatedMin", replicatedMin); + return CompletableFuture.completedFuture(newWatchReply(index, "replicatedMin", replicatedMin)); } - try { - CompletableFuture replyFuture = getClient().async().watch(index, watchType); - final RaftClientReply reply = replyFuture.get(); + final CompletableFuture replyFuture = new CompletableFuture<>(); + getClient().async().watch(index, watchType).thenAccept(reply -> { final long updated = updateCommitInfosMap(reply, watchType); - Preconditions.checkState(updated >= index, "Returned index " + updated + " is smaller than expected " + index); - return newWatchReply(index, watchType, updated); - } catch (Exception e) { + Preconditions.checkState(updated >= index, "Returned index " + updated + " < expected " + index); + replyFuture.complete(newWatchReply(index, watchType, updated)); + }).exceptionally(e -> { LOG.warn("{} way commit failed on pipeline {}", watchType, pipeline, e); - Throwable t = - HddsClientUtils.containsException(e, GroupMismatchException.class); - if (t != null) { - throw e; - } - if (watchType == ReplicationLevel.ALL_COMMITTED) { - Throwable nre = - HddsClientUtils.containsException(e, NotReplicatedException.class); - Collection commitInfoProtoList; + final boolean isGroupMismatch = HddsClientUtils.containsException(e, GroupMismatchException.class) != null; + if (!isGroupMismatch && watchType == ReplicationLevel.ALL_COMMITTED) { + final Throwable nre = HddsClientUtils.containsException(e, NotReplicatedException.class); if (nre instanceof NotReplicatedException) { // If NotReplicatedException is thrown from the Datanode leader // we can save one watch request round trip by using the CommitInfoProto // in the NotReplicatedException - commitInfoProtoList = ((NotReplicatedException) nre).getCommitInfos(); + final Collection commitInfoProtoList = ((NotReplicatedException) nre).getCommitInfos(); + replyFuture.complete(handleFailedAllCommit(index, commitInfoProtoList)); } else { - final RaftClientReply reply = getClient().async() - .watch(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED) - .get(); - commitInfoProtoList = reply.getCommitInfos(); + getClient().async().watch(index, ReplicationLevel.MAJORITY_COMMITTED) + .thenApply(reply -> handleFailedAllCommit(index, reply.getCommitInfos())) + .whenComplete(JavaUtils.asBiConsumer(replyFuture)); } - return handleFailedAllCommit(index, commitInfoProtoList); + } else { + replyFuture.completeExceptionally(e); } - throw e; - } + return null; + }); + return replyFuture; } private XceiverClientReply handleFailedAllCommit(long index, Collection commitInfoProtoList) { @@ -374,8 +374,7 @@ public XceiverClientReply sendCommandAsync( CompletableFuture containerCommandResponse = raftClientReply.whenComplete((reply, e) -> { if (LOG.isDebugEnabled()) { - LOG.debug("received reply {} for request: cmdType={} containerID={}" - + " pipelineID={} traceID={} exception: {}", reply, + LOG.debug("received reply {} for request: cmdType={}, containerID={}, pipelineID={}, traceID={}", reply, request.getCmdType(), request.getContainerID(), request.getPipelineID(), request.getTraceID(), e); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java index 6c5f9a0a989..f5a7c0ad550 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java @@ -254,9 +254,8 @@ public static Throwable checkForException(Exception e) { // This will return the underlying expected exception if it exists // in an exception trace. Otherwise, returns null. - public static Throwable containsException(Exception e, + public static Throwable containsException(Throwable t, Class expectedExceptionClass) { - Throwable t = e; while (t != null) { if (expectedExceptionClass.isInstance(t)) { return t; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractCommitWatcher.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractCommitWatcher.java index fb489d0d0c6..61bc73420e6 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractCommitWatcher.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/AbstractCommitWatcher.java @@ -37,7 +37,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; /** @@ -127,19 +126,17 @@ XceiverClientReply watchOnLastIndex() throws IOException { * @return minimum commit index replicated to all nodes * @throws IOException IOException in case watch gets timed out */ - XceiverClientReply watchForCommit(long commitIndex) - throws IOException { + CompletableFuture watchForCommitAsync(long commitIndex) { final MemoizedSupplier> supplier = JavaUtils.memoize(CompletableFuture::new); final CompletableFuture f = replies.compute(commitIndex, (key, value) -> value != null ? value : supplier.get()); if (!supplier.isInitialized()) { // future already exists - return f.join(); + return f; } - try { - final XceiverClientReply reply = client.watchForCommit(commitIndex); + return client.watchForCommit(commitIndex).thenApply(reply -> { f.complete(reply); final CompletableFuture removed = replies.remove(commitIndex); Preconditions.checkState(removed == f); @@ -147,11 +144,17 @@ XceiverClientReply watchForCommit(long commitIndex) final long index = reply != null ? reply.getLogIndex() : 0; adjustBuffers(index); return reply; + }); + } + + XceiverClientReply watchForCommit(long commitIndex) throws IOException { + try { + return watchForCommitAsync(commitIndex).get(); } catch (InterruptedException e) { // Re-interrupt the thread while catching InterruptedException Thread.currentThread().interrupt(); throw getIOExceptionForWatchForCommit(commitIndex, e); - } catch (TimeoutException | ExecutionException e) { + } catch (ExecutionException e) { throw getIOExceptionForWatchForCommit(commitIndex, e); } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java index aab70a692e3..e88b097c499 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java @@ -27,6 +27,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; @@ -94,11 +95,14 @@ public class BlockOutputStream extends OutputStream { KeyValue.newBuilder().setKey(FULL_CHUNK).build(); private AtomicReference blockID; + // planned block full size + private long blockSize; + private AtomicBoolean eofSent = new AtomicBoolean(false); private final AtomicReference previousChunkInfo = new AtomicReference<>(); private final BlockData.Builder containerBlockData; - private XceiverClientFactory xceiverClientFactory; + private volatile XceiverClientFactory xceiverClientFactory; private XceiverClientSpi xceiverClient; private OzoneClientConfig config; private StreamBufferArgs streamBufferArgs; @@ -149,6 +153,7 @@ public class BlockOutputStream extends OutputStream { private Pipeline pipeline; private final ContainerClientMetrics clientMetrics; private boolean allowPutBlockPiggybacking; + private boolean supportIncrementalChunkList; private CompletableFuture lastFlushFuture; private CompletableFuture allPendingFlushFutures = CompletableFuture.completedFuture(null); @@ -164,6 +169,7 @@ public class BlockOutputStream extends OutputStream { @SuppressWarnings("checkstyle:ParameterNumber") public BlockOutputStream( BlockID blockID, + long blockSize, XceiverClientFactory xceiverClientManager, Pipeline pipeline, BufferPool bufferPool, @@ -175,6 +181,7 @@ public BlockOutputStream( this.xceiverClientFactory = xceiverClientManager; this.config = config; this.blockID = new AtomicReference<>(blockID); + this.blockSize = blockSize; replicationIndex = pipeline.getReplicaIndex(pipeline.getClosestNode()); KeyValue keyValue = KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build(); @@ -189,8 +196,13 @@ public BlockOutputStream( } this.containerBlockData = BlockData.newBuilder().setBlockID( blkIDBuilder.build()).addMetadata(keyValue); + this.pipeline = pipeline; // tell DataNode I will send incremental chunk list - if (config.getIncrementalChunkList()) { + // EC does not support incremental chunk list. + this.supportIncrementalChunkList = config.getIncrementalChunkList() && + this instanceof RatisBlockOutputStream && allDataNodesSupportPiggybacking(); + LOG.debug("incrementalChunkList is {}", supportIncrementalChunkList); + if (supportIncrementalChunkList) { this.containerBlockData.addMetadata(INCREMENTAL_CHUNK_LIST_KV); this.lastChunkBuffer = DIRECT_BUFFER_POOL.getBuffer(config.getStreamBufferSize()); this.lastChunkOffset = 0; @@ -204,7 +216,8 @@ public BlockOutputStream( this.token.encodeToUrlString(); //number of buffers used before doing a flush - refreshCurrentBuffer(); + currentBuffer = null; + currentBufferRemaining = 0; flushPeriod = (int) (streamBufferArgs.getStreamBufferFlushSize() / streamBufferArgs .getStreamBufferSize()); @@ -223,16 +236,17 @@ public BlockOutputStream( checksum = new Checksum(config.getChecksumType(), config.getBytesPerChecksum()); this.clientMetrics = clientMetrics; - this.pipeline = pipeline; this.streamBufferArgs = streamBufferArgs; this.allowPutBlockPiggybacking = config.getEnablePutblockPiggybacking() && allDataNodesSupportPiggybacking(); + LOG.debug("PutBlock piggybacking is {}", allowPutBlockPiggybacking); } private boolean allDataNodesSupportPiggybacking() { // return true only if all DataNodes in the pipeline are on a version // that supports PutBlock piggybacking. for (DatanodeDetails dn : pipeline.getNodes()) { + LOG.debug("dn = {}, version = {}", dn, dn.getCurrentVersion()); if (dn.getCurrentVersion() < COMBINED_PUTBLOCK_WRITECHUNK_RPC.toProtoValue()) { return false; @@ -241,12 +255,6 @@ private boolean allDataNodesSupportPiggybacking() { return true; } - synchronized void refreshCurrentBuffer() { - currentBuffer = bufferPool.getCurrentBuffer(); - currentBufferRemaining = - currentBuffer != null ? currentBuffer.remaining() : 0; - } - public BlockID getBlockID() { return blockID.get(); } @@ -405,42 +413,44 @@ private void updatePutBlockLength() { * @param len length of data to write * @throws IOException if error occurred */ - - // In this case, the data is already cached in the currentBuffer. public synchronized void writeOnRetry(long len) throws IOException { if (len == 0) { return; } + + // In this case, the data from the failing (previous) block already cached in the allocated buffers in + // the BufferPool. For each pending buffers in the BufferPool, we sequentially flush it and wait synchronously. + + List allocatedBuffers = bufferPool.getAllocatedBuffers(); if (LOG.isDebugEnabled()) { - LOG.debug("Retrying write length {} for blockID {}", len, blockID); + LOG.debug("{}: Retrying write length {} on target blockID {}, {} buffers", this, len, blockID, + allocatedBuffers.size()); } Preconditions.checkArgument(len <= streamBufferArgs.getStreamBufferMaxSize()); int count = 0; - List allocatedBuffers = bufferPool.getAllocatedBuffers(); while (len > 0) { ChunkBuffer buffer = allocatedBuffers.get(count); long writeLen = Math.min(buffer.position(), len); - if (!buffer.hasRemaining()) { - writeChunk(buffer); - } len -= writeLen; count++; writtenDataLength += writeLen; - // we should not call isBufferFull/shouldFlush here. - // The buffer might already be full as whole data is already cached in - // the buffer. We should just validate - // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to - // call for handling full buffer/flush buffer condition. - if (writtenDataLength % streamBufferArgs.getStreamBufferFlushSize() == 0) { - // reset the position to zero as now we will be reading the - // next buffer in the list - updateWriteChunkLength(); - updatePutBlockLength(); - CompletableFuture putBlockResultFuture = executePutBlock(false, false); - recordWatchForCommitAsync(putBlockResultFuture); + updateWriteChunkLength(); + updatePutBlockLength(); + LOG.debug("Write chunk on retry buffer = {}", buffer); + CompletableFuture putBlockFuture; + if (allowPutBlockPiggybacking) { + putBlockFuture = writeChunkAndPutBlock(buffer, false); + } else { + writeChunk(buffer); + putBlockFuture = executePutBlock(false, false); } - if (writtenDataLength == streamBufferArgs.getStreamBufferMaxSize()) { - handleFullBuffer(); + CompletableFuture watchForCommitAsync = watchForCommitAsync(putBlockFuture); + try { + watchForCommitAsync.get(); + } catch (InterruptedException e) { + handleInterruptedException(e, true); + } catch (ExecutionException e) { + handleExecutionException(e); } } } @@ -466,14 +476,6 @@ private void handleFullBuffer() throws IOException { void releaseBuffersOnException() { } - // It may happen that once the exception is encountered , we still might - // have successfully flushed up to a certain index. Make sure the buffers - // only contain data which have not been sufficiently replicated - private void adjustBuffersOnException() { - releaseBuffersOnException(); - refreshCurrentBuffer(); - } - /** * Watch for a specific commit index. */ @@ -530,15 +532,17 @@ CompletableFuture executePutBlock(boolean close, final XceiverClientReply asyncReply; try { BlockData blockData = containerBlockData.build(); - LOG.debug("sending PutBlock {}", blockData); + LOG.debug("sending PutBlock {} flushPos {}", blockData, flushPos); - if (config.getIncrementalChunkList()) { + if (supportIncrementalChunkList) { // remove any chunks in the containerBlockData list. // since they are sent. containerBlockData.clearChunks(); } - asyncReply = putBlockAsync(xceiverClient, blockData, close, tokenString); + // if block is full, send the eof + boolean isBlockFull = (blockSize != -1 && flushPos == blockSize); + asyncReply = putBlockAsync(xceiverClient, blockData, close || isBlockFull, tokenString); CompletableFuture future = asyncReply.getResponse(); flushFuture = future.thenApplyAsync(e -> { try { @@ -550,6 +554,7 @@ CompletableFuture executePutBlock(boolean close, if (getIoException() == null && !force) { handleSuccessfulPutBlock(e.getPutBlock().getCommittedBlockLength(), asyncReply, flushPos, byteBufferList); + eofSent.set(close || isBlockFull); } return e; }, responseExecutor).exceptionally(e -> { @@ -617,6 +622,9 @@ private CompletableFuture writeChunkAndPutBlock(ChunkBuffer buff protected void handleFlush(boolean close) throws IOException { try { handleFlushInternal(close); + if (close) { + waitForAllPendingFlushes(); + } } catch (ExecutionException e) { handleExecutionException(e); } catch (InterruptedException ex) { @@ -659,6 +667,17 @@ private void handleFlushInternal(boolean close) } } + public void waitForAllPendingFlushes() throws IOException { + // When closing, must wait for all flush futures to complete. + try { + allPendingFlushFutures.get(); + } catch (InterruptedException e) { + handleInterruptedException(e, true); + } catch (ExecutionException e) { + handleExecutionException(e); + } + } + private synchronized CompletableFuture handleFlushInternalSynchronized(boolean close) throws IOException { CompletableFuture putBlockResultFuture = null; // flush the last chunk data residing on the currentBuffer @@ -690,7 +709,7 @@ private synchronized CompletableFuture handleFlushInternalSynchronized(boo // There're no pending written data, but there're uncommitted data. updatePutBlockLength(); putBlockResultFuture = executePutBlock(close, false); - } else if (close) { + } else if (close && !eofSent.get()) { // forcing an "empty" putBlock if stream is being closed without new // data since latest flush - we need to send the "EOF" flag updatePutBlockLength(); @@ -724,6 +743,7 @@ public void close() throws IOException { // Preconditions.checkArgument(buffer.position() == 0); // bufferPool.checkBufferPoolEmpty(); } else { + waitForAllPendingFlushes(); cleanup(false); } } @@ -767,7 +787,7 @@ public void setIoException(Exception e) { void cleanup() { } - public void cleanup(boolean invalidateClient) { + public synchronized void cleanup(boolean invalidateClient) { if (xceiverClientFactory != null) { xceiverClientFactory.releaseClient(xceiverClient, invalidateClient); } @@ -795,7 +815,6 @@ void checkOpen() throws IOException { if (isClosed()) { throw new IOException("BlockOutputStream has been closed."); } else if (getIoException() != null) { - adjustBuffersOnException(); throw getIoException(); } } @@ -866,7 +885,7 @@ private CompletableFuture writeChunkToContainer( try { BlockData blockData = null; - if (config.getIncrementalChunkList()) { + if (supportIncrementalChunkList) { updateBlockDataForWriteChunk(chunk); } else { containerBlockData.addChunks(chunkInfo); @@ -880,7 +899,7 @@ private CompletableFuture writeChunkToContainer( blockData = containerBlockData.build(); LOG.debug("piggyback chunk list {}", blockData); - if (config.getIncrementalChunkList()) { + if (supportIncrementalChunkList) { // remove any chunks in the containerBlockData list. // since they are sent. containerBlockData.clearChunks(); @@ -1132,7 +1151,6 @@ void handleInterruptedException(Exception ex, */ private void handleExecutionException(Exception ex) throws IOException { setIoException(ex); - adjustBuffersOnException(); throw getIoException(); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java index bbb3f30687a..12ca9978c68 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java @@ -80,7 +80,7 @@ public ECBlockOutputStream( ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs, Supplier executorServiceSupplier ) throws IOException { - super(blockID, xceiverClientManager, + super(blockID, -1, xceiverClientManager, pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs, executorServiceSupplier); // In EC stream, there will be only one node in pipeline. this.datanodeDetails = pipeline.getClosestNode(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java index c0e99a5b4a0..d32c37eba6c 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java @@ -72,6 +72,7 @@ public class RatisBlockOutputStream extends BlockOutputStream @SuppressWarnings("checkstyle:ParameterNumber") public RatisBlockOutputStream( BlockID blockID, + long blockSize, XceiverClientFactory xceiverClientManager, Pipeline pipeline, BufferPool bufferPool, @@ -80,7 +81,7 @@ public RatisBlockOutputStream( ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs, Supplier blockOutputStreamResourceProvider ) throws IOException { - super(blockID, xceiverClientManager, pipeline, + super(blockID, blockSize, xceiverClientManager, pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs, blockOutputStreamResourceProvider); this.commitWatcher = new CommitWatcher(bufferPool, getXceiverClient()); } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java index df55b5bf57a..d3425b7d2b0 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java @@ -174,6 +174,7 @@ private BlockOutputStream createBlockOutputStream(BufferPool bufferPool) return new RatisBlockOutputStream( new BlockID(1L, 1L), + -1, xcm, pipeline, bufferPool, @@ -276,7 +277,7 @@ public ReplicationType getPipelineType() { } @Override - public XceiverClientReply watchForCommit(long index) { + public CompletableFuture watchForCommit(long index) { final ContainerCommandResponseProto.Builder builder = ContainerCommandResponseProto.newBuilder() .setCmdType(Type.WriteChunk) @@ -284,7 +285,7 @@ public XceiverClientReply watchForCommit(long index) { final XceiverClientReply xceiverClientReply = new XceiverClientReply( CompletableFuture.completedFuture(builder.build())); xceiverClientReply.setLogIndex(index); - return xceiverClientReply; + return CompletableFuture.completedFuture(xceiverClientReply); } @Override diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml index aeec60f9790..29cb513bb6f 100644 --- a/hadoop-hdds/common/pom.xml +++ b/hadoop-hdds/common/pom.xml @@ -64,6 +64,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.commons commons-lang3 + + commons-collections + commons-collections + commons-io commons-io @@ -72,6 +76,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> com.fasterxml.jackson.core jackson-annotations + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + com.fasterxml.jackson.datatype jackson-datatype-jsr310 @@ -93,6 +105,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> jakarta.annotation jakarta.annotation-api + + jakarta.xml.bind + jakarta.xml.bind-api + io.dropwizard.metrics @@ -114,6 +130,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ratis + ratis-common + ratis-netty org.apache.ratis @@ -122,6 +142,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis-grpc org.apache.ratis + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-client + + + org.apache.ratis + ratis-thirdparty-misc + com.google.errorprone error_prone_annotations @@ -134,6 +166,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> bcpkix-jdk18on ${bouncycastle.version} + + org.bouncycastle + bcprov-jdk18on + + + org.bouncycastle + bcutil-jdk18on + commons-validator commons-validator @@ -142,10 +182,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> io.jaegertracing jaeger-client + + io.jaegertracing + jaeger-core + org.jetbrains.kotlin kotlin-stdlib + + io.opentracing + opentracing-api + io.opentracing opentracing-util @@ -158,6 +206,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.reflections reflections + + org.slf4j + slf4j-api + org.apache.ozone hdds-interface-client diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java index abdd25f0680..87707f75dc4 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java @@ -367,6 +367,9 @@ private HddsConfigKeys() { public static final String HDDS_DATANODE_HANDLER_COUNT_KEY = "hdds.datanode.handler.count"; public static final int HDDS_DATANODE_HANDLER_COUNT_DEFAULT = 10; + public static final String HDDS_DATANODE_READ_THREADPOOL_KEY = + "hdds.datanode.read.threadpool"; + public static final int HDDS_DATANODE_READ_THREADPOOL_DEFAULT = 10; public static final String HDDS_DATANODE_HTTP_BIND_HOST_DEFAULT = "0.0.0.0"; public static final int HDDS_DATANODE_HTTP_BIND_PORT_DEFAULT = 9882; public static final int HDDS_DATANODE_HTTPS_BIND_PORT_DEFAULT = 9883; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/RatisConfUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/RatisConfUtils.java new file mode 100644 index 00000000000..3b247273abd --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/RatisConfUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.conf; + +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.SizeInBytes; + +/** + * Utilities for Ratis configurations. + */ +public class RatisConfUtils { + /** For {@link GrpcConfigKeys}. */ + public static class Grpc { + /** For setting {@link GrpcConfigKeys#setMessageSizeMax(RaftProperties, SizeInBytes)}. */ + public static void setMessageSizeMax(RaftProperties properties, int max) { + Preconditions.assertTrue(max > 0, () -> "max = " + max + " <= 0"); + + final long logAppenderBufferByteLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties).getSize(); + Preconditions.assertTrue(max >= logAppenderBufferByteLimit, + () -> "max = " + max + " < logAppenderBufferByteLimit = " + logAppenderBufferByteLimit); + + // Need an 1MB gap; see RATIS-2135 + GrpcConfigKeys.setMessageSizeMax(properties, SizeInBytes.valueOf(max + SizeInBytes.ONE_MB.getSize())); + } + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java index e5ffbd024b1..01f508d257c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java @@ -391,6 +391,9 @@ public static DatanodeDetails.Builder newBuilder( } if (datanodeDetailsProto.hasCurrentVersion()) { builder.setCurrentVersion(datanodeDetailsProto.getCurrentVersion()); + } else { + // fallback to version 1 if not present + builder.setCurrentVersion(DatanodeVersion.SEPARATE_RATIS_PORTS_AVAILABLE.toProtoValue()); } return builder; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index 593764ee09c..36d4dbd45a2 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -228,15 +228,27 @@ public final class ScmConfigKeys { "ozone.scm.handler.count.key"; public static final String OZONE_SCM_CLIENT_HANDLER_COUNT_KEY = "ozone.scm.client.handler.count.key"; + public static final String OZONE_SCM_CLIENT_READ_THREADPOOL_KEY = + "ozone.scm.client.read.threadpool"; + public static final int OZONE_SCM_CLIENT_READ_THREADPOOL_DEFAULT = 10; public static final String OZONE_SCM_BLOCK_HANDLER_COUNT_KEY = "ozone.scm.block.handler.count.key"; + public static final String OZONE_SCM_BLOCK_READ_THREADPOOL_KEY = + "ozone.scm.block.read.threadpool"; + public static final int OZONE_SCM_BLOCK_READ_THREADPOOL_DEFAULT = 10; public static final String OZONE_SCM_DATANODE_HANDLER_COUNT_KEY = "ozone.scm.datanode.handler.count.key"; + public static final String OZONE_SCM_DATANODE_READ_THREADPOOL_KEY = + "ozone.scm.datanode.read.threadpool"; + public static final int OZONE_SCM_DATANODE_READ_THREADPOOL_DEFAULT = 10; public static final int OZONE_SCM_HANDLER_COUNT_DEFAULT = 100; public static final String OZONE_SCM_SECURITY_HANDLER_COUNT_KEY = "ozone.scm.security.handler.count.key"; public static final int OZONE_SCM_SECURITY_HANDLER_COUNT_DEFAULT = 2; + public static final String OZONE_SCM_SECURITY_READ_THREADPOOL_KEY = + "ozone.scm.security.read.threadpool"; + public static final int OZONE_SCM_SECURITY_READ_THREADPOOL_DEFAULT = 1; public static final String OZONE_SCM_DEADNODE_INTERVAL = "ozone.scm.dead.node.interval"; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index f6529e84bdc..9ac32c469ca 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -22,8 +22,8 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hdds.HddsUtils; @@ -176,9 +176,9 @@ public static IOException getIOExceptionForSendCommand( * @return reply containing the min commit index replicated to all or majority * servers in case of a failure */ - public abstract XceiverClientReply watchForCommit(long index) - throws InterruptedException, ExecutionException, TimeoutException, - IOException; + public CompletableFuture watchForCommit(long index) { + return CompletableFuture.completedFuture(null); + } /** * returns the min commit index replicated to all servers. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java index 0c23a846563..79db6985e76 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.security.exception.OzoneSecurityException; import org.apache.hadoop.hdds.security.ssl.ReloadingX509KeyManager; import org.apache.hadoop.hdds.security.ssl.ReloadingX509TrustManager; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; import org.apache.hadoop.hdds.security.x509.exception.CertificateException; @@ -156,13 +157,13 @@ boolean verifySignature(byte[] data, byte[] signature, X509Certificate cert) throws CertificateException; /** - * Returns a CSR builder that can be used to create a Certificate sigining - * request. + * Returns a CertificateSignRequest Builder object, that can be used to configure the sign request + * which we use to get a signed certificate from our CA server implementation. * - * @return CertificateSignRequest.Builder + * @return CertificateSignRequest.Builder a {@link CertificateSignRequest} + * based on which the certificate may be issued to this client. */ - CertificateSignRequest.Builder getCSRBuilder() - throws CertificateException; + CertificateSignRequest.Builder configureCSRBuilder() throws SCMSecurityException; default void assertValidKeysAndCertificate() throws OzoneSecurityException { try { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateSignRequest.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateSignRequest.java index c1cc6712154..1f04e868a85 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateSignRequest.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateSignRequest.java @@ -157,34 +157,43 @@ public static ASN1Set getPkcs9ExtRequest(PKCS10CertificationRequest csr) throw new CertificateException("No PKCS#9 extension found in CSR"); } - private PKCS10CertificationRequest generateCSR() throws - OperatorCreationException { - X500Name dnName = getDistinguishedName(subject, scmID, clusterID); - PKCS10CertificationRequestBuilder p10Builder = - new JcaPKCS10CertificationRequestBuilder(dnName, keyPair.getPublic()); - - ContentSigner contentSigner = - new JcaContentSignerBuilder(config.getSignatureAlgo()) - .setProvider(config.getProvider()) - .build(keyPair.getPrivate()); - - if (extensions != null) { - p10Builder.addAttribute( - PKCSObjectIdentifiers.pkcs_9_at_extensionRequest, extensions); - } - return p10Builder.build(contentSigner); - } - public static String getEncodedString(PKCS10CertificationRequest request) - throws IOException { - PemObject pemObject = - new PemObject("CERTIFICATE REQUEST", request.getEncoded()); + /** + * Encodes this CertificateSignRequest to a String representation, that can be transferred over the wire to + * the CA server for signing. + * + * @return the Certificate Sign Request encoded to a String + * @throws IOException if an error occurs during encoding. + */ + public String toEncodedFormat() throws IOException { StringWriter str = new StringWriter(); try (JcaPEMWriter pemWriter = new JcaPEMWriter(str)) { + PemObject pemObject = new PemObject("CERTIFICATE REQUEST", generateCSR().getEncoded()); pemWriter.writeObject(pemObject); } return str.toString(); } + //TODO: this should be private once the server side of removing PKCS10CertReq class is done. + public PKCS10CertificationRequest generateCSR() throws IOException { + X500Name dnName = getDistinguishedName(subject, scmID, clusterID); + PKCS10CertificationRequestBuilder p10Builder = + new JcaPKCS10CertificationRequestBuilder(dnName, keyPair.getPublic()); + + try { + ContentSigner contentSigner = + new JcaContentSignerBuilder(config.getSignatureAlgo()) + .setProvider(config.getProvider()) + .build(keyPair.getPrivate()); + + if (extensions != null) { + p10Builder.addAttribute( + PKCSObjectIdentifiers.pkcs_9_at_extensionRequest, extensions); + } + return p10Builder.build(contentSigner); + } catch (OperatorCreationException e) { + throw new IOException(e); + } + } /** * Gets a CertificateRequest Object from PEM encoded CSR. @@ -413,7 +422,7 @@ private Extensions createExtensions() throws IOException { extensions.toArray(new Extension[extensions.size()])); } - public PKCS10CertificationRequest build() throws SCMSecurityException { + public CertificateSignRequest build() throws SCMSecurityException { Preconditions.checkNotNull(key, "KeyPair cannot be null"); Preconditions.checkArgument(StringUtils.isNotBlank(subject), "Subject " + "cannot be blank"); @@ -421,15 +430,11 @@ public PKCS10CertificationRequest build() throws SCMSecurityException { try { CertificateSignRequest csr = new CertificateSignRequest(subject, scmID, clusterID, key, config, createExtensions()); - return csr.generateCSR(); + return csr; } catch (IOException ioe) { throw new CertificateException(String.format("Unable to create " + "extension for certificate sign request for %s.", getDistinguishedName(subject, scmID, clusterID)), ioe.getCause()); - } catch (OperatorCreationException ex) { - throw new CertificateException(String.format("Unable to create " + - "certificate sign request for %s.", - getDistinguishedName(subject, scmID, clusterID)), ex.getCause()); } } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java index 38ebc80b27e..c61502ff4a8 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java @@ -126,7 +126,7 @@ public final class OzoneConfigKeys { public static final String OZONE_FS_HSYNC_ENABLED = "ozone.fs.hsync.enabled"; public static final boolean OZONE_FS_HSYNC_ENABLED_DEFAULT - = true; + = false; /** * hsync lease soft limit. @@ -583,6 +583,13 @@ public final class OzoneConfigKeys { public static final boolean OZONE_CLIENT_KEY_LATEST_VERSION_LOCATION_DEFAULT = true; + public static final String OZONE_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS = + "ozone.client.server-defaults.validity.period.ms"; + + public static final long + OZONE_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT = + TimeUnit.HOURS.toMillis(1); // 1 hour + public static final String OZONE_FLEXIBLE_FQDN_RESOLUTION_ENABLED = "ozone.network.flexible.fqdn.resolution.enabled"; public static final boolean OZONE_FLEXIBLE_FQDN_RESOLUTION_ENABLED_DEFAULT = diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 186b5aaded9..20c1bed89be 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -1140,6 +1140,36 @@ The default value is 100. + + ozone.scm.client.read.threadpool + 10 + OZONE, MANAGEMENT, PERFORMANCE + + The number of threads in RPC server reading from the socket used by Client to access SCM. + This config overrides Hadoop configuration "ipc.server.read.threadpool.size" for SCMClientProtocolServer. + The default value is 10. + + + + ozone.scm.block.read.threadpool + 10 + OZONE, MANAGEMENT, PERFORMANCE + + The number of threads in RPC server reading from the socket when accessing blocks. + This config overrides Hadoop configuration "ipc.server.read.threadpool.size" for SCMBlockProtocolServer. + The default value is 10. + + + + ozone.scm.datanode.read.threadpool + 10 + OZONE, MANAGEMENT, PERFORMANCE + + The number of threads in RPC server reading from the socket used by DataNode to access SCM. + This config overrides Hadoop configuration "ipc.server.read.threadpool.size" for SCMDatanodeProtocolServer. + The default value is 10. + + hdds.heartbeat.interval 30s @@ -2493,6 +2523,16 @@ OZONE, HDDS, SECURITY Threads configured for SCMSecurityProtocolServer. + + ozone.scm.security.read.threadpool + 1 + OZONE, HDDS, SECURITY, PERFORMANCE + + The number of threads in RPC server reading from the socket when performing security related operations with SCM. + This config overrides Hadoop configuration "ipc.server.read.threadpool.size" for SCMSecurityProtocolServer. + The default value is 1. + + ozone.scm.security.service.address @@ -2935,6 +2975,16 @@ service endpoints. + + hdds.datanode.read.threadpool + 10 + OZONE, HDDS, PERFORMANCE + + The number of threads in RPC server reading from the socket for Datanode client service endpoints. + This config overrides Hadoop configuration "ipc.server.read.threadpool.size" for HddsDatanodeClientProtocolServer. + The default value is 10. + + ozone.client.failover.max.attempts 500 @@ -3521,6 +3571,19 @@ + + ozone.client.server-defaults.validity.period.ms + OZONE, CLIENT, SECURITY + 3600000 + + The amount of milliseconds after which cached server defaults are updated. + + By default this parameter is set to 1 hour. + Support multiple time unit suffix(case insensitive). + If no time unit is specified then milliseconds is assumed. + + + ozone.scm.info.wait.duration OZONE, SCM, OM @@ -4163,7 +4226,7 @@ ozone.fs.hsync.enabled - true + false OZONE, CLIENT Enable hsync/hflush. By default they are disabled. diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/conf/TestRatisConfUtils.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/conf/TestRatisConfUtils.java new file mode 100644 index 00000000000..50bf524f025 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/conf/TestRatisConfUtils.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.conf; + +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.util.SizeInBytes; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test {@link RatisConfUtils}. + */ +public class TestRatisConfUtils { + private static final Logger LOG = LoggerFactory.getLogger(TestRatisConfUtils.class); + + @Test + void testGrpcSetMessageSizeMax() { + final RaftProperties properties = new RaftProperties(); + + final int logAppenderBufferByteLimit = 1000; + + // setMessageSizeMax without setBufferByteLimit + Assertions.assertThrows(IllegalStateException.class, + () -> RatisConfUtils.Grpc.setMessageSizeMax(properties, logAppenderBufferByteLimit)); + + RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties, SizeInBytes.valueOf(logAppenderBufferByteLimit)); + + // setMessageSizeMax with a value smaller than logAppenderBufferByteLimit + Assertions.assertThrows(IllegalStateException.class, + () -> RatisConfUtils.Grpc.setMessageSizeMax(properties, logAppenderBufferByteLimit - 1)); + + // setMessageSizeMax with the correct logAppenderBufferByteLimit + RatisConfUtils.Grpc.setMessageSizeMax(properties, logAppenderBufferByteLimit); + + final SizeInBytes max = GrpcConfigKeys.messageSizeMax(properties, LOG::info); + Assertions.assertEquals(SizeInBytes.ONE_MB.getSize(), max.getSize() - logAppenderBufferByteLimit); + } +} 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 b05deaa0d66..aeb1e207e70 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,6 +17,7 @@ */ package org.apache.hadoop.hdds.protocol; +import org.apache.hadoop.hdds.DatanodeVersion; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.junit.jupiter.api.Test; @@ -48,6 +49,24 @@ void protoIncludesNewPortsOnlyForV1() { assertPorts(protoV1, ALL_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(); + HddsProtos.DatanodeDetailsProto.Builder protoBuilder = + dn.toProtoBuilder(DEFAULT_VERSION.toProtoValue()); + 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()); + DatanodeDetails dn3 = DatanodeDetails.newBuilder(protoBuilder.build()).build(); + assertEquals(DatanodeVersion.CURRENT.toProtoValue(), dn3.getCurrentVersion()); + } + public static void assertPorts(HddsProtos.DatanodeDetailsProto dn, Set expectedPorts) throws IllegalArgumentException { assertEquals(expectedPorts.size(), dn.getPortsCount()); diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml index 7a341bd66a8..d73bea95895 100644 --- a/hadoop-hdds/container-service/pom.xml +++ b/hadoop-hdds/container-service/pom.xml @@ -34,10 +34,35 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-common + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-interface-client + + + org.apache.ozone + hdds-interface-server + + + org.apache.ozone + hdds-managed-rocksdb + + org.apache.commons commons-compress + + org.apache.logging.log4j + log4j-api + + + commons-io + commons-io + com.github.luben zstd-jni @@ -54,6 +79,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> commons-codec commons-codec + + commons-collections + commons-collections + io.dropwizard.metrics metrics-core @@ -70,6 +99,15 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> provided + + org.apache.ratis + ratis-server + + + + jakarta.annotation + jakarta.annotation-api + jakarta.xml.bind jakarta.xml.bind-api @@ -78,10 +116,22 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.glassfish.jaxb jaxb-runtime + + info.picocli + picocli + io.netty netty-transport + + io.netty + netty-buffer + + + io.netty + netty-common + io.netty netty-codec @@ -90,10 +140,76 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> io.netty netty-handler + + io.opentracing + opentracing-api + + + io.opentracing + opentracing-util + + + org.apache.commons + commons-lang3 + org.apache.commons commons-text + + org.apache.ratis + ratis-client + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-grpc + + + org.apache.ratis + ratis-netty + + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-server-api + + + org.apache.ratis + ratis-thirdparty-misc + + + + org.rocksdb + rocksdbjni + + + org.slf4j + slf4j-api + + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.google.guava + guava + + + com.google.protobuf + protobuf-java + diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeClientProtocolServer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeClientProtocolServer.java index 8b0b3a7ca23..b0308f79ed5 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeClientProtocolServer.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeClientProtocolServer.java @@ -42,6 +42,8 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_CLIENT_ADDRESS_KEY; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_READ_THREADPOOL_KEY; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_READ_THREADPOOL_DEFAULT; import static org.apache.hadoop.hdds.HddsUtils.preserveThreadName; import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.CLIENT_RPC; @@ -106,6 +108,8 @@ private RPC.Server getRpcServer(OzoneConfiguration configuration, final int handlerCount = conf.getInt(HDDS_DATANODE_HANDLER_COUNT_KEY, HDDS_DATANODE_HANDLER_COUNT_DEFAULT); + final int readThreads = conf.getInt(HDDS_DATANODE_READ_THREADPOOL_KEY, + HDDS_DATANODE_READ_THREADPOOL_DEFAULT); ReconfigureProtocolServerSideTranslatorPB reconfigureServerProtocol = new ReconfigureProtocolServerSideTranslatorPB(reconfigurationHandler); BlockingService reconfigureService = ReconfigureProtocolProtos @@ -113,7 +117,7 @@ private RPC.Server getRpcServer(OzoneConfiguration configuration, reconfigureServerProtocol); return preserveThreadName(() -> startRpcServer(configuration, rpcAddress, - ReconfigureProtocolDatanodePB.class, reconfigureService, handlerCount)); + ReconfigureProtocolDatanodePB.class, reconfigureService, handlerCount, readThreads)); } /** @@ -130,7 +134,7 @@ private RPC.Server getRpcServer(OzoneConfiguration configuration, private RPC.Server startRpcServer( Configuration configuration, InetSocketAddress addr, Class protocol, BlockingService instance, - int handlerCount) + int handlerCount, int readThreads) throws IOException { return new RPC.Builder(configuration) .setProtocol(protocol) @@ -138,6 +142,7 @@ private RPC.Server startRpcServer( .setBindAddress(addr.getHostString()) .setPort(addr.getPort()) .setNumHandlers(handlerCount) + .setNumReaders(readThreads) .setVerbose(false) .setSecretManager(null) .build(); 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 b2ed8691a6f..6b32b74dc7c 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 @@ -288,7 +288,7 @@ public void start() { .register(REPLICATION_STREAMS_LIMIT_KEY, this::reconfigReplicationStreamsLimit); - datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf, + datanodeStateMachine = new DatanodeStateMachine(this, datanodeDetails, conf, dnCertClient, secretKeyClient, this::terminateDatanode, reconfigurationHandler); try { @@ -620,6 +620,10 @@ public void saveNewCertId(String newCertId) { } } + public boolean isStopped() { + return isStopped.get(); + } + /** * Check ozone admin privilege, throws exception if not admin. */ diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerChecksumTreeManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerChecksumTreeManager.java index 5d33caa383d..30e938f574a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerChecksumTreeManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerChecksumTreeManager.java @@ -66,10 +66,13 @@ public class ContainerChecksumTreeManager { public ContainerChecksumTreeManager(ConfigurationSource conf) { fileLock = SimpleStriped.custom(conf.getObject(DatanodeConfiguration.class).getContainerChecksumLockStripes(), () -> new ReentrantLock(true)); - // TODO: TO unregister metrics on stop. metrics = ContainerMerkleTreeMetrics.create(); } + public void stop() { + ContainerMerkleTreeMetrics.unregister(); + } + /** * Writes the specified container merkle tree to the specified container's checksum file. * The data merkle tree within the file is replaced with the {@code tree} parameter, but all other content of the diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerMerkleTreeMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerMerkleTreeMetrics.java index 3d76288616e..c1bab5aa485 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerMerkleTreeMetrics.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ContainerMerkleTreeMetrics.java @@ -32,7 +32,6 @@ public class ContainerMerkleTreeMetrics { public static ContainerMerkleTreeMetrics create() { MetricsSystem ms = DefaultMetricsSystem.instance(); - // TODO: Remove when checksum manager is moved from KeyValueHandler. MetricsSource source = ms.getSource(METRICS_SOURCE_NAME); if (source != null) { ms.unregisterSource(METRICS_SOURCE_NAME); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ReconcileContainerTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ReconcileContainerTask.java new file mode 100644 index 00000000000..ac42efd45ad --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/checksum/ReconcileContainerTask.java @@ -0,0 +1,88 @@ +/* + * 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.checksum; + +import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; +import org.apache.hadoop.ozone.container.replication.AbstractReplicationTask; +import org.apache.hadoop.ozone.protocol.commands.ReconcileContainerCommand; +import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; + +/** + * Used to execute a container reconciliation task that has been queued from the ReplicationSupervisor. + */ +public class ReconcileContainerTask extends AbstractReplicationTask { + private final ReconcileContainerCommand command; + private final DNContainerOperationClient dnClient; + private final ContainerController controller; + + private static final Logger LOG = + LoggerFactory.getLogger(ReconcileContainerTask.class); + + public ReconcileContainerTask(ContainerController controller, + DNContainerOperationClient dnClient, ReconcileContainerCommand command) { + super(command.getContainerID(), command.getDeadline(), command.getTerm()); + this.command = command; + this.controller = controller; + this.dnClient = dnClient; + } + + @Override + public void runTask() { + long start = Time.monotonicNow(); + + LOG.info("{}", this); + + try { + controller.reconcileContainer(dnClient, command.getContainerID(), command.getPeerDatanodes()); + setStatus(Status.DONE); + long elapsed = Time.monotonicNow() - start; + LOG.info("{} completed in {} ms", this, elapsed); + } catch (Exception e) { + long elapsed = Time.monotonicNow() - start; + setStatus(Status.FAILED); + LOG.warn("{} failed in {} ms", this, elapsed, e); + } + } + + @Override + protected Object getCommandForDebug() { + return command.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReconcileContainerTask that = (ReconcileContainerTask) o; + return Objects.equals(command, that.command); + } + + @Override + public int hashCode() { + return Objects.hash(getContainerId()); + } +} 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 ce8dfe189c5..21fa76a7bee 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 @@ -96,7 +96,9 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor { private static final String AUDIT_PARAM_FORCE_DELETE = "forceDelete"; private static final String AUDIT_PARAM_START_CONTAINER_ID = "startContainerID"; private static final String AUDIT_PARAM_BLOCK_DATA = "blockData"; - private static final String AUDIT_PARAM_BLOCK_DATA_SIZE = "blockDataSize"; + private static final String AUDIT_PARAM_BLOCK_DATA_OFFSET = "offset"; + private static final String AUDIT_PARAM_BLOCK_DATA_SIZE = "size"; + private static final String AUDIT_PARAM_BLOCK_DATA_STAGE = "stage"; private static final String AUDIT_PARAM_COUNT = "count"; private static final String AUDIT_PARAM_START_LOCAL_ID = "startLocalID"; private static final String AUDIT_PARAM_PREV_CHUNKNAME = "prevChunkName"; @@ -112,7 +114,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor { private String clusterId; private ContainerMetrics metrics; private final TokenVerifier tokenVerifier; - private long slowOpThresholdMs; + private long slowOpThresholdNs; private VolumeUsage.MinFreeSpaceCalculator freeSpaceCalculator; /** @@ -134,7 +136,7 @@ public HddsDispatcher(ConfigurationSource config, ContainerSet contSet, HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT); this.tokenVerifier = tokenVerifier != null ? tokenVerifier : new NoopTokenVerifier(); - this.slowOpThresholdMs = getSlowOpThresholdMs(conf); + this.slowOpThresholdNs = getSlowOpThresholdMs(conf) * 1000000; protocolMetrics = new ProtocolMessageMetrics<>( @@ -279,7 +281,7 @@ private ContainerCommandResponseProto dispatchRequest( "ContainerID " + containerID + " has been lost and cannot be recreated on this DataNode", ContainerProtos.Result.CONTAINER_MISSING); - audit(action, eventType, msg, AuditEventStatus.FAILURE, sce); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, sce); return ContainerUtils.logAndReturnError(LOG, sce, msg); } @@ -306,7 +308,7 @@ private ContainerCommandResponseProto dispatchRequest( StorageContainerException sce = new StorageContainerException( "ContainerID " + containerID + " creation failed", responseProto.getResult()); - audit(action, eventType, msg, AuditEventStatus.FAILURE, sce); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, sce); return ContainerUtils.logAndReturnError(LOG, sce, msg); } Preconditions.checkArgument(isWriteStage && container2BCSIDMap != null @@ -325,13 +327,13 @@ private ContainerCommandResponseProto dispatchRequest( StorageContainerException sce = new StorageContainerException( "ContainerID " + containerID + " does not exist", ContainerProtos.Result.CONTAINER_NOT_FOUND); - audit(action, eventType, msg, AuditEventStatus.FAILURE, sce); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, sce); return ContainerUtils.logAndReturnError(LOG, sce, msg); } containerType = getContainerType(container); } else { if (!msg.hasCreateContainer()) { - audit(action, eventType, msg, AuditEventStatus.FAILURE, + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception("MALFORMED_REQUEST")); return malformedRequest(msg); } @@ -348,10 +350,10 @@ private ContainerCommandResponseProto dispatchRequest( "ContainerType " + containerType, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); // log failure - audit(action, eventType, msg, AuditEventStatus.FAILURE, ex); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, ex); return ContainerUtils.logAndReturnError(LOG, ex, msg); } - perf.appendPreOpLatencyMs(Time.monotonicNow() - startTime); + perf.appendPreOpLatencyNano(Time.monotonicNowNanos() - startTime); responseProto = handler.handle(msg, container, dispatcherContext); long opLatencyNs = Time.monotonicNowNanos() - startTime; if (responseProto != null) { @@ -417,7 +419,7 @@ private ContainerCommandResponseProto dispatchRequest( } if (result == Result.SUCCESS) { updateBCSID(container, dispatcherContext, cmdType); - audit(action, eventType, msg, AuditEventStatus.SUCCESS, null); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.SUCCESS, null); } else { //TODO HDDS-7096: // This is a too general place for on demand scanning. @@ -425,16 +427,16 @@ private ContainerCommandResponseProto dispatchRequest( // and move this general scan to where it is more appropriate. // Add integration tests to test the full functionality. OnDemandContainerDataScanner.scanContainer(container); - audit(action, eventType, msg, AuditEventStatus.FAILURE, + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception(responseProto.getMessage())); } - perf.appendOpLatencyMs(opLatencyNs); - performanceAudit(action, msg, perf, opLatencyNs); + perf.appendOpLatencyNanos(opLatencyNs); + performanceAudit(action, msg, dispatcherContext, perf, opLatencyNs); return responseProto; } else { // log failure - audit(action, eventType, msg, AuditEventStatus.FAILURE, + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception("UNSUPPORTED_REQUEST")); return unsupportedRequest(msg); } @@ -547,7 +549,7 @@ public void validateContainerCommand( StorageContainerException ex = new StorageContainerException( "Invalid ContainerType " + containerType, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); - audit(action, eventType, msg, AuditEventStatus.FAILURE, ex); + audit(action, eventType, msg, null, AuditEventStatus.FAILURE, ex); throw ex; } @@ -567,12 +569,12 @@ public void validateContainerCommand( // if the container is not open/recovering, no updates can happen. Just // throw an exception ContainerNotOpenException cex = new ContainerNotOpenException(log); - audit(action, eventType, msg, AuditEventStatus.FAILURE, cex); + audit(action, eventType, msg, null, AuditEventStatus.FAILURE, cex); throw cex; } } else if (HddsUtils.isReadOnly(msg) && containerState == State.INVALID) { InvalidContainerStateException iex = new InvalidContainerStateException(log); - audit(action, eventType, msg, AuditEventStatus.FAILURE, iex); + audit(action, eventType, msg, null, AuditEventStatus.FAILURE, iex); throw iex; } } @@ -678,14 +680,14 @@ private EventType getEventType(ContainerCommandRequestProto msg) { } private void audit(AuditAction action, EventType eventType, - ContainerCommandRequestProto msg, AuditEventStatus result, - Throwable exception) { + ContainerCommandRequestProto msg, DispatcherContext dispatcherContext, + AuditEventStatus result, Throwable exception) { Map params; AuditMessage amsg; switch (result) { case SUCCESS: if (isAllowed(action.getAction())) { - params = getAuditParams(msg); + params = getAuditParams(msg, dispatcherContext); if (eventType == EventType.READ && AUDIT.getLogger().isInfoEnabled(AuditMarker.READ.getMarker())) { amsg = buildAuditMessageForSuccess(action, params); @@ -699,7 +701,7 @@ private void audit(AuditAction action, EventType eventType, break; case FAILURE: - params = getAuditParams(msg); + params = getAuditParams(msg, dispatcherContext); if (eventType == EventType.READ && AUDIT.getLogger().isErrorEnabled(AuditMarker.READ.getMarker())) { amsg = buildAuditMessageForFailure(action, params, exception); @@ -719,9 +721,9 @@ private void audit(AuditAction action, EventType eventType, } private void performanceAudit(AuditAction action, ContainerCommandRequestProto msg, - PerformanceStringBuilder performance, long opLatencyMs) { - if (isOperationSlow(opLatencyMs)) { - Map params = getAuditParams(msg); + DispatcherContext dispatcherContext, PerformanceStringBuilder performance, long opLatencyNs) { + if (isOperationSlow(opLatencyNs)) { + Map params = getAuditParams(msg, dispatcherContext); AuditMessage auditMessage = buildAuditMessageForPerformance(action, params, performance); AUDIT.logPerformance(auditMessage); @@ -838,7 +840,7 @@ private static DNAction getAuditAction(Type cmdType) { } private static Map getAuditParams( - ContainerCommandRequestProto msg) { + ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) { Map auditParams = new TreeMap<>(); Type cmdType = msg.getCmdType(); String containerID = String.valueOf(msg.getContainerID()); @@ -905,6 +907,8 @@ private static Map getAuditParams( case ReadChunk: auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getReadChunk().getBlockID()).toString()); + auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET, + String.valueOf(msg.getReadChunk().getChunkData().getOffset())); auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE, String.valueOf(msg.getReadChunk().getChunkData().getLen())); return auditParams; @@ -919,8 +923,13 @@ private static Map getAuditParams( auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockID.getFromProtobuf(msg.getWriteChunk().getBlockID()) .toString()); + auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET, + String.valueOf(msg.getWriteChunk().getChunkData().getOffset())); auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE, String.valueOf(msg.getWriteChunk().getChunkData().getLen())); + if (dispatcherContext != null && dispatcherContext.getStage() != null) { + auditParams.put(AUDIT_PARAM_BLOCK_DATA_STAGE, dispatcherContext.getStage().toString()); + } return auditParams; case ListChunk: @@ -937,6 +946,8 @@ private static Map getAuditParams( auditParams.put(AUDIT_PARAM_BLOCK_DATA, BlockData.getFromProtoBuf(msg.getPutSmallFile() .getBlock().getBlockData()).toString()); + auditParams.put(AUDIT_PARAM_BLOCK_DATA_OFFSET, + String.valueOf(msg.getPutSmallFile().getChunkInfo().getOffset())); auditParams.put(AUDIT_PARAM_BLOCK_DATA_SIZE, String.valueOf(msg.getPutSmallFile().getChunkInfo().getLen())); } catch (IOException ex) { @@ -976,7 +987,7 @@ private static Map getAuditParams( } - private boolean isOperationSlow(long opLatencyMs) { - return opLatencyMs >= slowOpThresholdMs; + private boolean isOperationSlow(long opLatencyNs) { + return opLatencyNs >= slowOpThresholdNs; } } 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 01435d8002a..1579f4af8ea 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.List; +import java.util.Set; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -30,6 +30,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerData; @@ -70,16 +72,17 @@ protected Handler(ConfigurationSource config, String datanodeId, this.icrSender = icrSender; } + @SuppressWarnings("checkstyle:ParameterNumber") public static Handler getHandlerForContainerType( final ContainerType containerType, final ConfigurationSource config, final String datanodeId, final ContainerSet contSet, final VolumeSet volumeSet, final ContainerMetrics metrics, - IncrementalReportSender icrSender) { + IncrementalReportSender icrSender, ContainerChecksumTreeManager checksumManager) { switch (containerType) { case KeyValueContainer: return new KeyValueHandler(config, datanodeId, contSet, volumeSet, metrics, - icrSender); + icrSender, checksumManager); default: throw new IllegalArgumentException("Handler for ContainerType: " + containerType + "doesn't exist."); @@ -199,7 +202,8 @@ public abstract void deleteContainer(Container container, boolean force) * @param container container to be reconciled. * @param peers The other datanodes with a copy of this container whose data should be checked. */ - public abstract void reconcileContainer(Container container, List peers) throws IOException; + public abstract void reconcileContainer(DNContainerOperationClient dnClient, Container container, + Set peers) throws IOException; /** * Deletes the given files associated with a block of the container. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java index 16cacd6fae8..16d282da5a4 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java @@ -118,6 +118,11 @@ public class DatanodeConfiguration extends ReconfigurableConfig { public static final int CONTAINER_CLIENT_CACHE_STALE_THRESHOLD_MILLISECONDS_DEFAULT = 10000; + private static final long + AUTO_COMPACTION_SMALL_SST_FILE_INTERVAL_MINUTES_DEFAULT = 120; + private static final int + AUTO_COMPACTION_SMALL_SST_FILE_THREADS_DEFAULT = 1; + /** * Number of threads per volume that Datanode will use for chunk read. */ @@ -544,6 +549,24 @@ public void setWaitOnAllFollowers(boolean val) { ) private int autoCompactionSmallSstFileNum = 512; + @Config(key = "rocksdb.auto-compaction-small-sst-file.interval.minutes", + defaultValue = "120", + type = ConfigType.LONG, + tags = { DATANODE }, + description = "Auto compact small SST files interval in minutes." + ) + private long autoCompactionSmallSstFileIntervalMinutes = + AUTO_COMPACTION_SMALL_SST_FILE_INTERVAL_MINUTES_DEFAULT; + + @Config(key = "rocksdb.auto-compaction-small-sst-file.threads", + defaultValue = "1", + type = ConfigType.INT, + tags = { DATANODE }, + description = "Auto compact small SST files threads." + ) + private int autoCompactionSmallSstFileThreads = + AUTO_COMPACTION_SMALL_SST_FILE_THREADS_DEFAULT; + /** * Whether to check container directory or not to determine * container is empty. @@ -983,4 +1006,24 @@ public int getContainerClientCacheSize() { public int getContainerClientCacheStaleThreshold() { return containerClientCacheStaleThreshold; } + + public long getAutoCompactionSmallSstFileIntervalMinutes() { + return autoCompactionSmallSstFileIntervalMinutes; + } + + public void setAutoCompactionSmallSstFileIntervalMinutes( + long autoCompactionSmallSstFileIntervalMinutes) { + this.autoCompactionSmallSstFileIntervalMinutes = + autoCompactionSmallSstFileIntervalMinutes; + } + + public int getAutoCompactionSmallSstFileThreads() { + return autoCompactionSmallSstFileThreads; + } + + public void setAutoCompactionSmallSstFileThreads( + int autoCompactionSmallSstFileThreads) { + this.autoCompactionSmallSstFileThreads = + autoCompactionSmallSstFileThreads; + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java index a8deb9823d7..55fcbcdb3cc 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java @@ -44,7 +44,9 @@ import org.apache.hadoop.hdds.upgrade.HDDSLayoutVersionManager; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.NettyMetrics; +import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.HddsDatanodeStopService; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage; import org.apache.hadoop.ozone.container.common.report.ReportManager; import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.CloseContainerCommandHandler; @@ -138,7 +140,9 @@ public class DatanodeStateMachine implements Closeable { * @param certClient - Datanode Certificate client, required if security is * enabled */ - public DatanodeStateMachine(DatanodeDetails datanodeDetails, + @SuppressWarnings("checkstyle:ParameterNumber") + public DatanodeStateMachine(HddsDatanodeService hddsDatanodeService, + DatanodeDetails datanodeDetails, ConfigurationSource conf, CertificateClient certClient, SecretKeyClient secretKeyClient, @@ -178,7 +182,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails, // HDDS-3116 for more details. constructionLock.writeLock().lock(); try { - container = new OzoneContainer(this.datanodeDetails, + container = new OzoneContainer(hddsDatanodeService, this.datanodeDetails, conf, context, certClient, secretKeyClient); } finally { constructionLock.writeLock().unlock(); @@ -225,6 +229,10 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails, new ReconstructECContainersCommandHandler(conf, supervisor, ecReconstructionCoordinator); + // TODO HDDS-11218 combine the clients used for reconstruction and reconciliation so they share the same cache of + // datanode clients. + DNContainerOperationClient dnClient = new DNContainerOperationClient(conf, certClient, secretKeyClient); + ThreadFactory threadFactory = new ThreadFactoryBuilder() .setNameFormat(threadNamePrefix + "PipelineCommandHandlerThread-%d") .build(); @@ -253,7 +261,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails, supervisor::nodeStateUpdated)) .addHandler(new FinalizeNewLayoutVersionCommandHandler()) .addHandler(new RefreshVolumeUsageCommandHandler()) - .addHandler(new ReconcileContainerCommandHandler(threadNamePrefix)) + .addHandler(new ReconcileContainerCommandHandler(supervisor, dnClient)) .setConnectionManager(connectionManager) .setContainer(container) .setContext(context) @@ -275,7 +283,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails, @VisibleForTesting public DatanodeStateMachine(DatanodeDetails datanodeDetails, ConfigurationSource conf) throws IOException { - this(datanodeDetails, conf, null, null, null, + this(null, datanodeDetails, conf, null, null, null, new ReconfigurationHandler("DN", (OzoneConfiguration) conf, op -> { })); } 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 0b9005422ef..a6c3b11de92 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 @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.io.Closeable; -import java.io.IOException; import java.net.InetSocketAddress; import java.time.ZonedDateTime; import java.util.concurrent.ExecutorService; @@ -51,7 +50,7 @@ public class EndpointStateMachine private final InetSocketAddress address; private final Lock lock; private final ConfigurationSource conf; - private EndPointStates state; + private EndPointStates state = EndPointStates.FIRST; private VersionResponse version; private ZonedDateTime lastSuccessfulHeartbeat; private boolean isPassive; @@ -72,7 +71,6 @@ public EndpointStateMachine(InetSocketAddress address, this.endPoint = endPoint; this.missedCount = new AtomicLong(0); this.address = address; - state = EndPointStates.getInitState(); lock = new ReentrantLock(); this.conf = conf; executorService = Executors.newSingleThreadExecutor( @@ -153,11 +151,9 @@ public ExecutorService getExecutorService() { /** * Closes the connection. - * - * @throws IOException */ @Override - public void close() throws IOException { + public void close() { if (endPoint != null) { endPoint.close(); } @@ -232,9 +228,8 @@ public void logIfNeeded(Exception ex) { String serverName = "SCM"; if (isPassive) { - // Recon connection failures can be logged 10 times lower than regular - // SCM. - missCounter = this.getMissedCount() % (10 * getLogWarnInterval(conf)); + // Recon connection failures can be logged 10 times lower than regular SCM. + missCounter = this.getMissedCount() % (10L * getLogWarnInterval(conf)); serverName = "Recon"; } @@ -272,50 +267,16 @@ public void setPassive(boolean passive) { *

* This is a sorted list of states that EndPoint will traverse. *

- * GetNextState will move this enum from getInitState to getLastState. + * {@link #getNextState()} will move from {@link #FIRST} to {@link #LAST}. */ public enum EndPointStates { - GETVERSION(1), - REGISTER(2), - HEARTBEAT(3), - SHUTDOWN(4); // if you add value after this please edit getLastState too. - private final int value; - - /** - * Constructs endPointStates. - * - * @param value state. - */ - EndPointStates(int value) { - this.value = value; - } - - /** - * Returns the first State. - * - * @return First State. - */ - public static EndPointStates getInitState() { - return GETVERSION; - } - - /** - * The last state of endpoint states. - * - * @return last state. - */ - public static EndPointStates getLastState() { - return SHUTDOWN; - } + GETVERSION, + REGISTER, + HEARTBEAT, + SHUTDOWN; - /** - * returns the numeric value associated with the endPoint. - * - * @return int. - */ - public int getValue() { - return value; - } + private static final EndPointStates FIRST = values()[0]; + private static final EndPointStates LAST = values()[values().length - 1]; /** * Returns the next logical state that endPoint should move to. @@ -324,15 +285,8 @@ public int getValue() { * @return NextState. */ public EndPointStates getNextState() { - if (this.getValue() < getLastState().getValue()) { - int stateValue = this.getValue() + 1; - for (EndPointStates iter : values()) { - if (stateValue == iter.getValue()) { - return iter; - } - } - } - return getLastState(); + final int n = this.ordinal(); + return n >= LAST.ordinal() ? LAST : values()[n + 1]; } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java index 93a45905975..55cd57d9dc7 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java @@ -27,6 +27,10 @@ import java.util.OptionalLong; import java.util.Queue; import java.util.Set; +import java.util.Objects; +import java.util.LinkedHashMap; +import java.util.Collections; +import java.util.Iterator; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -45,6 +49,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.Descriptors.Descriptor; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.CommandStatus.Status; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.CommandStatusReportsProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerAction; @@ -53,10 +58,12 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineAction; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto; import org.apache.hadoop.ozone.container.common.states.DatanodeState; import org.apache.hadoop.ozone.container.common.states.datanode.InitDatanodeState; import org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.protocol.commands.CommandStatus; import org.apache.hadoop.ozone.protocol.commands.DeleteBlockCommandStatus.DeleteBlockCommandStatusBuilder; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; @@ -112,7 +119,7 @@ public class StateContext { private final Map> incrementalReportsQueue; private final Map> containerActions; - private final Map> pipelineActions; + private final Map pipelineActions; private DatanodeStateMachine.DatanodeStates state; private boolean shutdownOnError = false; private boolean shutdownGracefully = false; @@ -178,7 +185,7 @@ public StateContext(ConfigurationSource conf, pipelineReports = new AtomicReference<>(); endpoints = new HashSet<>(); containerActions = new HashMap<>(); - pipelineActions = new HashMap<>(); + pipelineActions = new ConcurrentHashMap<>(); lock = new ReentrantLock(); stateExecutionCount = new AtomicLong(0); threadPoolNotAvailableCount = new AtomicLong(0); @@ -518,47 +525,16 @@ public List getPendingContainerAction( } } - /** - * Helper function for addPipelineActionIfAbsent that check if inputs are the - * same close pipeline action. - * - * Important Note: Make sure to double check for correctness before using this - * helper function for other purposes! - * - * @return true if a1 and a2 are the same close pipeline action, - * false otherwise - */ - boolean isSameClosePipelineAction(PipelineAction a1, PipelineAction a2) { - return a1.getAction() == a2.getAction() - && a1.hasClosePipeline() - && a2.hasClosePipeline() - && a1.getClosePipeline().getPipelineID() - .equals(a2.getClosePipeline().getPipelineID()); - } - /** * Add PipelineAction to PipelineAction queue if it's not present. * * @param pipelineAction PipelineAction to be added */ public void addPipelineActionIfAbsent(PipelineAction pipelineAction) { - synchronized (pipelineActions) { - /** - * If pipelineAction queue already contains entry for the pipeline id - * with same action, we should just return. - * Note: We should not use pipelineActions.contains(pipelineAction) here - * as, pipelineAction has a msg string. So even if two msgs differ though - * action remains same on the given pipeline, it will end up adding it - * multiple times here. - */ - for (InetSocketAddress endpoint : endpoints) { - final Queue actionsForEndpoint = - pipelineActions.get(endpoint); - if (actionsForEndpoint.stream().noneMatch( - action -> isSameClosePipelineAction(action, pipelineAction))) { - actionsForEndpoint.add(pipelineAction); - } - } + // Put only if the pipeline id with the same action is absent. + final PipelineKey key = new PipelineKey(pipelineAction); + for (InetSocketAddress endpoint : endpoints) { + pipelineActions.get(endpoint).putIfAbsent(key, pipelineAction); } } @@ -571,34 +547,17 @@ public void addPipelineActionIfAbsent(PipelineAction pipelineAction) { public List getPendingPipelineAction( InetSocketAddress endpoint, int maxLimit) { - List pipelineActionList = new ArrayList<>(); - List persistPipelineAction = new ArrayList<>(); - synchronized (pipelineActions) { - if (!pipelineActions.isEmpty() && - CollectionUtils.isNotEmpty(pipelineActions.get(endpoint))) { - Queue actionsForEndpoint = - this.pipelineActions.get(endpoint); - int size = actionsForEndpoint.size(); - int limit = size > maxLimit ? maxLimit : size; - for (int count = 0; count < limit; count++) { - // Add closePipeline back to the pipelineAction queue until - // pipeline is closed and removed from the DN. - PipelineAction action = actionsForEndpoint.poll(); - if (action.hasClosePipeline()) { - if (parentDatanodeStateMachine.getContainer().getPipelineReport() - .getPipelineReportList().stream().noneMatch( - report -> action.getClosePipeline().getPipelineID() - .equals(report.getPipelineID()))) { - continue; - } - persistPipelineAction.add(action); - } - pipelineActionList.add(action); - } - actionsForEndpoint.addAll(persistPipelineAction); - } - return pipelineActionList; + final PipelineActionMap map = pipelineActions.get(endpoint); + if (map == null) { + return Collections.emptyList(); } + final OzoneContainer ozoneContainer = parentDatanodeStateMachine. + getContainer(); + if (ozoneContainer == null) { + return Collections.emptyList(); + } + final PipelineReportsProto reports = ozoneContainer.getPipelineReport(); + return map.getActions(reports.getPipelineReportList(), maxLimit); } /** @@ -927,7 +886,7 @@ public void addEndpoint(InetSocketAddress endpoint) { if (!endpoints.contains(endpoint)) { this.endpoints.add(endpoint); this.containerActions.put(endpoint, new LinkedList<>()); - this.pipelineActions.put(endpoint, new LinkedList<>()); + this.pipelineActions.put(endpoint, new PipelineActionMap()); this.incrementalReportsQueue.put(endpoint, new LinkedList<>()); Map mp = new HashMap<>(); fullReportTypeList.forEach(e -> { @@ -988,4 +947,79 @@ public DatanodeQueueMetrics getQueueMetrics() { public String getThreadNamePrefix() { return threadNamePrefix; } + + static class PipelineActionMap { + private final LinkedHashMap map = + new LinkedHashMap<>(); + + synchronized int size() { + return map.size(); + } + + synchronized void putIfAbsent(PipelineKey key, + PipelineAction pipelineAction) { + map.putIfAbsent(key, pipelineAction); + } + + synchronized List getActions(List reports, + int max) { + if (map.isEmpty()) { + return Collections.emptyList(); + } + final List pipelineActionList = new ArrayList<>(); + final int limit = Math.min(map.size(), max); + final Iterator> i = + map.entrySet().iterator(); + for (int count = 0; count < limit && i.hasNext(); count++) { + final Map.Entry entry = i.next(); + final PipelineAction action = entry.getValue(); + // Add closePipeline back to the pipelineAction queue until + // pipeline is closed and removed from the DN. + if (action.hasClosePipeline()) { + if (reports.stream().noneMatch(entry.getKey()::equalsId)) { + // pipeline is removed from the DN, this action is no longer needed. + i.remove(); + continue; + } + // pipeline is closed but not yet removed from the DN. + } else { + i.remove(); + } + pipelineActionList.add(action); + } + // add all + return pipelineActionList; + } + } + + static class PipelineKey { + private final HddsProtos.PipelineID pipelineID; + private final PipelineAction.Action action; + + PipelineKey(PipelineAction p) { + this.pipelineID = p.getClosePipeline().getPipelineID(); + this.action = p.getAction(); + } + + boolean equalsId(PipelineReport report) { + return pipelineID.equals(report.getPipelineID()); + } + + @Override + public int hashCode() { + return Objects.hashCode(pipelineID); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (!(obj instanceof PipelineKey)) { + return false; + } + final PipelineKey that = (PipelineKey) obj; + return Objects.equals(this.action, that.action) + && Objects.equals(this.pipelineID, that.pipelineID); + } + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconcileContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconcileContainerCommandHandler.java index 9a4110c7dfc..99185a7e10b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconcileContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ReconcileContainerCommandHandler.java @@ -18,66 +18,38 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; +import org.apache.hadoop.ozone.container.checksum.ReconcileContainerTask; 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.ReplicationSupervisor; import org.apache.hadoop.ozone.protocol.commands.ReconcileContainerCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.apache.hadoop.util.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; /** * Handles commands from SCM to reconcile a container replica on this datanode with the replicas on its peers. */ public class ReconcileContainerCommandHandler implements CommandHandler { - private static final Logger LOG = - LoggerFactory.getLogger(ReconcileContainerCommandHandler.class); - + private final ReplicationSupervisor supervisor; private final AtomicLong invocationCount; - private final AtomicInteger queuedCount; - private final ExecutorService executor; - private long totalTime; + private final DNContainerOperationClient dnClient; - public ReconcileContainerCommandHandler(String threadNamePrefix) { - invocationCount = new AtomicLong(0); - queuedCount = new AtomicInteger(0); - // TODO Allow configurable thread pool size with a default value when the implementation is ready. - executor = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder() - .setNameFormat(threadNamePrefix + "ReconcileContainerThread-%d") - .build()); - totalTime = 0; + public ReconcileContainerCommandHandler(ReplicationSupervisor supervisor, DNContainerOperationClient dnClient) { + this.supervisor = supervisor; + this.dnClient = dnClient; + this.invocationCount = new AtomicLong(0); } @Override public void handle(SCMCommand command, OzoneContainer container, StateContext context, SCMConnectionManager connectionManager) { - queuedCount.incrementAndGet(); - CompletableFuture.runAsync(() -> { - invocationCount.incrementAndGet(); - long startTime = Time.monotonicNow(); - ReconcileContainerCommand reconcileCommand = (ReconcileContainerCommand) command; - LOG.info("Processing reconcile container command for container {} with peers {}", - reconcileCommand.getContainerID(), reconcileCommand.getPeerDatanodes()); - try { - container.getController().reconcileContainer(reconcileCommand.getContainerID(), - reconcileCommand.getPeerDatanodes()); - } catch (IOException ex) { - LOG.error("Failed to reconcile container {}.", reconcileCommand.getContainerID(), ex); - } finally { - long endTime = Time.monotonicNow(); - totalTime += endTime - startTime; - } - }, executor).whenComplete((v, e) -> queuedCount.decrementAndGet()); + invocationCount.incrementAndGet(); + ReconcileContainerCommand reconcileCommand = (ReconcileContainerCommand) command; + supervisor.addTask(new ReconcileContainerTask(container.getController(), dnClient, reconcileCommand)); } @Override @@ -90,21 +62,20 @@ public int getInvocationCount() { return (int)invocationCount.get(); } + // Uses ReplicationSupervisor for these metrics. + @Override public long getAverageRunTime() { - if (invocationCount.get() > 0) { - return totalTime / invocationCount.get(); - } return 0; } @Override public long getTotalRunTime() { - return totalTime; + return 0; } @Override public int getQueuedCount() { - return queuedCount.get(); + return 0; } } 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 90fca79550b..b3398de07ad 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -60,6 +61,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.utils.Cache; import org.apache.hadoop.hdds.utils.ResourceCache; +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.interfaces.ContainerDispatcher; @@ -70,6 +72,7 @@ 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; @@ -94,6 +97,7 @@ 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.LifeCycle; import org.apache.ratis.util.TaskQueue; import org.apache.ratis.util.function.CheckedSupplier; import org.apache.ratis.util.JavaUtils; @@ -198,19 +202,23 @@ long getStartTime() { private final Semaphore applyTransactionSemaphore; private final boolean waitOnBothFollowers; + private final HddsDatanodeService datanodeService; + private static Semaphore semaphore = new Semaphore(1); + /** * CSM metrics. */ private final CSMMetrics metrics; @SuppressWarnings("parameternumber") - public ContainerStateMachine(RaftGroupId gid, + public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupId gid, ContainerDispatcher dispatcher, ContainerController containerController, List chunkExecutors, XceiverServerRatis ratisServer, ConfigurationSource conf, String threadNamePrefix) { + this.datanodeService = hddsDatanodeService; this.gid = gid; this.dispatcher = dispatcher; this.containerController = containerController; @@ -573,7 +581,10 @@ private CompletableFuture writeStateMachineData( writeChunkFuture.thenApply(r -> { if (r.getResult() != ContainerProtos.Result.SUCCESS && r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN - && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO) { + && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO + // After concurrent flushes are allowed on the same key, chunk file inconsistencies can happen and + // that should not crash the pipeline. + && r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) { StorageContainerException sce = new StorageContainerException(r.getMessage(), r.getResult()); LOG.error(gid + ": writeChunk writeStateMachineData failed: blockId" + @@ -904,6 +915,49 @@ public void notifyTermIndexUpdated(long term, long index) { removeStateMachineDataIfNeeded(index); } + @Override + public void notifyServerShutdown(RaftProtos.RoleInfoProto roleInfo, boolean allServer) { + // if datanodeService is stopped , it indicates this `close` originates + // from `HddsDatanodeService.stop()`, otherwise, it indicates this `close` originates from ratis. + if (allServer) { + if (datanodeService != null && !datanodeService.isStopped()) { + LOG.info("{} is closed by ratis", gid); + if (semaphore.tryAcquire()) { + // run with a different thread, so this raft group can be closed + Runnable runnable = () -> { + try { + int closed = 0, total = 0; + try { + Thread.sleep(5000); // sleep 5s + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + Iterator iterator = ratisServer.getServer().getGroupIds().iterator(); + while (iterator.hasNext()) { + RaftGroupId id = iterator.next(); + RaftServer.Division division = ratisServer.getServer().getDivision(id); + if (division.getRaftServer().getLifeCycleState() == LifeCycle.State.CLOSED) { + closed++; + } + total++; + } + LOG.error("Container statemachine is closed by ratis, terminating HddsDatanodeService. " + + "closed({})/total({})", closed, total); + datanodeService.terminateDatanode(); + } catch (IOException e) { + LOG.warn("Failed to get division for raft groups", e); + LOG.error("Container statemachine is closed by ratis, terminating HddsDatanodeService"); + datanodeService.terminateDatanode(); + } + }; + CompletableFuture.runAsync(runnable); + } + } else { + LOG.info("{} is closed by HddsDatanodeService", gid); + } + } + } + private CompletableFuture applyTransaction( ContainerCommandRequestProto request, DispatcherContext context, Consumer exceptionHandler) { @@ -1010,7 +1064,8 @@ public CompletableFuture applyTransaction(TransactionContext trx) { // unhealthy if (r.getResult() != ContainerProtos.Result.SUCCESS && r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN - && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO) { + && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO + && r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) { StorageContainerException sce = new StorageContainerException(r.getMessage(), r.getResult()); LOG.error( diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index dc0c4b06760..7899cdcc0e6 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig; +import org.apache.hadoop.hdds.conf.RatisConfUtils; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -59,6 +60,7 @@ import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient; import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.hdds.utils.HddsServerUtil; +import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.impl.ContainerData; @@ -177,13 +179,15 @@ private static long nextCallId() { private final boolean shouldDeleteRatisLogDirectory; private final boolean streamEnable; private final DatanodeRatisServerConfig ratisServerConfig; + private final HddsDatanodeService datanodeService; - private XceiverServerRatis(DatanodeDetails dd, + private XceiverServerRatis(HddsDatanodeService hddsDatanodeService, DatanodeDetails dd, ContainerDispatcher dispatcher, ContainerController containerController, StateContext context, ConfigurationSource conf, Parameters parameters) throws IOException { this.conf = conf; Objects.requireNonNull(dd, "DatanodeDetails == null"); + datanodeService = hddsDatanodeService; datanodeDetails = dd; ratisServerConfig = conf.getObject(DatanodeRatisServerConfig.class); assignPorts(); @@ -241,7 +245,7 @@ private int determinePort(String key, int defaultValue) { } private ContainerStateMachine getStateMachine(RaftGroupId gid) { - return new ContainerStateMachine(gid, dispatcher, containerController, + return new ContainerStateMachine(datanodeService, gid, dispatcher, containerController, chunkExecutors, this, conf, datanodeDetails.threadNamePrefix()); } @@ -276,11 +280,14 @@ public RaftProperties newRaftProperties() { final RpcType rpc = setRpcType(properties); // set raft segment size - setRaftSegmentAndWriteBufferSize(properties); + final int logAppenderBufferByteLimit = setRaftSegmentAndWriteBufferSize(properties); + + // set grpc message size max + final int max = Math.max(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE, logAppenderBufferByteLimit); + RatisConfUtils.Grpc.setMessageSizeMax(properties, max); // set raft segment pre-allocated size - final long raftSegmentPreallocatedSize = - setRaftSegmentPreallocatedSize(properties); + setRaftSegmentPreallocatedSize(properties); // setup ratis stream if datastream is enabled if (streamEnable) { @@ -311,11 +318,6 @@ public RaftProperties newRaftProperties() { RaftServerConfigKeys.setStorageDir(properties, storageDirs); - // For grpc set the maximum message size - GrpcConfigKeys.setMessageSizeMax(properties, - SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE - + raftSegmentPreallocatedSize)); - // Set the ratis port number if (rpc == SupportedRpcType.GRPC) { GrpcConfigKeys.Admin.setPort(properties, adminPort); @@ -404,17 +406,16 @@ private void setTimeoutForRetryCache(RaftProperties properties) { .setExpiryTime(properties, retryCacheTimeout); } - private long setRaftSegmentPreallocatedSize(RaftProperties properties) { + private void setRaftSegmentPreallocatedSize(RaftProperties properties) { final long raftSegmentPreallocatedSize = (long) conf.getStorageSize( OzoneConfigKeys.HDDS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, OzoneConfigKeys.HDDS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT, StorageUnit.BYTES); RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf(raftSegmentPreallocatedSize)); - return raftSegmentPreallocatedSize; } - private void setRaftSegmentAndWriteBufferSize(RaftProperties properties) { + private int setRaftSegmentAndWriteBufferSize(RaftProperties properties) { final int logAppenderQueueNumElements = conf.getInt( HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS, HDDS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT); @@ -443,6 +444,7 @@ private void setRaftSegmentAndWriteBufferSize(RaftProperties properties) { SizeInBytes.valueOf(raftSegmentSize)); RaftServerConfigKeys.Log.setWriteBufferSize(properties, SizeInBytes.valueOf(raftSegmentBufferSize)); + return logAppenderQueueByteLimit; } private void setStateMachineDataConfigurations(RaftProperties properties) { @@ -521,14 +523,14 @@ private void setPendingRequestsLimits(RaftProperties properties) { .valueOf(pendingRequestsMegaBytesLimit, TraditionalBinaryPrefix.MEGA)); } - public static XceiverServerRatis newXceiverServerRatis( + public static XceiverServerRatis newXceiverServerRatis(HddsDatanodeService hddsDatanodeService, DatanodeDetails datanodeDetails, ConfigurationSource ozoneConf, ContainerDispatcher dispatcher, ContainerController containerController, CertificateClient caClient, StateContext context) throws IOException { Parameters parameters = createTlsParameters( new SecurityConfig(ozoneConf), caClient); - return new XceiverServerRatis(datanodeDetails, dispatcher, + return new XceiverServerRatis(hddsDatanodeService, datanodeDetails, dispatcher, containerController, context, ozoneConf, parameters); } @@ -591,6 +593,7 @@ private int getRealPort(InetSocketAddress address, Port.Name name) { public void stop() { if (isStarted) { try { + LOG.info("Stopping {} {}", getClass().getSimpleName(), server.getId()); // shutdown server before the executors as while shutting down, // some of the tasks would be executed using the executors. server.close(); 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 d4cdaf2cfe4..b22b9148bb1 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 @@ -38,6 +38,7 @@ import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil; import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3; +import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -276,15 +277,6 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } - // TODO HDDS-8784 trigger compaction outside of volume check. Then the - // exception can be removed. - if (df.autoCompactionSmallSstFile()) { - // Calculate number of files per level and size per level - RawDB rawDB = DatanodeStoreCache.getInstance().getDB( - dbFile.getAbsolutePath(), getConf()); - rawDB.getStore().compactionIfNeeded(); - } - return VolumeCheckResult.HEALTHY; } @@ -470,4 +462,20 @@ private void closeDbStore() { LOG.info("SchemaV3 db is stopped at {} for volume {}", containerDBPath, getStorageID()); } + + public void compactDb() { + File dbFile = new File(getDbParentDir(), CONTAINER_DB_NAME); + String dbFilePath = dbFile.getAbsolutePath(); + try { + // Calculate number of files per level and size per level + RawDB rawDB = + DatanodeStoreCache.getInstance().getDB(dbFilePath, getConf()); + long start = Time.monotonicNowNanos(); + rawDB.getStore().compactionIfNeeded(); + volumeInfoMetrics.dbCompactTimesNanoSecondsIncr( + Time.monotonicNowNanos() - start); + } catch (Exception e) { + LOG.warn("compact rocksdb error in {}", dbFilePath, e); + } + } } 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 e59cab0d539..68140600db9 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 @@ -22,8 +22,10 @@ import org.apache.hadoop.metrics2.annotation.Metric; import org.apache.hadoop.metrics2.annotation.Metrics; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MutableRate; import org.apache.hadoop.ozone.OzoneConsts; + /** * This class is used to track Volume Info stats for each HDDS Volume. */ @@ -33,6 +35,8 @@ public class VolumeInfoMetrics { private String metricsSourceName = VolumeInfoMetrics.class.getSimpleName(); private final HddsVolume volume; + @Metric("Returns the RocksDB compact times of the Volume") + private MutableRate dbCompactLatency; /** * @param identifier Typically, path to volume root. E.g. /data/hdds @@ -145,4 +149,8 @@ public long getCommitted() { return volume.getCommittedBytes(); } + public void dbCompactTimesNanoSecondsIncr(long time) { + dbCompactLatency.add(time); + } + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java index f462510bf74..194c8c1309b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java @@ -47,6 +47,7 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import org.apache.hadoop.util.DirectBufferPool; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,6 +71,7 @@ public class KeyValueContainerCheck { private String metadataPath; private HddsVolume volume; private KeyValueContainer container; + private static final DirectBufferPool BUFFER_POOL = new DirectBufferPool(); public KeyValueContainerCheck(String metadataPath, ConfigurationSource conf, long containerID, HddsVolume volume, KeyValueContainer container) { @@ -364,8 +366,12 @@ chunkFile, new IOException("Missing chunk file " + } } else if (chunk.getChecksumData().getType() != ContainerProtos.ChecksumType.NONE) { - ScanResult result = verifyChecksum(block, chunk, chunkFile, layout, + int bytesPerChecksum = chunk.getChecksumData().getBytesPerChecksum(); + ByteBuffer buffer = BUFFER_POOL.getBuffer(bytesPerChecksum); + ScanResult result = verifyChecksum(block, chunk, chunkFile, layout, buffer, throttler, canceler); + buffer.clear(); + BUFFER_POOL.returnBuffer(buffer); if (!result.isHealthy()) { return result; } @@ -377,7 +383,7 @@ chunkFile, new IOException("Missing chunk file " + private static ScanResult verifyChecksum(BlockData block, ContainerProtos.ChunkInfo chunk, File chunkFile, - ContainerLayoutVersion layout, + ContainerLayoutVersion layout, ByteBuffer buffer, DataTransferThrottler throttler, Canceler canceler) { ChecksumData checksumData = ChecksumData.getFromProtoBuf(chunk.getChecksumData()); @@ -385,7 +391,6 @@ private static ScanResult verifyChecksum(BlockData block, int bytesPerChecksum = checksumData.getBytesPerChecksum(); Checksum cal = new Checksum(checksumData.getChecksumType(), bytesPerChecksum); - ByteBuffer buffer = ByteBuffer.allocate(bytesPerChecksum); long bytesRead = 0; try (FileChannel channel = FileChannel.open(chunkFile.toPath(), ChunkUtils.READ_OPTIONS, ChunkUtils.NO_ATTRIBUTES)) { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java index 03050308a94..5118f4ba565 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerMetadataInspector.java @@ -35,6 +35,7 @@ import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl; import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; +import org.apache.hadoop.ozone.container.metadata.DatanodeStoreWithIncrementalChunkList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -501,8 +502,6 @@ static PendingDelete countPendingDeletesSchemaV2( Table delTxTable = schemaTwoStore.getDeleteTransactionTable(); - final Table blockDataTable - = schemaTwoStore.getBlockDataTable(); try (TableIterator> iterator = delTxTable.iterator()) { @@ -515,7 +514,7 @@ static PendingDelete countPendingDeletesSchemaV2( // counted towards bytes used and total block count above. pendingDeleteBlockCountTotal += localIDs.size(); pendingDeleteBytes += computePendingDeleteBytes( - localIDs, containerData, blockDataTable); + localIDs, containerData, schemaTwoStore); } } @@ -525,12 +524,12 @@ static PendingDelete countPendingDeletesSchemaV2( static long computePendingDeleteBytes(List localIDs, KeyValueContainerData containerData, - Table blockDataTable) { + DatanodeStoreWithIncrementalChunkList store) { long pendingDeleteBytes = 0; for (long id : localIDs) { try { final String blockKey = containerData.getBlockKey(id); - final BlockData blockData = blockDataTable.get(blockKey); + final BlockData blockData = store.getBlockByID(null, blockKey); if (blockData != null) { pendingDeleteBytes += blockData.getSize(); } @@ -544,23 +543,21 @@ static long computePendingDeleteBytes(List localIDs, } static PendingDelete countPendingDeletesSchemaV3( - DatanodeStoreSchemaThreeImpl schemaThreeStore, + DatanodeStoreSchemaThreeImpl store, KeyValueContainerData containerData) throws IOException { long pendingDeleteBlockCountTotal = 0; long pendingDeleteBytes = 0; - final Table blockDataTable - = schemaThreeStore.getBlockDataTable(); try ( TableIterator> - iter = schemaThreeStore.getDeleteTransactionTable() + iter = store.getDeleteTransactionTable() .iterator(containerData.containerPrefix())) { while (iter.hasNext()) { DeletedBlocksTransaction delTx = iter.next().getValue(); final List localIDs = delTx.getLocalIDList(); pendingDeleteBlockCountTotal += localIDs.size(); pendingDeleteBytes += computePendingDeleteBytes( - localIDs, containerData, blockDataTable); + localIDs, containerData, store); } } return new PendingDelete(pendingDeleteBlockCountTotal, 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 5f5cddf26a5..b35ba174e46 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 @@ -32,6 +32,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.locks.Lock; import java.util.function.Function; @@ -65,6 +66,7 @@ import org.apache.hadoop.ozone.common.OzoneChecksumException; import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; @@ -164,14 +166,15 @@ public KeyValueHandler(ConfigurationSource config, ContainerSet contSet, VolumeSet volSet, ContainerMetrics metrics, - IncrementalReportSender icrSender) { + IncrementalReportSender icrSender, + ContainerChecksumTreeManager checksumManager) { super(config, datanodeId, contSet, volSet, metrics, icrSender); blockManager = new BlockManagerImpl(config); validateChunkChecksumData = conf.getObject( DatanodeConfiguration.class).isChunkDataValidationCheck(); chunkManager = ChunkManagerFactory.createChunkManager(config, blockManager, volSet); - checksumManager = new ContainerChecksumTreeManager(config); + this.checksumManager = checksumManager; try { volumeChoosingPolicy = VolumeChoosingPolicyFactory.getPolicy(conf); } catch (Exception e) { @@ -1322,7 +1325,8 @@ public void deleteContainer(Container container, boolean force) } @Override - public void reconcileContainer(Container container, List peers) throws IOException { + public void reconcileContainer(DNContainerOperationClient dnClient, Container container, + Set peers) throws IOException { // TODO Just a deterministic placeholder hash for testing until actual implementation is finished. ContainerData data = container.getContainerData(); long id = data.getContainerID(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java index c3f6ac92120..0fac45571c7 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java @@ -39,10 +39,10 @@ import java.util.EnumSet; import java.util.List; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Supplier; +import java.util.concurrent.locks.ReadWriteLock; import java.util.function.ToLongFunction; +import com.google.common.util.concurrent.Striped; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.OzoneConsts; @@ -65,6 +65,7 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; import static org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil.onFailure; +import org.apache.ratis.util.AutoCloseableLock; import org.apache.ratis.util.function.CheckedFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,8 +75,6 @@ */ public final class ChunkUtils { - private static final Set LOCKS = ConcurrentHashMap.newKeySet(); - private static final Logger LOG = LoggerFactory.getLogger(ChunkUtils.class); @@ -91,10 +90,29 @@ public final class ChunkUtils { StandardOpenOption.READ )); public static final FileAttribute[] NO_ATTRIBUTES = {}; + public static final int DEFAULT_FILE_LOCK_STRIPED_SIZE = 2048; + private static Striped fileStripedLock = + Striped.readWriteLock(DEFAULT_FILE_LOCK_STRIPED_SIZE); /** Never constructed. **/ private ChunkUtils() { + } + + @VisibleForTesting + public static void setStripedLock(Striped stripedLock) { + fileStripedLock = stripedLock; + } + + private static ReadWriteLock getFileLock(Path filePath) { + return fileStripedLock.get(filePath); + } + private static AutoCloseableLock getFileReadLock(Path filePath) { + return AutoCloseableLock.acquire(getFileLock(filePath).readLock()); + } + + private static AutoCloseableLock getFileWriteLock(Path filePath) { + return AutoCloseableLock.acquire(getFileLock(filePath).writeLock()); } /** @@ -156,24 +174,19 @@ private static void writeData(ChunkBuffer data, String filename, private static long writeDataToFile(File file, ChunkBuffer data, long offset, boolean sync) { final Path path = file.toPath(); - try { - return processFileExclusively(path, () -> { - FileChannel channel = null; - try { - channel = open(path, WRITE_OPTIONS, NO_ATTRIBUTES); - - try (FileLock ignored = channel.lock()) { - return writeDataToChannel(channel, data, offset); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } finally { - closeFile(channel, sync); + try (AutoCloseableLock ignoredLock = getFileWriteLock(path)) { + FileChannel channel = null; + try { + channel = open(path, WRITE_OPTIONS, NO_ATTRIBUTES); + + try (FileLock ignored = channel.lock()) { + return writeDataToChannel(channel, data, offset); } - }); - } catch (InterruptedException e) { - throw new UncheckedIOException(new InterruptedIOException( - "Interrupted while waiting to write file " + path)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + closeFile(channel, sync); + } } } @@ -211,20 +224,11 @@ private static void readData(File file, long offset, long len, final long startTime = Time.monotonicNow(); final long bytesRead; - try { - bytesRead = processFileExclusively(path, () -> { - try (FileChannel channel = open(path, READ_OPTIONS, NO_ATTRIBUTES); - FileLock ignored = channel.lock(offset, len, true)) { - return readMethod.apply(channel); - } catch (IOException e) { - onFailure(volume); - throw new UncheckedIOException(e); - } - }); - } catch (UncheckedIOException e) { + try (AutoCloseableLock ignoredLock = getFileReadLock(path); + FileChannel channel = open(path, READ_OPTIONS, NO_ATTRIBUTES)) { + bytesRead = readMethod.apply(channel); + } catch (IOException e) { onFailure(volume); - throw wrapInStorageContainerException(e.getCause()); - } catch (InterruptedException e) { throw wrapInStorageContainerException(e); } @@ -391,29 +395,6 @@ public static void verifyChunkFileExists(File file) } } - @VisibleForTesting - static T processFileExclusively(Path path, Supplier op) - throws InterruptedException { - long period = 1; - for (;;) { - if (LOCKS.add(path)) { - break; - } else { - Thread.sleep(period); - // exponentially backoff until the sleep time is over 1 second. - if (period < 1000) { - period *= 2; - } - } - } - - try { - return op.get(); - } finally { - LOCKS.remove(path); - } - } - private static void closeFile(FileChannel file, boolean sync) { if (file != null) { try { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java index dd18636ec00..7b3852011d3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java @@ -343,7 +343,7 @@ public List listBlock(Container container, long startLocalID, int .getSequentialRangeKVs(startKey, count, cData.containerPrefix(), cData.getUnprefixedKeyFilter()); for (Table.KeyValue entry : range) { - result.add(entry.getValue()); + result.add(db.getStore().getCompleteBlockData(entry.getValue(), null, entry.getKey())); } return result; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingTask.java index 38c2bfad2de..098015e3307 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingTask.java @@ -22,6 +22,8 @@ import java.time.Duration; import java.time.Instant; +import java.util.HashSet; +import java.util.Set; import java.util.LinkedList; import java.util.Objects; import java.util.ArrayList; @@ -335,6 +337,8 @@ private ContainerBackgroundTaskResult deleteViaTransactionStore( try { Table blockDataTable = meta.getStore().getBlockDataTable(); + Table lastChunkInfoTable = + meta.getStore().getLastChunkInfoTable(); DeleteTransactionStore txnStore = (DeleteTransactionStore) meta.getStore(); Table deleteTxns = @@ -387,8 +391,11 @@ private ContainerBackgroundTaskResult deleteViaTransactionStore( for (DeletedBlocksTransaction delTx : deletedBlocksTxs) { deleter.apply(deleteTxns, batch, delTx.getTxID()); for (Long blk : delTx.getLocalIDList()) { + // delete from both blockDataTable and lastChunkInfoTable. blockDataTable.deleteWithBatch(batch, containerData.getBlockKey(blk)); + lastChunkInfoTable.deleteWithBatch(batch, + containerData.getBlockKey(blk)); } } @@ -437,14 +444,27 @@ private DeleteTransactionStats deleteTransactions( List delBlocks, Handler handler, Table blockDataTable, Container container) throws IOException { + int blocksProcessed = 0; int blocksDeleted = 0; long bytesReleased = 0; List deletedBlocksTxs = new ArrayList<>(); Instant startTime = Instant.now(); + // Track deleted blocks to avoid duplicate deletion + Set deletedBlockSet = new HashSet<>(); + for (DeletedBlocksTransaction entry : delBlocks) { for (Long blkLong : entry.getLocalIDList()) { + // Increment blocksProcessed for every block processed + blocksProcessed++; + + // Check if the block has already been deleted + if (deletedBlockSet.contains(blkLong)) { + LOG.debug("Skipping duplicate deletion for block {}", blkLong); + continue; + } + String blk = containerData.getBlockKey(blkLong); BlockData blkInfo = blockDataTable.get(blk); LOG.debug("Deleting block {}", blkLong); @@ -455,8 +475,6 @@ private DeleteTransactionStats deleteTransactions( LOG.error("Failed to delete files for unreferenced block {} of" + " container {}", blkLong, container.getContainerData().getContainerID(), e); - } finally { - blocksProcessed++; } continue; } @@ -466,14 +484,14 @@ private DeleteTransactionStats deleteTransactions( handler.deleteBlock(container, blkInfo); blocksDeleted++; deleted = true; + // Track this block as deleted + deletedBlockSet.add(blkLong); } catch (IOException e) { // TODO: if deletion of certain block retries exceed the certain // number of times, service should skip deleting it, // otherwise invalid numPendingDeletionBlocks could accumulate // beyond the limit and the following deletion will stop. LOG.error("Failed to delete files for block {}", blkLong, e); - } finally { - blocksProcessed++; } if (deleted) { 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 35f8bf83222..d791d9bbeab 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 @@ -135,6 +135,11 @@ default BlockData getBlockByID(BlockID blockID, // check block data table BlockData blockData = getBlockDataTable().get(blockKey); + return getCompleteBlockData(blockData, blockID, blockKey); + } + + default BlockData getCompleteBlockData(BlockData blockData, + BlockID blockID, String blockKey) throws IOException { if (blockData == null) { throw new StorageContainerException( NO_SUCH_BLOCK_ERR_MSG + " BlockID : " + blockID, NO_SUCH_BLOCK); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java index 1f5c442601b..2b34fae7398 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java @@ -20,6 +20,7 @@ import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; +import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.FixedLengthStringCodec; @@ -31,6 +32,7 @@ import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; +import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import org.rocksdb.LiveFileMetaData; import java.io.File; @@ -106,6 +108,9 @@ public void removeKVContainerData(long containerID) throws IOException { try (BatchOperation batch = getBatchHandler().initBatchOperation()) { getMetadataTable().deleteBatchWithPrefix(batch, prefix); getBlockDataTable().deleteBatchWithPrefix(batch, prefix); + if (VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.HBASE_SUPPORT)) { + getLastChunkInfoTable().deleteBatchWithPrefix(batch, prefix); + } getDeleteTransactionTable().deleteBatchWithPrefix(batch, prefix); getBatchHandler().commitBatchOperation(batch); } @@ -118,6 +123,10 @@ public void dumpKVContainerData(long containerID, File dumpDir) getTableDumpFile(getMetadataTable(), dumpDir), prefix); getBlockDataTable().dumpToFileWithPrefix( getTableDumpFile(getBlockDataTable(), dumpDir), prefix); + if (VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.HBASE_SUPPORT)) { + getLastChunkInfoTable().dumpToFileWithPrefix( + getTableDumpFile(getLastChunkInfoTable(), dumpDir), prefix); + } getDeleteTransactionTable().dumpToFileWithPrefix( getTableDumpFile(getDeleteTransactionTable(), dumpDir), prefix); @@ -129,6 +138,10 @@ public void loadKVContainerData(File dumpDir) getTableDumpFile(getMetadataTable(), dumpDir)); getBlockDataTable().loadFromFile( getTableDumpFile(getBlockDataTable(), dumpDir)); + if (VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.HBASE_SUPPORT)) { + getLastChunkInfoTable().loadFromFile( + getTableDumpFile(getLastChunkInfoTable(), dumpDir)); + } getDeleteTransactionTable().loadFromFile( getTableDumpFile(getDeleteTransactionTable(), dumpDir)); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java index d3fa7584f36..a71bb93963a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java @@ -56,11 +56,9 @@ public DatanodeStoreWithIncrementalChunkList(ConfigurationSource config, @Override - public BlockData getBlockByID(BlockID blockID, - String blockKey) throws IOException { + public BlockData getCompleteBlockData(BlockData blockData, + BlockID blockID, String blockKey) throws IOException { BlockData lastChunk = null; - // check block data table - BlockData blockData = getBlockDataTable().get(blockKey); if (blockData == null || isPartialChunkList(blockData)) { // check last chunk table lastChunk = getLastChunkInfoTable().get(blockKey); @@ -190,18 +188,29 @@ private void moveLastChunkToBlockData(BatchOperation batch, long localID, private void putBlockWithPartialChunks(BatchOperation batch, long localID, BlockData data, KeyValueContainerData containerData) throws IOException { + String blockKey = containerData.getBlockKey(localID); + BlockData blockData = getBlockDataTable().get(blockKey); if (data.getChunks().size() == 1) { // Case (3.1) replace/update the last chunk info table getLastChunkInfoTable().putWithBatch( - batch, containerData.getBlockKey(localID), data); + batch, blockKey, data); + // If the block does not exist in the block data table because it is the first chunk + // and the chunk is not full, then add an empty block data to populate the block table. + // This is required because some of the test code and utilities expect the block to be + // present in the block data table, they don't check the last chunk info table. + if (blockData == null) { + // populate blockDataTable with empty chunk list + blockData = new BlockData(data.getBlockID()); + blockData.addMetadata(INCREMENTAL_CHUNK_LIST, ""); + blockData.setBlockCommitSequenceId(data.getBlockCommitSequenceId()); + getBlockDataTable().putWithBatch(batch, blockKey, blockData); + } } else { int lastChunkIndex = data.getChunks().size() - 1; // received more than one chunk this time List lastChunkInfo = Collections.singletonList( data.getChunks().get(lastChunkIndex)); - BlockData blockData = getBlockDataTable().get( - containerData.getBlockKey(localID)); if (blockData == null) { // Case 3.2: if the block does not exist in the block data table List chunkInfos = 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 47b503ee054..6a1ceef0c03 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 @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.ContainerReportsProto; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.Container; @@ -39,7 +40,6 @@ import java.io.OutputStream; import java.time.Instant; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; @@ -215,12 +215,13 @@ public void deleteContainer(final long containerId, boolean force) } } - public void reconcileContainer(long containerID, List peers) throws IOException { + public void reconcileContainer(DNContainerOperationClient dnClient, long containerID, Set peers) + throws IOException { Container container = containerSet.getContainer(containerID); if (container == null) { LOG.warn("Container {} to reconcile not found on this datanode.", containerID); } else { - getHandler(container).reconcileContainer(container, peers); + getHandler(container).reconcileContainer(dnClient, container, peers); } } 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 6a6ac8bb35f..4523d15991b 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 @@ -34,8 +34,8 @@ 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.ozone.container.checksum.ContainerMerkleTreeMetrics; import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; +import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.BlockDeletingService; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; @@ -75,6 +75,9 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -124,7 +127,7 @@ public class OzoneContainer { private DatanodeDetails datanodeDetails; private StateContext context; private final ContainerChecksumTreeManager checksumTreeManager; - + private ScheduledExecutorService dbCompactionExecutorService; private final ContainerMetrics metrics; @@ -141,7 +144,7 @@ enum InitializingStatus { * @throws DiskOutOfSpaceException * @throws IOException */ - public OzoneContainer( + public OzoneContainer(HddsDatanodeService hddsDatanodeService, DatanodeDetails datanodeDetails, ConfigurationSource conf, StateContext context, CertificateClient certClient, SecretKeyVerifierClient secretKeyClient) throws IOException { @@ -160,9 +163,22 @@ public OzoneContainer( dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null : new MutableVolumeSet(datanodeDetails.getUuidString(), conf, context, VolumeType.DB_VOLUME, volumeChecker); + final DatanodeConfiguration dnConf = + conf.getObject(DatanodeConfiguration.class); if (SchemaV3.isFinalizedAndEnabled(config)) { HddsVolumeUtil.loadAllHddsVolumeDbStore( volumeSet, dbVolumeSet, false, LOG); + if (dnConf.autoCompactionSmallSstFile()) { + this.dbCompactionExecutorService = Executors.newScheduledThreadPool( + dnConf.getAutoCompactionSmallSstFileThreads(), + new ThreadFactoryBuilder().setNameFormat( + datanodeDetails.threadNamePrefix() + + "RocksDBCompactionThread-%d").build()); + this.dbCompactionExecutorService.scheduleWithFixedDelay(this::compactDb, + dnConf.getAutoCompactionSmallSstFileIntervalMinutes(), + dnConf.getAutoCompactionSmallSstFileIntervalMinutes(), + TimeUnit.MINUTES); + } } long recoveringContainerTimeout = config.getTimeDuration( @@ -188,12 +204,13 @@ public OzoneContainer( } }; + checksumTreeManager = new ContainerChecksumTreeManager(config); for (ContainerType containerType : ContainerType.values()) { handlers.put(containerType, Handler.getHandlerForContainerType( containerType, conf, context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, icrSender)); + containerSet, volumeSet, metrics, icrSender, checksumTreeManager)); } SecurityConfig secConf = new SecurityConfig(conf); @@ -208,7 +225,7 @@ public OzoneContainer( */ controller = new ContainerController(containerSet, handlers); - writeChannel = XceiverServerRatis.newXceiverServerRatis( + writeChannel = XceiverServerRatis.newXceiverServerRatis(hddsDatanodeService, datanodeDetails, config, hddsDispatcher, controller, certClient, context); @@ -223,10 +240,7 @@ public OzoneContainer( readChannel = new XceiverServerGrpc( datanodeDetails, config, hddsDispatcher, certClient); - Duration blockDeletingSvcInterval = conf.getObject( - DatanodeConfiguration.class).getBlockDeletionInterval(); - - checksumTreeManager = new ContainerChecksumTreeManager(config); + Duration blockDeletingSvcInterval = dnConf.getBlockDeletionInterval(); long blockDeletingServiceTimeout = config .getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_TIMEOUT, @@ -244,8 +258,8 @@ public OzoneContainer( checksumTreeManager, context.getParent().getReconfigurationHandler()); - Duration recoveringContainerScrubbingSvcInterval = conf.getObject( - DatanodeConfiguration.class).getRecoveringContainerScrubInterval(); + Duration recoveringContainerScrubbingSvcInterval = + dnConf.getRecoveringContainerScrubInterval(); long recoveringContainerScrubbingServiceTimeout = config .getTimeDuration(OZONE_RECOVERING_CONTAINER_SCRUBBING_SERVICE_TIMEOUT, @@ -283,7 +297,7 @@ public OzoneContainer( public OzoneContainer( DatanodeDetails datanodeDetails, ConfigurationSource conf, StateContext context) throws IOException { - this(datanodeDetails, conf, context, null, null); + this(null, datanodeDetails, conf, context, null, null); } public GrpcTlsConfig getTlsClientConfig() { @@ -497,11 +511,13 @@ public void stop() { if (dbVolumeSet != null) { dbVolumeSet.shutdown(); } + if (dbCompactionExecutorService != null) { + dbCompactionExecutorService.shutdown(); + } blockDeletingService.shutdown(); recoveringContainerScrubbingService.shutdown(); ContainerMetrics.remove(); - // TODO: To properly shut down ContainerMerkleTreeMetrics - ContainerMerkleTreeMetrics.unregister(); + checksumTreeManager.stop(); } public void handleVolumeFailures() { @@ -598,4 +614,12 @@ public ReplicationServer getReplicationServer() { return replicationServer; } + public void compactDb() { + for (StorageVolume volume : volumeSet.getVolumesList()) { + HddsVolume hddsVolume = (HddsVolume) volume; + CompletableFuture.runAsync(hddsVolume::compactDb, + dbCompactionExecutorService); + } + } + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconcileContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconcileContainerCommand.java index cdd4522cc69..3d24756a408 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconcileContainerCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconcileContainerCommand.java @@ -25,18 +25,20 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReconcileContainerCommandProto; import java.util.List; +import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; -import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; /** * Asks datanodes to reconcile the specified container with other container replicas. */ public class ReconcileContainerCommand extends SCMCommand { - private final List peerDatanodes; + private final Set peerDatanodes; - public ReconcileContainerCommand(long containerID, List peerDatanodes) { + public ReconcileContainerCommand(long containerID, Set peerDatanodes) { // Container ID serves as command ID, since only one reconciliation should be in progress at a time. super(containerID); this.peerDatanodes = peerDatanodes; @@ -58,7 +60,7 @@ public ReconcileContainerCommandProto getProto() { return builder.build(); } - public List getPeerDatanodes() { + public Set getPeerDatanodes() { return peerDatanodes; } @@ -70,11 +72,11 @@ public static ReconcileContainerCommand getFromProtobuf(ReconcileContainerComman Preconditions.checkNotNull(protoMessage); List peers = protoMessage.getPeersList(); - List peerNodes = !peers.isEmpty() + Set peerNodes = !peers.isEmpty() ? peers.stream() .map(DatanodeDetails::getFromProtoBuf) - .collect(Collectors.toList()) - : emptyList(); + .collect(Collectors.toSet()) + : emptySet(); return new ReconcileContainerCommand(protoMessage.getContainerID(), peerNodes); } @@ -85,4 +87,22 @@ public String toString() { ": containerId=" + getContainerID() + ", peerNodes=" + peerDatanodes; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReconcileContainerCommand that = (ReconcileContainerCommand) o; + return getContainerID() == that.getContainerID() && + Objects.equals(peerDatanodes, that.peerDatanodes); + } + + @Override + public int hashCode() { + return Objects.hash(getContainerID(), peerDatanodes); + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java index 3e9ed9493eb..eeb99b5a3db 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java @@ -84,11 +84,9 @@ public StorageContainerDatanodeProtocolClientSideTranslatorPB( * fail require careful attention. It is strongly advised to relinquish the * underlying resources and to internally mark the {@code Closeable} * as closed, prior to throwing the {@code IOException}. - * - * @throws IOException if an I/O error occurs */ @Override - public void close() throws IOException { + public void close() { RPC.stopProxy(rpcProxy); } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/checksum/TestReconcileContainerTask.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/checksum/TestReconcileContainerTask.java new file mode 100644 index 00000000000..04d08347ed4 --- /dev/null +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/checksum/TestReconcileContainerTask.java @@ -0,0 +1,126 @@ +/* + * 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.checksum; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; +import org.apache.hadoop.ozone.container.replication.AbstractReplicationTask; +import org.apache.hadoop.ozone.protocol.commands.ReconcileContainerCommand; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; + +class TestReconcileContainerTask { + private DNContainerOperationClient mockClient; + private ContainerController mockController; + + @BeforeEach + public void init() { + mockClient = mock(DNContainerOperationClient.class); + mockController = mock(ContainerController.class); + } + + @Test + public void testFailedTaskStatus() throws Exception { + doThrow(IOException.class).when(mockController).reconcileContainer(any(), anyLong(), any()); + ReconcileContainerTask task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(1, Collections.emptySet())); + + assertEquals(AbstractReplicationTask.Status.QUEUED, task.getStatus()); + task.runTask(); + assertEquals(AbstractReplicationTask.Status.FAILED, task.getStatus()); + } + + @Test + public void testSuccessfulTaskStatus() { + ReconcileContainerTask task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(1, Collections.emptySet())); + + assertEquals(AbstractReplicationTask.Status.QUEUED, task.getStatus()); + task.runTask(); + assertEquals(AbstractReplicationTask.Status.DONE, task.getStatus()); + } + + @Test + public void testEqualityWhenContainerIDsMatch() { + final long containerID = 1; + final UUID dnID1 = UUID.randomUUID(); + + Set peerSet1 = new HashSet<>(); + peerSet1.add(buildDn(dnID1)); + Set peerSet1Other = new HashSet<>(); + peerSet1Other.add(buildDn(dnID1)); + Set peerSet2 = new HashSet<>(); + peerSet2.add(buildDn()); + + ReconcileContainerTask peerSet1Task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(containerID, peerSet1)); + ReconcileContainerTask otherPeerSet1Task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(containerID, peerSet1Other)); + ReconcileContainerTask peerSet2Task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(containerID, peerSet2)); + + // Same container ID and peers. + assertEquals(peerSet1Task, otherPeerSet1Task); + // Same container ID, different peers. + assertNotEquals(peerSet1Task, peerSet2Task); + } + + @Test + public void testEqualityWhenContainerIDsDifferent() { + Set peerSet = new HashSet<>(); + peerSet.add(buildDn()); + + ReconcileContainerTask id1Task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(1, peerSet)); + ReconcileContainerTask id2Task = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(2, peerSet)); + ReconcileContainerTask id2NoPeersTask = new ReconcileContainerTask(mockController, mockClient, + new ReconcileContainerCommand(2, Collections.emptySet())); + + // Different container ID, same peers. + assertNotEquals(id1Task, id2Task); + // Different container ID, different peers. + assertNotEquals(id1Task, id2NoPeersTask); + } + + private DatanodeDetails buildDn(UUID id) { + return DatanodeDetails.newBuilder() + .setUuid(id) + .build(); + } + + private DatanodeDetails buildDn() { + return DatanodeDetails.newBuilder() + .setUuid(UUID.randomUUID()) + .build(); + } +} 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 b5b578554b1..823ac09bd37 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 @@ -17,6 +17,7 @@ package org.apache.hadoop.ozone.container.common; +import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.conf.ConfigurationSource; @@ -27,6 +28,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; +import org.apache.hadoop.hdds.security.token.TokenVerifier; import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource; import org.apache.hadoop.hdfs.util.Canceler; import org.apache.hadoop.hdfs.util.DataTransferThrottler; @@ -36,9 +38,12 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; +import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; @@ -54,8 +59,10 @@ import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy; +import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; @@ -73,6 +80,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -179,6 +187,46 @@ public static KeyValueContainer getContainer(long containerId, return new KeyValueContainer(kvData, new OzoneConfiguration()); } + /** + * Constructs an instance of KeyValueHandler that can be used for testing. + * This instance can be used for tests that do not need an ICR sender or {@link ContainerChecksumTreeManager}. + */ + public static KeyValueHandler getKeyValueHandler(ConfigurationSource config, + String datanodeId, ContainerSet contSet, VolumeSet volSet, ContainerMetrics metrics) { + return new KeyValueHandler(config, datanodeId, contSet, volSet, metrics, c -> { }, + new ContainerChecksumTreeManager(config)); + } + + /** + * Constructs an instance of KeyValueHandler that can be used for testing. + * This instance can be used for tests that do not need an ICR sender, metrics, or a + * {@link ContainerChecksumTreeManager}. + */ + public static KeyValueHandler getKeyValueHandler(ConfigurationSource config, + String datanodeId, ContainerSet contSet, VolumeSet volSet) { + return getKeyValueHandler(config, datanodeId, contSet, volSet, ContainerMetrics.create(config)); + } + + public static HddsDispatcher getHddsDispatcher(OzoneConfiguration conf, + ContainerSet contSet, + VolumeSet volSet, + StateContext context) { + return getHddsDispatcher(conf, contSet, volSet, context, null); + } + + public static HddsDispatcher getHddsDispatcher(OzoneConfiguration conf, + ContainerSet contSet, + VolumeSet volSet, + StateContext context, TokenVerifier verifier) { + ContainerMetrics metrics = ContainerMetrics.create(conf); + Map handlers = Maps.newHashMap(); + handlers.put(ContainerType.KeyValueContainer, ContainerTestUtils.getKeyValueHandler(conf, + context.getParent().getDatanodeDetails().getUuidString(), contSet, volSet, metrics)); + assertEquals(1, ContainerType.values().length, "Tests only cover KeyValueContainer type"); + return new HddsDispatcher( + conf, contSet, volSet, handlers, context, metrics, verifier); + } + public static void enableSchemaV3(OzoneConfiguration conf) { DatanodeConfiguration dc = conf.getObject(DatanodeConfiguration.class); dc.setContainerSchemaV3Enabled(true); @@ -346,7 +394,7 @@ public static XceiverServerRatis newXceiverServerRatis( conf.setInt(OzoneConfigKeys.HDDS_CONTAINER_RATIS_IPC_PORT, dn.getPort(DatanodeDetails.Port.Name.RATIS).getValue()); - return XceiverServerRatis.newXceiverServerRatis(dn, conf, + return XceiverServerRatis.newXceiverServerRatis(null, dn, conf, getNoopContainerDispatcher(), getEmptyContainerController(), null, null); } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java index 5cb698482a7..c333ba1b698 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java @@ -74,9 +74,9 @@ public ScmTestMock() { scmId = UUID.randomUUID().toString(); } - public ScmTestMock(String clusterId, String scmId) { + public ScmTestMock(String clusterId) { this.clusterId = clusterId; - this.scmId = scmId; + this.scmId = UUID.randomUUID().toString(); } // Map of datanode to containers @@ -368,6 +368,10 @@ public void setClusterId(String clusterId) { this.clusterId = clusterId; } + public String getScmId() { + return scmId; + } + public void setScmId(String scmId) { this.scmId = scmId; } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java index ab313d0ce66..fd0c81fb023 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java @@ -481,9 +481,7 @@ public void testPendingDeleteBlockReset(ContainerTestVersionInfo versionInfo) // runs so we can trigger it manually. ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); OzoneContainer ozoneContainer = mockDependencies(containerSet, keyValueHandler); BlockDeletingService svc = new BlockDeletingService(ozoneContainer, @@ -550,9 +548,7 @@ public void testBlockDeletion(ContainerTestVersionInfo versionInfo) createToDeleteBlocks(containerSet, 1, 3, 1); ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); BlockDeletingServiceTestImpl svc = getBlockDeletingService(containerSet, conf, keyValueHandler); svc.start(); @@ -684,9 +680,7 @@ public void testWithUnrecordedBlocks(ContainerTestVersionInfo versionInfo) ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); BlockDeletingServiceTestImpl svc = getBlockDeletingService(containerSet, conf, keyValueHandler); svc.start(); @@ -792,9 +786,7 @@ public void testShutdownService(ContainerTestVersionInfo versionInfo) createToDeleteBlocks(containerSet, 1, 100, 1); ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); BlockDeletingServiceTestImpl service = getBlockDeletingService(containerSet, conf, keyValueHandler); service.start(); @@ -822,9 +814,7 @@ public void testBlockDeletionTimeout(ContainerTestVersionInfo versionInfo) createToDeleteBlocks(containerSet, 1, 3, 1); ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); // set timeout value as 1ns to trigger timeout behavior long timeout = 1; OzoneContainer ozoneContainer = @@ -929,9 +919,7 @@ public void testContainerThrottle(ContainerTestVersionInfo versionInfo) chunksPerBlock); ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); BlockDeletingServiceTestImpl service = getBlockDeletingService(containerSet, conf, keyValueHandler); service.start(); @@ -988,9 +976,7 @@ public void testContainerMaxLockHoldingTime( createToDeleteBlocks(containerSet, containerCount, blocksPerContainer, chunksPerBlock); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - ContainerMetrics.create(conf), c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet); BlockDeletingServiceTestImpl service = getBlockDeletingService(containerSet, conf, keyValueHandler); service.start(); @@ -1047,9 +1033,7 @@ public void testBlockThrottle(ContainerTestVersionInfo versionInfo) ContainerSet containerSet = new ContainerSet(1000); ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, metrics); int containerCount = 5; int blocksPerContainer = 3; createToDeleteBlocks(containerSet, containerCount, @@ -1119,7 +1103,7 @@ public void testChecksumFileUpdatedWhenDeleteRetried(ContainerTestVersionInfo ve ContainerSet containerSet = new ContainerSet(1000); KeyValueContainerData contData = createToDeleteBlocks(containerSet, numBlocks, 4); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, ContainerMetrics.create(conf), c -> { }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet); BlockDeletingServiceTestImpl svc = getBlockDeletingService(containerSet, conf, keyValueHandler); svc.start(); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java index ad5ca482189..77746041270 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java @@ -28,7 +28,6 @@ import org.apache.hadoop.ozone.OzoneConsts; 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.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator; @@ -279,11 +278,8 @@ public void testDelete(String schemaVersion) throws Exception { ContainerSet containerSet = makeContainerSet(); VolumeSet volumeSet = new MutableVolumeSet(datanodeUuid, conf, null, StorageVolume.VolumeType.DATA_VOLUME, null); - ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet); long initialTotalSpace = newKvData().getBytesUsed(); long blockSpace = initialTotalSpace / TestDB.KEY_COUNT; @@ -352,11 +348,8 @@ public void testReadDeletedBlockChunkInfo(String schemaVersion) ContainerSet containerSet = makeContainerSet(); VolumeSet volumeSet = new MutableVolumeSet(datanodeUuid, conf, null, StorageVolume.VolumeType.DATA_VOLUME, null); - ContainerMetrics metrics = ContainerMetrics.create(conf); KeyValueHandler keyValueHandler = - new KeyValueHandler(conf, datanodeUuid, containerSet, volumeSet, - metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet); KeyValueContainerData cData = newKvData(); try (DBHandle refCountedDB = BlockUtils.getDB(cData, conf)) { // Read blocks that were already deleted before the upgrade. diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java index 0c4612b79fa..da0d2384ab6 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaTwoBackwardsCompatibility.java @@ -32,7 +32,6 @@ import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; -import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator; @@ -135,8 +134,7 @@ public void setup() throws Exception { chunkManager = new FilePerBlockStrategy(true, blockManager, volumeSet); containerSet = new ContainerSet(1000); - keyValueHandler = new KeyValueHandler(conf, datanodeUuid, - containerSet, volumeSet, ContainerMetrics.create(conf), c -> { }); + keyValueHandler = ContainerTestUtils.getKeyValueHandler(conf, datanodeUuid, containerSet, volumeSet); ozoneContainer = mock(OzoneContainer.class); when(ozoneContainer.getContainerSet()).thenReturn(containerSet); when(ozoneContainer.getWriteChannel()).thenReturn(null); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java index 3ff8f9e625d..993179d1b79 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java @@ -49,6 +49,7 @@ import org.apache.hadoop.ozone.common.ChecksumData; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; @@ -311,7 +312,7 @@ public void testDeleteNonEmptyContainer(ContainerTestVersionInfo versionInfo) KeyValueHandler kvHandler = new KeyValueHandler(conf, datanodeId, containerSet, volumeSet, metrics, - c -> icrReceived.incrementAndGet()); + c -> icrReceived.incrementAndGet(), new ContainerChecksumTreeManager(conf)); Exception exception = assertThrows( StorageContainerException.class, 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 1cbd6ee4706..05bebdd1b90 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 @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.container.common.impl; -import com.google.common.collect.Maps; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.StorageUnit; @@ -34,7 +33,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProtoOrBuilder; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerAction; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -48,8 +46,6 @@ import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.interfaces.Container; -import org.apache.hadoop.ozone.container.common.interfaces.Handler; -import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; @@ -78,7 +74,6 @@ import java.time.Duration; import java.util.Collections; import java.util.HashMap; -import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -113,10 +108,6 @@ public class TestHddsDispatcher { @TempDir private File testDir; - public static final IncrementalReportSender NO_OP_ICR_SENDER = - c -> { - }; - @ContainerLayoutTestInfo.ContainerTest public void testContainerCloseActionWhenFull( ContainerLayoutVersion layout) throws IOException { @@ -143,16 +134,7 @@ public void testContainerCloseActionWhenFull( container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), scmId.toString()); containerSet.addContainer(container); - ContainerMetrics metrics = ContainerMetrics.create(conf); - Map handlers = Maps.newHashMap(); - for (ContainerType containerType : ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType(containerType, conf, - context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, NO_OP_ICR_SENDER)); - } - HddsDispatcher hddsDispatcher = new HddsDispatcher( - conf, containerSet, volumeSet, handlers, context, metrics, null); + HddsDispatcher hddsDispatcher = ContainerTestUtils.getHddsDispatcher(conf, containerSet, volumeSet, context); hddsDispatcher.setClusterId(scmId.toString()); ContainerCommandResponseProto responseOne = hddsDispatcher .dispatch(getWriteChunkRequest(dd.getUuidString(), 1L, 1L), null); @@ -279,16 +261,7 @@ public void testContainerCloseActionWhenVolumeFull( container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), scmId.toString()); containerSet.addContainer(container); - ContainerMetrics metrics = ContainerMetrics.create(conf); - Map handlers = Maps.newHashMap(); - for (ContainerType containerType : ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType(containerType, conf, - context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, NO_OP_ICR_SENDER)); - } - HddsDispatcher hddsDispatcher = new HddsDispatcher( - conf, containerSet, volumeSet, handlers, context, metrics, null); + HddsDispatcher hddsDispatcher = ContainerTestUtils.getHddsDispatcher(conf, containerSet, volumeSet, context); hddsDispatcher.setClusterId(scmId.toString()); containerData.getVolume().getVolumeInfo() .ifPresent(volumeInfo -> volumeInfo.incrementUsedSpace(50)); @@ -528,17 +501,8 @@ static HddsDispatcher createDispatcher(DatanodeDetails dd, UUID scmId, } }); StateContext context = ContainerTestUtils.getMockContext(dd, conf); - ContainerMetrics metrics = ContainerMetrics.create(conf); - Map handlers = Maps.newHashMap(); - for (ContainerType containerType : ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType(containerType, conf, - context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, NO_OP_ICR_SENDER)); - } - - final HddsDispatcher hddsDispatcher = new HddsDispatcher(conf, - containerSet, volumeSet, handlers, context, metrics, tokenVerifier); + final HddsDispatcher hddsDispatcher = + ContainerTestUtils.getHddsDispatcher(conf, containerSet, volumeSet, context, tokenVerifier); hddsDispatcher.setClusterId(scmId.toString()); return hddsDispatcher; } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java index 8f2ad307e82..27257d5a0e1 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java @@ -18,8 +18,6 @@ package org.apache.hadoop.ozone.container.common.interfaces; -import java.util.Map; - import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; @@ -27,13 +25,11 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; -import org.apache.hadoop.ozone.container.common.impl.TestHddsDispatcher; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; -import com.google.common.collect.Maps; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -62,19 +58,7 @@ public void setup() throws Exception { DatanodeDetails datanodeDetails = mock(DatanodeDetails.class); StateContext context = ContainerTestUtils.getMockContext( datanodeDetails, conf); - ContainerMetrics metrics = ContainerMetrics.create(conf); - Map handlers = Maps.newHashMap(); - for (ContainerProtos.ContainerType containerType : - ContainerProtos.ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType( - containerType, conf, - context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, - TestHddsDispatcher.NO_OP_ICR_SENDER)); - } - this.dispatcher = new HddsDispatcher( - conf, containerSet, volumeSet, handlers, null, metrics, null); + this.dispatcher = ContainerTestUtils.getHddsDispatcher(conf, containerSet, volumeSet, context); } @AfterEach diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestStateContext.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestStateContext.java index 6933400fba1..fd4614335e3 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestStateContext.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/TestStateContext.java @@ -705,7 +705,7 @@ public void testCommandQueueSummary() throws IOException { ctx.addCommand(ReplicateContainerCommand.forTest(3)); ctx.addCommand(new ClosePipelineCommand(PipelineID.randomId())); ctx.addCommand(new CloseContainerCommand(1, PipelineID.randomId())); - ctx.addCommand(new ReconcileContainerCommand(4, Collections.emptyList())); + ctx.addCommand(new ReconcileContainerCommand(4, Collections.emptySet())); Map summary = ctx.getCommandQueueSummary(); assertEquals(3, diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconcileContainerCommandHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconcileContainerCommandHandler.java index d6be667f41b..f27ed097d2f 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconcileContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReconcileContainerCommandHandler.java @@ -24,6 +24,9 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; +import org.apache.hadoop.ozone.container.checksum.ReconcileContainerTask; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; @@ -39,16 +42,14 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; +import org.apache.hadoop.ozone.container.replication.ReplicationSupervisor; import org.apache.hadoop.ozone.protocol.commands.ReconcileContainerCommand; -import org.apache.ozone.test.GenericTestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import static java.util.Collections.singletonMap; import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails; @@ -56,7 +57,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -78,7 +80,14 @@ public void init(ContainerLayoutVersion layout, IncrementalReportSender { + ((ReconcileContainerTask)invocation.getArguments()[0]).runTask(); + return null; + }).when(mockSupervisor).addTask(any()); + + subject = new ReconcileContainerCommandHandler(mockSupervisor, mock(DNContainerOperationClient.class)); context = ContainerTestUtils.getMockContext(dnDetails, conf); containerSet = new ContainerSet(1000); @@ -91,7 +100,7 @@ public void init(ContainerLayoutVersion layout, IncrementalReportSender { }); - IncrementalReportSender icrSender = c -> { - try { - // Block the caller until the latch is counted down. - // Caller can check queue metrics in the meantime. - LOG.info("ICR sender waiting for latch"); - assertTrue(icrLatch.await(30, TimeUnit.SECONDS)); - LOG.info("ICR sender proceeding after latch"); - - Thread.sleep(minExecTimeMillis); - } catch (Exception ex) { - LOG.error("ICR sender failed", ex); - } - }; - - init(layout, icrSender); + assertEquals(0, subject.getInvocationCount()); // All commands submitted will be blocked until the latch is counted down. for (int id = 1; id <= NUM_CONTAINERS; id++) { - ReconcileContainerCommand cmd = new ReconcileContainerCommand(id, Collections.emptyList()); + ReconcileContainerCommand cmd = new ReconcileContainerCommand(id, Collections.emptySet()); subject.handle(cmd, ozoneContainer, context, null); } - assertEquals(NUM_CONTAINERS, subject.getQueuedCount()); - assertEquals(0, subject.getTotalRunTime()); - assertEquals(0, subject.getAverageRunTime()); - - // This will resume handling of the tasks. - icrLatch.countDown(); - waitForAllCommandsToFinish(); - assertEquals(NUM_CONTAINERS, subject.getInvocationCount()); - long totalRunTime = subject.getTotalRunTime(); - assertTrue(totalRunTime >= expectedTotalMinExecTimeMillis, - "Total run time " + totalRunTime + "ms was not larger than the minimum total exec time " + - expectedTotalMinExecTimeMillis + "ms"); - long avgRunTime = subject.getAverageRunTime(); - assertTrue(avgRunTime >= minExecTimeMillis, - "Average run time " + avgRunTime + "ms was not larger than the minimum per task exec time " + - minExecTimeMillis + "ms"); - } - - private void waitForAllCommandsToFinish() throws Exception { - // Queue count should be decremented only after the task completes, so the other metrics should be consistent when - // it reaches zero. - GenericTestUtils.waitFor(() -> { - int qCount = subject.getQueuedCount(); - LOG.info("Waiting for queued command count to reach 0. Currently at " + qCount); - return qCount == 0; - }, 500, 3000); } - private void verifyAllContainerReports(Map reportsSent) throws Exception { + private void verifyAllContainerReports(Map reportsSent) { assertEquals(NUM_CONTAINERS, reportsSent.size()); for (Map.Entry entry: reportsSent.entrySet()) { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java index 6245489f13b..49b109b9139 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java @@ -33,9 +33,11 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.OptionalLong; +import java.util.Set; import java.util.UUID; import com.google.protobuf.Proto2Utils; @@ -118,7 +120,7 @@ public void testHandlesReconcileContainerCommand() throws Exception { StorageContainerDatanodeProtocolClientSideTranslatorPB scm = mock(StorageContainerDatanodeProtocolClientSideTranslatorPB.class); - List peerDNs = new ArrayList<>(); + Set peerDNs = new HashSet<>(); peerDNs.add(MockDatanodeDetails.randomDatanodeDetails()); peerDNs.add(MockDatanodeDetails.randomDatanodeDetails()); ReconcileContainerCommand cmd = new ReconcileContainerCommand(1, peerDNs); 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 15d0faefdf9..e1a3de30ddf 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 @@ -137,6 +137,7 @@ private void init(ContainerTestVersionInfo versionInfo) throws Exception { CodecBuffer.enableLeakDetection(); DatanodeConfiguration dc = CONF.getObject(DatanodeConfiguration.class); + dc.setAutoCompactionSmallSstFile(true); dc.setAutoCompactionSmallSstFileNum(100); dc.setRocksdbDeleteObsoleteFilesPeriod(5000); CONF.setFromObject(dc); @@ -896,7 +897,7 @@ void testAutoCompactionSmallSstFile( CONF).getStore(); List fileMetaDataList1 = ((RDBStore)(dnStore.getStore())).getDb().getLiveFilesMetaData(); - hddsVolume.check(true); + hddsVolume.compactDb(); // Sleep a while to wait for compaction to complete Thread.sleep(7000); List fileMetaDataList2 = 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 158932da4c0..1ad3432a97e 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 @@ -44,6 +44,8 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.security.token.TokenVerifier; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; +import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; @@ -51,6 +53,7 @@ import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.Handler; +import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil; import org.apache.hadoop.ozone.container.common.volume.HddsVolume; @@ -282,16 +285,11 @@ public void testVolumeSetInKeyValueHandler() throws Exception { null, StorageVolume.VolumeType.DATA_VOLUME, null); try { ContainerSet cset = new ContainerSet(1000); - int[] interval = new int[1]; - interval[0] = 2; - ContainerMetrics metrics = new ContainerMetrics(interval); DatanodeDetails datanodeDetails = mock(DatanodeDetails.class); StateContext context = ContainerTestUtils.getMockContext( datanodeDetails, conf); - KeyValueHandler keyValueHandler = new KeyValueHandler(conf, - context.getParent().getDatanodeDetails().getUuidString(), cset, - volumeSet, metrics, c -> { - }); + KeyValueHandler keyValueHandler = ContainerTestUtils.getKeyValueHandler(conf, + context.getParent().getDatanodeDetails().getUuidString(), cset, volumeSet); assertEquals("org.apache.hadoop.ozone.container.common" + ".volume.CapacityVolumeChoosingPolicy", keyValueHandler.getVolumeChoosingPolicyForTesting() @@ -301,8 +299,8 @@ public void testVolumeSetInKeyValueHandler() throws Exception { conf.set(HDDS_DATANODE_VOLUME_CHOOSING_POLICY, "org.apache.hadoop.ozone.container.common.impl.HddsDispatcher"); RuntimeException exception = assertThrows(RuntimeException.class, - () -> new KeyValueHandler(conf, context.getParent().getDatanodeDetails().getUuidString(), cset, volumeSet, - metrics, c -> { })); + () -> ContainerTestUtils.getKeyValueHandler(conf, context.getParent().getDatanodeDetails().getUuidString(), + cset, volumeSet)); assertThat(exception).hasMessageEndingWith( "class org.apache.hadoop.ozone.container.common.impl.HddsDispatcher " + @@ -392,7 +390,7 @@ public void testDeleteContainer() throws IOException { final KeyValueHandler kvHandler = new KeyValueHandler(conf, datanodeId, containerSet, volumeSet, metrics, - c -> icrReceived.incrementAndGet()); + c -> icrReceived.incrementAndGet(), new ContainerChecksumTreeManager(conf)); kvHandler.setClusterID(clusterId); final ContainerCommandRequestProto createContainer = @@ -466,8 +464,7 @@ public void testReconcileContainer(ContainerLayoutVersion layoutVersion) throws // Allows checking the invocation count of the lambda. AtomicInteger icrCount = new AtomicInteger(0); - KeyValueHandler keyValueHandler = new KeyValueHandler(conf, randomDatanodeDetails().getUuidString(), containerSet, - mock(MutableVolumeSet.class), mock(ContainerMetrics.class), c -> { + IncrementalReportSender icrSender = c -> { // Check that the ICR contains expected info about the container. ContainerReplicaProto report = c.getContainerReport(); long reportedID = report.getContainerID(); @@ -477,11 +474,14 @@ public void testReconcileContainer(ContainerLayoutVersion layoutVersion) throws Assertions.assertNotEquals(0, reportDataChecksum, "Container report should have populated the checksum field with a non-zero value."); icrCount.incrementAndGet(); - }); + }; + + KeyValueHandler keyValueHandler = new KeyValueHandler(conf, randomDatanodeDetails().getUuidString(), containerSet, + mock(MutableVolumeSet.class), mock(ContainerMetrics.class), icrSender, new ContainerChecksumTreeManager(conf)); Assertions.assertEquals(0, icrCount.get()); // This should trigger container report validation in the ICR handler above. - keyValueHandler.reconcileContainer(container, Collections.emptyList()); + keyValueHandler.reconcileContainer(mock(DNContainerOperationClient.class), container, Collections.emptySet()); Assertions.assertEquals(1, icrCount.get()); } 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 f0c8a2077ea..82079932185 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 @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.ozone.ClientVersion; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; @@ -254,7 +255,7 @@ private KeyValueHandler getDummyHandler() { stateMachine.getDatanodeDetails().getUuidString(), mock(ContainerSet.class), mock(MutableVolumeSet.class), - mock(ContainerMetrics.class), mockIcrSender); + mock(ContainerMetrics.class), mockIcrSender, mock(ContainerChecksumTreeManager.class)); } private KeyValueContainer getMockUnhealthyContainer() { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/helpers/TestChunkUtils.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/helpers/TestChunkUtils.java index 1a1158a210f..d04f3a5167f 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/helpers/TestChunkUtils.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/helpers/TestChunkUtils.java @@ -22,14 +22,15 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Random; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -37,6 +38,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.util.concurrent.Striped; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; @@ -81,6 +83,7 @@ static ChunkBuffer readData(File file, long off, long len) @Test void concurrentReadOfSameFile() throws Exception { + int threads = 10; String s = "Hello World"; byte[] array = s.getBytes(UTF_8); ChunkBuffer data = ChunkBuffer.wrap(ByteBuffer.wrap(array)); @@ -89,7 +92,6 @@ void concurrentReadOfSameFile() throws Exception { int offset = 0; File file = tempFile.toFile(); ChunkUtils.writeData(file, data, offset, len, null, true); - int threads = 10; ExecutorService executor = new ThreadPoolExecutor(threads, threads, 0, TimeUnit.SECONDS, new LinkedBlockingQueue<>()); AtomicInteger processed = new AtomicInteger(); @@ -104,12 +106,15 @@ void concurrentReadOfSameFile() throws Exception { assertEquals(1, buffers.size()); final ByteBuffer readBuffer = buffers.get(0); + int remaining = readBuffer.remaining(); + byte[] readArray = new byte[remaining]; + readBuffer.get(readArray); LOG.info("Read data ({}): {}", threadNumber, - new String(readBuffer.array(), UTF_8)); - if (!Arrays.equals(array, readBuffer.array())) { + new String(readArray, UTF_8)); + if (!Arrays.equals(array, readArray)) { failed.set(true); } - assertEquals(len, readBuffer.remaining()); + assertEquals(len, remaining); } catch (Exception e) { LOG.error("Failed to read data ({})", threadNumber, e); failed.set(true); @@ -127,38 +132,68 @@ void concurrentReadOfSameFile() throws Exception { } @Test - void concurrentProcessing() throws Exception { - final int perThreadWait = 1000; - final int maxTotalWait = 5000; - int threads = 20; + void concurrentReadWriteOfSameFile() { + final int threads = 10; + ChunkUtils.setStripedLock(Striped.readWriteLock(threads)); + final byte[] array = "Hello World".getBytes(UTF_8); - ExecutorService executor = new ThreadPoolExecutor(threads, threads, + Path tempFile = tempDir.toPath().resolve("concurrent_read_write"); + File file = tempFile.toFile(); + AtomicInteger success = new AtomicInteger(0); + AtomicInteger fail = new AtomicInteger(0); + + ExecutorService executor = new ThreadPoolExecutor(10, 10, 0, TimeUnit.SECONDS, new LinkedBlockingQueue<>()); - AtomicInteger processed = new AtomicInteger(); + + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { - Path path = tempDir.toPath().resolve(String.valueOf(i)); - executor.execute(() -> { + final int threadNumber = i; + final ChunkBuffer data = ChunkBuffer.wrap(ByteBuffer.wrap(array)); + final int len = data.limit(); + final int offset = i * len; + + CompletableFuture future = CompletableFuture.runAsync(() -> { try { - ChunkUtils.processFileExclusively(path, () -> { - try { - Thread.sleep(perThreadWait); - } catch (InterruptedException e) { - e.printStackTrace(); + ChunkUtils.writeData(file, data, offset, len, null, true); + success.getAndIncrement(); + } catch (StorageContainerException e) { + throw new RuntimeException(e); + } + }, executor).whenCompleteAsync((v, e) -> { + if (e == null) { + try { + final ChunkBuffer chunk = readData(file, offset, len); + // There should be only one element in readBuffers + final List buffers = chunk.asByteBufferList(); + assertEquals(1, buffers.size()); + final ByteBuffer readBuffer = buffers.get(0); + + LOG.info("Read data ({}): {}", threadNumber, + new String(readBuffer.array(), UTF_8)); + if (!Arrays.equals(array, readBuffer.array())) { + fail.getAndIncrement(); } - processed.incrementAndGet(); - return null; - }); - } catch (InterruptedException e) { - e.printStackTrace(); + assertEquals(len, readBuffer.remaining()); + } catch (Exception ee) { + LOG.error("Failed to read data ({})", threadNumber, ee); + fail.getAndIncrement(); + } + } else { + fail.getAndIncrement(); } - }); + }, executor); + futures.add(future); } try { - GenericTestUtils.waitFor(() -> processed.get() == threads, - 100, maxTotalWait); + for (CompletableFuture future : futures) { + future.join(); + } } finally { executor.shutdownNow(); } + assertEquals(success.get(), threads); + assertEquals(fail.get(), 0); } @Test @@ -177,9 +212,11 @@ void serialRead() throws IOException { final List buffers = chunk.asByteBufferList(); assertEquals(1, buffers.size()); final ByteBuffer readBuffer = buffers.get(0); - - assertArrayEquals(array, readBuffer.array()); - assertEquals(len, readBuffer.remaining()); + int remain = readBuffer.remaining(); + byte[] readArray = new byte[remain]; + readBuffer.get(readArray); + assertArrayEquals(array, readArray); + assertEquals(len, remain); } @Test @@ -233,29 +270,29 @@ void testReadData() throws Exception { // large file final int large = 10 << 20; // 10MB assertThat(large).isGreaterThan(MAPPED_BUFFER_THRESHOLD); - runTestReadFile(large, dir, true); + runTestReadFile(large, dir); // small file final int small = 30 << 10; // 30KB assertThat(small).isLessThanOrEqualTo(MAPPED_BUFFER_THRESHOLD); - runTestReadFile(small, dir, false); + runTestReadFile(small, dir); // boundary case - runTestReadFile(MAPPED_BUFFER_THRESHOLD, dir, false); + runTestReadFile(MAPPED_BUFFER_THRESHOLD, dir); // empty file - runTestReadFile(0, dir, false); + runTestReadFile(0, dir); for (int i = 0; i < 10; i++) { final int length = RANDOM.nextInt(2 * MAPPED_BUFFER_THRESHOLD) + 1; - runTestReadFile(length, dir, length > MAPPED_BUFFER_THRESHOLD); + runTestReadFile(length, dir); } } finally { FileUtils.deleteDirectory(dir); } } - void runTestReadFile(int length, File dir, boolean isMapped) + void runTestReadFile(int length, File dir) throws Exception { final File file; for (int i = length; ; i++) { @@ -295,7 +332,6 @@ void runTestReadFile(int length, File dir, boolean isMapped) RANDOM.setSeed(seed); for (ByteBuffer b : buffers) { - assertEquals(isMapped, b instanceof MappedByteBuffer); RANDOM.nextBytes(array); assertEquals(ByteBuffer.wrap(array, 0, b.remaining()), b); } 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 03901b99be3..b8c43460ba3 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 @@ -25,7 +25,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.ozone.OzoneConfigKeys; -import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.Handler; @@ -35,7 +35,6 @@ import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; -import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver; import org.junit.jupiter.api.AfterEach; @@ -129,11 +128,8 @@ public void init(boolean isZeroCopy) throws Exception { when(volumeSet.getVolumesList()).thenReturn(Collections.singletonList( new HddsVolume.Builder(testDir).conf(conf).build())); - ContainerMetrics metrics = ContainerMetrics.create(conf); Handler containerHandler = - new KeyValueHandler(conf, datanode.getUuidString(), containerSet, - volumeSet, metrics, c -> { - }); + ContainerTestUtils.getKeyValueHandler(conf, datanode.getUuidString(), containerSet, volumeSet); containerController = new ContainerController(containerSet, Collections.singletonMap( diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToHBaseSupport.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToHBaseSupport.java index 37bea49b342..55396446d50 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToHBaseSupport.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToHBaseSupport.java @@ -20,44 +20,28 @@ import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ozone.container.ContainerTestHelper; -import org.apache.hadoop.ozone.container.common.ContainerTestUtils; -import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage; import org.apache.hadoop.ozone.container.common.SCMTestUtils; import org.apache.hadoop.ozone.container.common.ScmTestMock; import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; -import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine; -import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import java.io.File; -import java.io.IOException; import java.net.InetSocketAddress; -import java.nio.file.Files; import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; -import java.util.List; -import java.util.Random; -import java.util.UUID; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State.OPEN; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests upgrading a single datanode from HADOOP_PRC_PORTS_IN_DATANODEDETAILS to HBASE_SUPPORT. @@ -67,22 +51,19 @@ public class TestDatanodeUpgradeToHBaseSupport { private Path tempFolder; private DatanodeStateMachine dsm; + private ContainerDispatcher dispatcher; private OzoneConfiguration conf; private static final String CLUSTER_ID = "clusterID"; private RPC.Server scmRpcServer; private InetSocketAddress address; - private Random random; - private void initTests() throws Exception { conf = new OzoneConfiguration(); setup(); } private void setup() throws Exception { - random = new Random(); - address = SCMTestUtils.getReuseableAddress(); conf.setSocketAddr(ScmConfigKeys.OZONE_SCM_NAMES, address); conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, @@ -107,26 +88,30 @@ public void teardown() throws Exception { public void testIncrementalChunkListBeforeAndAfterUpgrade() throws Exception { initTests(); // start DN and SCM - startScmServer(); - addHddsVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.HADOOP_PRC_PORTS_IN_DATANODEDETAILS.layoutVersion()); + dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); // Add data to read. - final long containerID = addContainer(pipeline); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); // incremental chunk list should be rejected before finalizing. - putBlock(containerID, pipeline, true, ContainerProtos.Result.UNSUPPORTED_REQUEST); + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline, true, ContainerProtos.Result.UNSUPPORTED_REQUEST); Container container = dsm.getContainer().getContainerSet().getContainer(containerID); assertEquals(OPEN, container.getContainerData().getState()); // close container to allow upgrade. - closeContainer(containerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline); dsm.finalizeUpgrade(); assertTrue(dsm.getLayoutVersionManager().isAllowed(HDDSLayoutFeature.HBASE_SUPPORT)); // open a new container after finalization - final long containerID2 = addContainer(pipeline); + final long containerID2 = UpgradeTestHelper.addContainer(dispatcher, pipeline); // incremental chunk list should work after finalizing. - putBlock(containerID2, pipeline, true); + UpgradeTestHelper.putBlock(dispatcher, containerID2, pipeline, true); Container container2 = dsm.getContainer().getContainerSet().getContainer(containerID2); assertEquals(OPEN, container2.getContainerData().getState()); } @@ -138,195 +123,36 @@ public void testIncrementalChunkListBeforeAndAfterUpgrade() throws Exception { public void testBlockFinalizationBeforeAndAfterUpgrade() throws Exception { initTests(); // start DN and SCM - startScmServer(); - addHddsVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.HADOOP_PRC_PORTS_IN_DATANODEDETAILS.layoutVersion()); + dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); // Add data to read. - final long containerID = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID, pipeline, false); - finalizeBlock(containerID, writeChunk.getBlockID().getLocalID(), ContainerProtos.Result.UNSUPPORTED_REQUEST); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk = + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline, false); + UpgradeTestHelper.finalizeBlock( + dispatcher, containerID, writeChunk.getBlockID().getLocalID(), ContainerProtos.Result.UNSUPPORTED_REQUEST); Container container = dsm.getContainer().getContainerSet().getContainer(containerID); assertEquals(OPEN, container.getContainerData().getState()); // close container to allow upgrade. - closeContainer(containerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline); dsm.finalizeUpgrade(); assertTrue(dsm.getLayoutVersionManager().isAllowed(HDDSLayoutFeature.HBASE_SUPPORT)); - final long containerID2 = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk2 = putBlock(containerID2, pipeline, false); + final long containerID2 = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk2 = + UpgradeTestHelper.putBlock(dispatcher, containerID2, pipeline, false); // Make sure we can read after finalizing too. - finalizeBlock(containerID2, writeChunk2.getBlockID().getLocalID(), ContainerProtos.Result.SUCCESS); + UpgradeTestHelper.finalizeBlock( + dispatcher, containerID2, writeChunk2.getBlockID().getLocalID(), ContainerProtos.Result.SUCCESS); Container container2 = dsm.getContainer().getContainerSet().getContainer(containerID2); assertEquals(OPEN, container2.getContainerData().getState()); } - /** - * Starts the datanode with the fore layout version, and calls the version - * endpoint task to get cluster ID and SCM ID. - * - * The daemon for the datanode state machine is not started in this test. - * This greatly speeds up execution time. - * It means we do not have heartbeat functionality or pre-finalize - * upgrade actions, but neither of those things are needed for these tests. - */ - public void startPreFinalizedDatanode() throws Exception { - // Set layout version. - conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFolder.toString()); - DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf, - UUID.randomUUID().toString(), - HDDSLayoutFeature.HADOOP_PRC_PORTS_IN_DATANODEDETAILS.layoutVersion()); - layoutStorage.initialize(); - - // Build and start the datanode. - DatanodeDetails dd = ContainerTestUtils.createDatanodeDetails(); - DatanodeStateMachine newDsm = new DatanodeStateMachine(dd, conf); - int actualMlv = newDsm.getLayoutVersionManager().getMetadataLayoutVersion(); - assertEquals( - HDDSLayoutFeature.HADOOP_PRC_PORTS_IN_DATANODEDETAILS.layoutVersion(), - actualMlv); - if (dsm != null) { - dsm.close(); - } - dsm = newDsm; - - callVersionEndpointTask(); - } - - /** - * Get the cluster ID and SCM ID from SCM to the datanode. - */ - public void callVersionEndpointTask() throws Exception { - try (EndpointStateMachine esm = ContainerTestUtils.createEndpoint(conf, - address, 1000)) { - VersionEndpointTask vet = new VersionEndpointTask(esm, conf, - dsm.getContainer()); - esm.setState(EndpointStateMachine.EndPointStates.GETVERSION); - vet.call(); - } - } - - public String startScmServer() throws IOException { - String scmID = UUID.randomUUID().toString(); - ScmTestMock scmServerImpl = new ScmTestMock(CLUSTER_ID, scmID); - scmRpcServer = SCMTestUtils.startScmRpcServer(conf, - scmServerImpl, address, 10); - return scmID; - } - - /// CONTAINER OPERATIONS /// - public void readChunk(ContainerProtos.WriteChunkRequestProto writeChunk, - Pipeline pipeline) throws Exception { - ContainerProtos.ContainerCommandRequestProto readChunkRequest = - ContainerTestHelper.getReadChunkRequest(pipeline, writeChunk); - - dispatchRequest(readChunkRequest); - } - - public ContainerProtos.WriteChunkRequestProto putBlock(long containerID, - Pipeline pipeline, boolean incremental) throws Exception { - return putBlock(containerID, pipeline, incremental, ContainerProtos.Result.SUCCESS); - } - - public ContainerProtos.WriteChunkRequestProto putBlock(long containerID, - Pipeline pipeline, boolean incremental, ContainerProtos.Result expectedResult) throws Exception { - ContainerProtos.ContainerCommandRequestProto writeChunkRequest = - getWriteChunk(containerID, pipeline); - dispatchRequest(writeChunkRequest); - - ContainerProtos.ContainerCommandRequestProto putBlockRequest = - ContainerTestHelper.getPutBlockRequest(pipeline, - writeChunkRequest.getWriteChunk(), incremental); - dispatchRequest(putBlockRequest, expectedResult); - - return writeChunkRequest.getWriteChunk(); - } - - public ContainerProtos.ContainerCommandRequestProto getWriteChunk( - long containerID, Pipeline pipeline) throws Exception { - return ContainerTestHelper.getWriteChunkRequest(pipeline, - ContainerTestHelper.getTestBlockID(containerID), 100); - } - - public Pipeline getPipeline() { - return MockPipeline.createPipeline( - Collections.singletonList(dsm.getDatanodeDetails())); - } - - public long addContainer(Pipeline pipeline) - throws Exception { - long containerID = random.nextInt(Integer.MAX_VALUE); - ContainerProtos.ContainerCommandRequestProto createContainerRequest = - ContainerTestHelper.getCreateContainerRequest(containerID, pipeline); - dispatchRequest(createContainerRequest); - - return containerID; - } - - public void deleteContainer(long containerID, Pipeline pipeline) - throws Exception { - ContainerProtos.ContainerCommandRequestProto deleteContainerRequest = - ContainerTestHelper.getDeleteContainer(pipeline, containerID, true); - dispatchRequest(deleteContainerRequest); - } - - public void closeContainer(long containerID, Pipeline pipeline) - throws Exception { - closeContainer(containerID, pipeline, ContainerProtos.Result.SUCCESS); - } - - public void closeContainer(long containerID, Pipeline pipeline, - ContainerProtos.Result expectedResult) throws Exception { - ContainerProtos.ContainerCommandRequestProto closeContainerRequest = - ContainerTestHelper.getCloseContainer(pipeline, containerID); - dispatchRequest(closeContainerRequest, expectedResult); - } - - public void finalizeBlock(long containerID, long localID, ContainerProtos.Result expectedResult) { - ContainerInfo container = mock(ContainerInfo.class); - when(container.getContainerID()).thenReturn(containerID); - - ContainerProtos.ContainerCommandRequestProto finalizeBlockRequest = - ContainerTestHelper.getFinalizeBlockRequest(localID, container, UUID.randomUUID().toString()); - - dispatchRequest(finalizeBlockRequest, expectedResult); - } - - public void dispatchRequest( - ContainerProtos.ContainerCommandRequestProto request) { - dispatchRequest(request, ContainerProtos.Result.SUCCESS); - } - - public void dispatchRequest( - ContainerProtos.ContainerCommandRequestProto request, - ContainerProtos.Result expectedResult) { - ContainerProtos.ContainerCommandResponseProto response = - dsm.getContainer().getDispatcher().dispatch(request, null); - assertEquals(expectedResult, response.getResult()); - } - - /// VOLUME OPERATIONS /// - - /** - * Append a datanode volume to the existing volumes in the configuration. - * @return The root directory for the new volume. - */ - public File addHddsVolume() throws IOException { - - File vol = Files.createDirectory(tempFolder.resolve(UUID.randomUUID() - .toString())).toFile(); - String[] existingVolumes = - conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY); - List allVolumes = new ArrayList<>(); - if (existingVolumes != null) { - allVolumes.addAll(Arrays.asList(existingVolumes)); - } - - allVolumes.add(vol.getAbsolutePath()); - conf.setStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, - allVolumes.toArray(new String[0])); - - return vol; - } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java index 23b7da26346..fc599f7f913 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToSchemaV3.java @@ -20,7 +20,6 @@ import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; @@ -29,21 +28,17 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage; import org.apache.hadoop.ozone.container.common.SCMTestUtils; import org.apache.hadoop.ozone.container.common.ScmTestMock; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; -import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine; -import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask; -import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil; import org.apache.hadoop.ozone.container.common.volume.DbVolume; import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.StorageVolume; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; -import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -52,15 +47,10 @@ import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; -import java.nio.file.Files; import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Random; import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -91,8 +81,6 @@ public class TestDatanodeUpgradeToSchemaV3 { private RPC.Server scmRpcServer; private InetSocketAddress address; - private Random random; - private void initTests(Boolean enable) throws Exception { boolean schemaV3Enabled = enable; conf = new OzoneConfiguration(); @@ -106,8 +94,6 @@ private void initTests(Boolean enable) throws Exception { } private void setup() throws Exception { - random = new Random(); - address = SCMTestUtils.getReuseableAddress(); conf.setSocketAddr(ScmConfigKeys.OZONE_SCM_NAMES, address); conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, @@ -135,10 +121,12 @@ public void teardown() throws Exception { public void testDBOnHddsVolume(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); - addHddsVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); HddsVolume dataVolume = (HddsVolume) dsm.getContainer().getVolumeSet() .getVolumesList().get(0); assertNull(dataVolume.getDbVolume()); @@ -170,11 +158,13 @@ public void testDBOnHddsVolume(boolean schemaV3Enabled) throws Exception { public void testDBOnDbVolume(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); - addHddsVolume(); - addDbVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + UpgradeTestHelper.addDbVolume(conf, tempFolder); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); HddsVolume dataVolume = (HddsVolume) dsm.getContainer().getVolumeSet() .getVolumesList().get(0); assertNull(dataVolume.getDbParentDir()); @@ -209,9 +199,10 @@ public void testDBCreatedInFinalize(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); // add one HddsVolume - addHddsVolume(); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); // Set layout version. DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf, @@ -232,7 +223,7 @@ public void testDBCreatedInFinalize(boolean schemaV3Enabled) assertNull(dataVolume.getDbParentDir()); // Restart DN and finalize upgrade - restartDatanode( + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), true); dsm.finalizeUpgrade(); @@ -255,13 +246,15 @@ public void testDBCreatedInFinalize(boolean schemaV3Enabled) public void testFinalizeTwice(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); // add one HddsVolume and two DbVolume - addHddsVolume(); - addDbVolume(); - addDbVolume(); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + UpgradeTestHelper.addDbVolume(conf, tempFolder); + UpgradeTestHelper.addDbVolume(conf, tempFolder); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); dsm.finalizeUpgrade(); DbVolume dbVolume = ((HddsVolume) dsm.getContainer().getVolumeSet() @@ -283,15 +276,18 @@ public void testAddHddsVolumeAfterFinalize(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); - addHddsVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); dsm.finalizeUpgrade(); // Add a new HddsVolume. It should have DB created after DN restart. - addHddsVolume(); - restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, + HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), false); for (StorageVolume vol: dsm.getContainer().getVolumeSet().getVolumesList()) { @@ -314,10 +310,12 @@ public void testAddHddsVolumeAfterFinalize(boolean schemaV3Enabled) public void testAddDbVolumeAfterFinalize(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); - startScmServer(); - addHddsVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); HddsVolume hddsVolume = (HddsVolume) dsm.getContainer().getVolumeSet() .getVolumesList().get(0); assertNull(hddsVolume.getDbParentDir()); @@ -328,8 +326,9 @@ public void testAddDbVolumeAfterFinalize(boolean schemaV3Enabled) hddsVolume.getStorageDir().getAbsolutePath())); // Add a new DbVolume - addDbVolume(); - restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), + UpgradeTestHelper.addDbVolume(conf, tempFolder); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, + HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), false); // HddsVolume should still use the rocksDB under it's volume @@ -354,15 +353,18 @@ public void testAddDbAndHddsVolumeAfterFinalize(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); - addHddsVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); dsm.finalizeUpgrade(); - addDbVolume(); - File newDataVolume = addHddsVolume(); - restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), + UpgradeTestHelper.addDbVolume(conf, tempFolder); + File newDataVolume = UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, + HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), false); DbVolume dbVolume = (DbVolume) dsm.getContainer().getDbVolumeSet() @@ -419,18 +421,22 @@ public void testWriteWithV3Disabled(boolean schemaV3Enabled) public void testWrite(boolean enable, String expectedVersion) throws Exception { // start DN and SCM - startScmServer(); - addHddsVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); // Disable Schema V3 conf.setBoolean(DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED, false); - startPreFinalizedDatanode(); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); + ContainerDispatcher dispatcher = dsm.getContainer().getDispatcher(); dsm.finalizeUpgrade(); - final Pipeline pipeline = getPipeline(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); // Create a container to write data. - final long containerID1 = addContainer(pipeline); - putBlock(containerID1, pipeline); - closeContainer(containerID1, pipeline); + final long containerID1 = UpgradeTestHelper.addContainer(dispatcher, pipeline); + UpgradeTestHelper.putBlock(dispatcher, containerID1, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID1, pipeline); KeyValueContainer container = (KeyValueContainer) dsm.getContainer().getContainerSet().getContainer(containerID1); // When SchemaV3 is disabled, new data should be saved as SchemaV2. @@ -440,13 +446,15 @@ public void testWrite(boolean enable, String expectedVersion) // Set SchemaV3 enable status conf.setBoolean(DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED, enable); - restartDatanode(HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, + HDDSLayoutFeature.DATANODE_SCHEMA_V3.layoutVersion(), false); + dispatcher = dsm.getContainer().getDispatcher(); // Write new data - final long containerID2 = addContainer(pipeline); - putBlock(containerID2, pipeline); - closeContainer(containerID2, pipeline); + final long containerID2 = UpgradeTestHelper.addContainer(dispatcher, pipeline); + UpgradeTestHelper.putBlock(dispatcher, containerID2, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID2, pipeline); container = (KeyValueContainer) dsm.getContainer().getContainerSet().getContainer(containerID2); // If SchemaV3 is enabled, new data should be saved as SchemaV3 @@ -464,16 +472,20 @@ public void testReadsDuringFinalize(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); - addHddsVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); + ContainerDispatcher dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); // Add data to read. - final long containerID = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID, - pipeline); - closeContainer(containerID, pipeline); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk = + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline); // Create thread to keep reading during finalization. ExecutorService executor = Executors.newFixedThreadPool(1); @@ -481,10 +493,10 @@ public void testReadsDuringFinalize(boolean schemaV3Enabled) // Layout version check should be thread safe. while (!dsm.getLayoutVersionManager() .isAllowed(HDDSLayoutFeature.DATANODE_SCHEMA_V3)) { - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); } // Make sure we can read after finalizing too. - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); return null; }); @@ -502,8 +514,9 @@ public void testReadsDuringFinalize(boolean schemaV3Enabled) public void testFinalizeFailure(boolean schemaV3Enabled) throws Exception { initTests(schemaV3Enabled); // start DN and SCM - startScmServer(); - addHddsVolume(); + scmRpcServer = SCMTestUtils.startScmRpcServer(conf, + new ScmTestMock(CLUSTER_ID), address, 10); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); // Let HddsVolume be formatted to mimic the real cluster upgrade // Set layout version. DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf, @@ -523,15 +536,17 @@ public void testFinalizeFailure(boolean schemaV3Enabled) throws Exception { assertNull(dataVolume.getDbParentDir()); // Restart DN - restartDatanode( + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), true); + ContainerDispatcher dispatcher = dsm.getContainer().getDispatcher(); // Write some data. - final Pipeline pipeline = getPipeline(); - final long containerID = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID, - pipeline); - closeContainer(containerID, pipeline); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk = + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline); KeyValueContainer container = (KeyValueContainer) dsm.getContainer().getContainerSet().getContainer(containerID); assertEquals(OzoneConsts.SCHEMA_V2, @@ -558,227 +573,18 @@ public void testFinalizeFailure(boolean schemaV3Enabled) throws Exception { dsm.getContainer().getContainerSet().getContainer(containerID); assertEquals(OzoneConsts.SCHEMA_V2, container.getContainerData().getSchemaVersion()); - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); // SchemaV3 is not finalized, so still ERASURE_CODED_STORAGE_SUPPORT - restartDatanode( + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, false, tempFolder, address, HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), true); + dispatcher = dsm.getContainer().getDispatcher(); // Old data is readable after DN restart container = (KeyValueContainer) dsm.getContainer().getContainerSet().getContainer(containerID); assertEquals(OzoneConsts.SCHEMA_V2, container.getContainerData().getSchemaVersion()); - readChunk(writeChunk, pipeline); - } - - public void checkContainerPathID(long containerID, String expectedID) { - KeyValueContainerData data = - (KeyValueContainerData) dsm.getContainer().getContainerSet() - .getContainer(containerID).getContainerData(); - assertThat(data.getChunksPath()).contains(expectedID); - assertThat(data.getMetadataPath()).contains(expectedID); - } - - public List getHddsSubdirs(File volume) { - File[] subdirsArray = getHddsRoot(volume).listFiles(File::isDirectory); - assertNotNull(subdirsArray); - return Arrays.asList(subdirsArray); - } - - public File getHddsRoot(File volume) { - return new File(HddsVolumeUtil.getHddsRoot(volume.getAbsolutePath())); - } - - /** - * Starts the datanode with the fore layout version, and calls the version - * endpoint task to get cluster ID and SCM ID. - * - * The daemon for the datanode state machine is not started in this test. - * This greatly speeds up execution time. - * It means we do not have heartbeat functionality or pre-finalize - * upgrade actions, but neither of those things are needed for these tests. - */ - public void startPreFinalizedDatanode() throws Exception { - // Set layout version. - conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFolder.toString()); - DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf, - UUID.randomUUID().toString(), - HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion()); - layoutStorage.initialize(); - - // Build and start the datanode. - DatanodeDetails dd = ContainerTestUtils.createDatanodeDetails(); - DatanodeStateMachine newDsm = new DatanodeStateMachine(dd, conf); - int actualMlv = newDsm.getLayoutVersionManager().getMetadataLayoutVersion(); - assertEquals( - HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT.layoutVersion(), - actualMlv); - if (dsm != null) { - dsm.close(); - } - dsm = newDsm; - - callVersionEndpointTask(); - } - - public void restartDatanode(int expectedMlv, boolean exactMatch) - throws Exception { - // Stop existing datanode. - DatanodeDetails dd = dsm.getDatanodeDetails(); - dsm.close(); - - // Start new datanode with the same configuration. - dsm = new DatanodeStateMachine(dd, conf); - int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion(); - if (exactMatch) { - assertEquals(expectedMlv, mlv); - } else { - assertThat(expectedMlv).isLessThanOrEqualTo(mlv); - } - - callVersionEndpointTask(); - } - - /** - * Get the cluster ID and SCM ID from SCM to the datanode. - */ - public void callVersionEndpointTask() throws Exception { - try (EndpointStateMachine esm = ContainerTestUtils.createEndpoint(conf, - address, 1000)) { - VersionEndpointTask vet = new VersionEndpointTask(esm, conf, - dsm.getContainer()); - esm.setState(EndpointStateMachine.EndPointStates.GETVERSION); - vet.call(); - } - } - - public String startScmServer() throws IOException { - String scmID = UUID.randomUUID().toString(); - ScmTestMock scmServerImpl = new ScmTestMock(CLUSTER_ID, scmID); - scmRpcServer = SCMTestUtils.startScmRpcServer(conf, - scmServerImpl, address, 10); - return scmID; - } - - /// CONTAINER OPERATIONS /// - public void readChunk(ContainerProtos.WriteChunkRequestProto writeChunk, - Pipeline pipeline) throws Exception { - ContainerProtos.ContainerCommandRequestProto readChunkRequest = - ContainerTestHelper.getReadChunkRequest(pipeline, writeChunk); - - dispatchRequest(readChunkRequest); - } - - public ContainerProtos.WriteChunkRequestProto putBlock(long containerID, - Pipeline pipeline) throws Exception { - ContainerProtos.ContainerCommandRequestProto writeChunkRequest = - getWriteChunk(containerID, pipeline); - dispatchRequest(writeChunkRequest); - - ContainerProtos.ContainerCommandRequestProto putBlockRequest = - ContainerTestHelper.getPutBlockRequest(pipeline, - writeChunkRequest.getWriteChunk()); - dispatchRequest(putBlockRequest); - - return writeChunkRequest.getWriteChunk(); - } - - public ContainerProtos.ContainerCommandRequestProto getWriteChunk( - long containerID, Pipeline pipeline) throws Exception { - return ContainerTestHelper.getWriteChunkRequest(pipeline, - ContainerTestHelper.getTestBlockID(containerID), 100); - } - - public Pipeline getPipeline() { - return MockPipeline.createPipeline( - Collections.singletonList(dsm.getDatanodeDetails())); - } - - public long addContainer(Pipeline pipeline) - throws Exception { - long containerID = random.nextInt(Integer.MAX_VALUE); - ContainerProtos.ContainerCommandRequestProto createContainerRequest = - ContainerTestHelper.getCreateContainerRequest(containerID, pipeline); - dispatchRequest(createContainerRequest); - - return containerID; - } - - public void deleteContainer(long containerID, Pipeline pipeline) - throws Exception { - ContainerProtos.ContainerCommandRequestProto deleteContainerRequest = - ContainerTestHelper.getDeleteContainer(pipeline, containerID, true); - dispatchRequest(deleteContainerRequest); - } - - public void closeContainer(long containerID, Pipeline pipeline) - throws Exception { - closeContainer(containerID, pipeline, ContainerProtos.Result.SUCCESS); - } - - public void closeContainer(long containerID, Pipeline pipeline, - ContainerProtos.Result expectedResult) throws Exception { - ContainerProtos.ContainerCommandRequestProto closeContainerRequest = - ContainerTestHelper.getCloseContainer(pipeline, containerID); - dispatchRequest(closeContainerRequest, expectedResult); - } - - public void dispatchRequest( - ContainerProtos.ContainerCommandRequestProto request) { - dispatchRequest(request, ContainerProtos.Result.SUCCESS); - } - - public void dispatchRequest( - ContainerProtos.ContainerCommandRequestProto request, - ContainerProtos.Result expectedResult) { - ContainerProtos.ContainerCommandResponseProto response = - dsm.getContainer().getDispatcher().dispatch(request, null); - assertEquals(expectedResult, response.getResult()); - } - - /// VOLUME OPERATIONS /// - - /** - * Append a datanode volume to the existing volumes in the configuration. - * @return The root directory for the new volume. - */ - public File addHddsVolume() throws IOException { - - File vol = Files.createDirectory(tempFolder.resolve(UUID.randomUUID() - .toString())).toFile(); - String[] existingVolumes = - conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY); - List allVolumes = new ArrayList<>(); - if (existingVolumes != null) { - allVolumes.addAll(Arrays.asList(existingVolumes)); - } - - allVolumes.add(vol.getAbsolutePath()); - conf.setStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, - allVolumes.toArray(new String[0])); - - return vol; - } - - /** - * Append a db volume to the existing volumes in the configuration. - * @return The root directory for the new volume. - */ - public File addDbVolume() throws Exception { - File vol = Files.createDirectory(tempFolder.resolve(UUID.randomUUID() - .toString())).toFile(); - String[] existingVolumes = - conf.getStrings(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR); - List allVolumes = new ArrayList<>(); - if (existingVolumes != null) { - allVolumes.addAll(Arrays.asList(existingVolumes)); - } - - allVolumes.add(vol.getAbsolutePath()); - conf.setStrings(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR, - allVolumes.toArray(new String[0])); - - return vol; + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java index 59b88bcbea4..d4a27e74cda 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java @@ -18,9 +18,7 @@ package org.apache.hadoop.ozone.container.upgrade; -import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -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.ScmConfigKeys; @@ -28,17 +26,12 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ozone.container.ContainerTestHelper; -import org.apache.hadoop.ozone.container.common.ContainerTestUtils; -import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage; import org.apache.hadoop.ozone.container.common.SCMTestUtils; import org.apache.hadoop.ozone.container.common.ScmTestMock; 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.DatanodeStateMachine; -import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine; -import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask; import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil; -import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.ozone.container.replication.ContainerImporter; import org.apache.hadoop.ozone.container.replication.ContainerReplicationSource; @@ -55,11 +48,9 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Random; import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -81,6 +72,7 @@ public class TestDatanodeUpgradeToScmHA { private Path tempFolder; private DatanodeStateMachine dsm; + private ContainerDispatcher dispatcher; private OzoneConfiguration conf; private static final String CLUSTER_ID = "clusterID"; private boolean scmHAAlreadyEnabled; @@ -89,8 +81,6 @@ public class TestDatanodeUpgradeToScmHA { private InetSocketAddress address; private ScmTestMock scmServerImpl; - private Random random; - private void setScmHAEnabled(boolean enableSCMHA) throws Exception { this.scmHAAlreadyEnabled = enableSCMHA; @@ -100,8 +90,6 @@ private void setScmHAEnabled(boolean enableSCMHA) } private void setup() throws Exception { - random = new Random(); - address = SCMTestUtils.getReuseableAddress(); conf.setSocketAddr(ScmConfigKeys.OZONE_SCM_NAMES, address); } @@ -124,15 +112,18 @@ public void testReadsDuringFinalization(boolean enableSCMHA) setScmHAEnabled(enableSCMHA); // start DN and SCM startScmServer(); - addVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.INITIAL_VERSION.layoutVersion()); + dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); // Add data to read. - final long containerID = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID, - pipeline); - closeContainer(containerID, pipeline); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk = + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline); // Create thread to keep reading during finalization. ExecutorService executor = Executors.newFixedThreadPool(1); @@ -140,10 +131,10 @@ public void testReadsDuringFinalization(boolean enableSCMHA) // Layout version check should be thread safe. while (!dsm.getLayoutVersionManager() .isAllowed(HDDSLayoutFeature.SCM_HA)) { - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); } // Make sure we can read after finalizing too. - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); return null; }); @@ -159,40 +150,45 @@ public void testImportContainer(boolean enableSCMHA) throws Exception { setScmHAEnabled(enableSCMHA); // start DN and SCM startScmServer(); - addVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.INITIAL_VERSION.layoutVersion()); + dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); // Pre-export a container to continuously import and delete. - final long exportContainerID = addContainer(pipeline); + final long exportContainerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); ContainerProtos.WriteChunkRequestProto exportWriteChunk = - putBlock(exportContainerID, pipeline); - closeContainer(exportContainerID, pipeline); + UpgradeTestHelper.putBlock(dispatcher, exportContainerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, exportContainerID, pipeline); File exportedContainerFile = exportContainer(exportContainerID); - deleteContainer(exportContainerID, pipeline); + UpgradeTestHelper.deleteContainer(dispatcher, exportContainerID, pipeline); // Export another container to import while pre-finalized and read // finalized. - final long exportContainerID2 = addContainer(pipeline); + final long exportContainerID2 = UpgradeTestHelper.addContainer(dispatcher, pipeline); ContainerProtos.WriteChunkRequestProto exportWriteChunk2 = - putBlock(exportContainerID2, pipeline); - closeContainer(exportContainerID2, pipeline); + UpgradeTestHelper.putBlock(dispatcher, exportContainerID2, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, exportContainerID2, pipeline); File exportedContainerFile2 = exportContainer(exportContainerID2); - deleteContainer(exportContainerID2, pipeline); + UpgradeTestHelper.deleteContainer(dispatcher, exportContainerID2, pipeline); // Make sure we can import and read a container pre-finalized. importContainer(exportContainerID2, exportedContainerFile2); - readChunk(exportWriteChunk2, pipeline); + UpgradeTestHelper.readChunk(dispatcher, exportWriteChunk2, pipeline); // Now SCM and enough other DNs finalize to enable SCM HA. This DN is // restarted with SCM HA config and gets a different SCM ID. conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true); changeScmID(); - restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, true, tempFolder, address, + HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true); + dispatcher = dsm.getContainer().getDispatcher(); // Make sure the existing container can be read. - readChunk(exportWriteChunk2, pipeline); + UpgradeTestHelper.readChunk(dispatcher, exportWriteChunk2, pipeline); // Create thread to keep importing containers during the upgrade. // Since the datanode's MLV is behind SCM's, container creation is not @@ -204,12 +200,12 @@ public void testImportContainer(boolean enableSCMHA) throws Exception { while (!dsm.getLayoutVersionManager() .isAllowed(HDDSLayoutFeature.SCM_HA)) { importContainer(exportContainerID, exportedContainerFile); - readChunk(exportWriteChunk, pipeline); - deleteContainer(exportContainerID, pipeline); + UpgradeTestHelper.readChunk(dispatcher, exportWriteChunk, pipeline); + UpgradeTestHelper.deleteContainer(dispatcher, exportContainerID, pipeline); } // Make sure we can import after finalizing too. importContainer(exportContainerID, exportedContainerFile); - readChunk(exportWriteChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, exportWriteChunk, pipeline); return null; }); @@ -220,7 +216,7 @@ public void testImportContainer(boolean enableSCMHA) throws Exception { // Make sure we can read the container that was imported while // pre-finalized after finalizing. - readChunk(exportWriteChunk2, pipeline); + UpgradeTestHelper.readChunk(dispatcher, exportWriteChunk2, pipeline); } @ParameterizedTest(name = "{index}: scmHAAlreadyEnabled={0}") @@ -230,10 +226,14 @@ public void testFailedVolumeDuringFinalization(boolean enableSCMHA) setScmHAEnabled(enableSCMHA); /// SETUP /// - String originalScmID = startScmServer(); - File volume = addVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + startScmServer(); + String originalScmID = scmServerImpl.getScmId(); + File volume = UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.INITIAL_VERSION.layoutVersion()); + dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); /// PRE-FINALIZED: Write and Read from formatted volume /// @@ -243,10 +243,10 @@ public void testFailedVolumeDuringFinalization(boolean enableSCMHA) dsm.getContainer().getVolumeSet().getFailedVolumesList().size()); // Add container with data, make sure it can be read and written. - final long containerID = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID, - pipeline); - readChunk(writeChunk, pipeline); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk = + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); checkPreFinalizedVolumePathID(volume, originalScmID, CLUSTER_ID); checkContainerPathID(containerID, originalScmID, CLUSTER_ID); @@ -256,7 +256,7 @@ public void testFailedVolumeDuringFinalization(boolean enableSCMHA) failVolume(volume); // Since volume is failed, container should be marked unhealthy. // Finalization should proceed anyways. - closeContainer(containerID, pipeline, + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline, ContainerProtos.Result.CONTAINER_FILES_CREATE_ERROR); State containerState = dsm.getContainer().getContainerSet() .getContainer(containerID).getContainerState(); @@ -286,11 +286,13 @@ public void testFailedVolumeDuringFinalization(boolean enableSCMHA) // imported to it. // This should log a warning about reading from an unhealthy container // but otherwise proceed successfully. - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); /// FINALIZED: Restart datanode to upgrade the failed volume /// - restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, true, tempFolder, address, + HDDSLayoutFeature.SCM_HA.layoutVersion(), false); + dispatcher = dsm.getContainer().getDispatcher(); assertEquals(1, dsm.getContainer().getVolumeSet().getVolumesList().size()); @@ -301,12 +303,12 @@ public void testFailedVolumeDuringFinalization(boolean enableSCMHA) checkContainerPathID(containerID, originalScmID, CLUSTER_ID); // Read container from before upgrade. The upgrade required it to be closed. - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); // Write and read container after upgrade. - long newContainerID = addContainer(pipeline); + long newContainerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); ContainerProtos.WriteChunkRequestProto newWriteChunk = - putBlock(newContainerID, pipeline); - readChunk(newWriteChunk, pipeline); + UpgradeTestHelper.putBlock(dispatcher, newContainerID, pipeline); + UpgradeTestHelper.readChunk(dispatcher, newWriteChunk, pipeline); // The new container should use cluster ID in its path. // The volume it is placed on is up to the implementation. checkContainerPathID(newContainerID, CLUSTER_ID); @@ -318,10 +320,14 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { setScmHAEnabled(enableSCMHA); /// SETUP /// - String originalScmID = startScmServer(); - File preFinVolume1 = addVolume(); - startPreFinalizedDatanode(); - final Pipeline pipeline = getPipeline(); + startScmServer(); + String originalScmID = scmServerImpl.getScmId(); + File preFinVolume1 = UpgradeTestHelper.addHddsVolume(conf, tempFolder); + dsm = UpgradeTestHelper.startPreFinalizedDatanode(conf, tempFolder, dsm, address, + HDDSLayoutFeature.INITIAL_VERSION.layoutVersion()); + dispatcher = dsm.getContainer().getDispatcher(); + final Pipeline pipeline = MockPipeline.createPipeline( + Collections.singletonList(dsm.getDatanodeDetails())); /// PRE-FINALIZED: Write and Read from formatted volume /// @@ -331,10 +337,10 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { dsm.getContainer().getVolumeSet().getFailedVolumesList().size()); // Add container with data, make sure it can be read and written. - final long containerID = addContainer(pipeline); - ContainerProtos.WriteChunkRequestProto writeChunk = putBlock(containerID, - pipeline); - readChunk(writeChunk, pipeline); + final long containerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); + ContainerProtos.WriteChunkRequestProto writeChunk = + UpgradeTestHelper.putBlock(dispatcher, containerID, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); checkPreFinalizedVolumePathID(preFinVolume1, originalScmID, CLUSTER_ID); checkContainerPathID(containerID, originalScmID, CLUSTER_ID); @@ -346,9 +352,11 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true); changeScmID(); // A new volume is added that must be formatted. - File preFinVolume2 = addVolume(); + File preFinVolume2 = UpgradeTestHelper.addHddsVolume(conf, tempFolder); - restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, true, tempFolder, address, + HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true); + dispatcher = dsm.getContainer().getDispatcher(); assertEquals(2, dsm.getContainer().getVolumeSet().getVolumesList().size()); @@ -356,7 +364,7 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { dsm.getContainer().getVolumeSet().getFailedVolumesList().size()); // Because DN mlv would be behind SCM mlv, only reads are allowed. - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); // On restart, there should have been no changes to the paths already used. checkPreFinalizedVolumePathID(preFinVolume1, originalScmID, CLUSTER_ID); @@ -369,7 +377,7 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { /// FINALIZE /// - closeContainer(containerID, pipeline); + UpgradeTestHelper.closeContainer(dispatcher, containerID, pipeline); dsm.finalizeUpgrade(); LambdaTestUtils.await(2000, 500, () -> dsm.getLayoutVersionManager() @@ -379,11 +387,13 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { // Add a new volume that should be formatted with cluster ID only, since // DN has finalized. - File finVolume = addVolume(); + File finVolume = UpgradeTestHelper.addHddsVolume(conf, tempFolder); // Yet another SCM ID is received this time, but it should not matter. changeScmID(); - restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false); + dsm = UpgradeTestHelper.restartDatanode(conf, dsm, true, tempFolder, address, + HDDSLayoutFeature.SCM_HA.layoutVersion(), false); + dispatcher = dsm.getContainer().getDispatcher(); assertEquals(3, dsm.getContainer().getVolumeSet().getVolumesList().size()); @@ -400,12 +410,12 @@ public void testFormattingNewVolumes(boolean enableSCMHA) throws Exception { /// FINALIZED: Read old data and write + read new data /// // Read container from before upgrade. The upgrade required it to be closed. - readChunk(writeChunk, pipeline); + UpgradeTestHelper.readChunk(dispatcher, writeChunk, pipeline); // Write and read container after upgrade. - long newContainerID = addContainer(pipeline); + long newContainerID = UpgradeTestHelper.addContainer(dispatcher, pipeline); ContainerProtos.WriteChunkRequestProto newWriteChunk = - putBlock(newContainerID, pipeline); - readChunk(newWriteChunk, pipeline); + UpgradeTestHelper.putBlock(dispatcher, newContainerID, pipeline); + UpgradeTestHelper.readChunk(dispatcher, newWriteChunk, pipeline); // The new container should use cluster ID in its path. // The volume it is placed on is up to the implementation. checkContainerPathID(newContainerID, CLUSTER_ID); @@ -496,82 +506,18 @@ public File getHddsRoot(File volume) { /// CLUSTER OPERATIONS /// - /** - * Starts the datanode with the first layout version, and calls the version - * endpoint task to get cluster ID and SCM ID. - * - * The daemon for the datanode state machine is not started in this test. - * This greatly speeds up execution time. - * It means we do not have heartbeat functionality or pre-finalize - * upgrade actions, but neither of those things are needed for these tests. - */ - public void startPreFinalizedDatanode() throws Exception { - // Set layout version. - conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, - tempFolder.toString()); - DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf, - UUID.randomUUID().toString(), - HDDSLayoutFeature.INITIAL_VERSION.layoutVersion()); - layoutStorage.initialize(); - - // Build and start the datanode. - DatanodeDetails dd = ContainerTestUtils.createDatanodeDetails(); - DatanodeStateMachine newDsm = new DatanodeStateMachine(dd, conf); - int actualMlv = newDsm.getLayoutVersionManager().getMetadataLayoutVersion(); - assertEquals(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), - actualMlv); - dsm = newDsm; - - callVersionEndpointTask(); - } - - public void restartDatanode(int expectedMlv, boolean exactMatch) - throws Exception { - // Stop existing datanode. - DatanodeDetails dd = dsm.getDatanodeDetails(); - dsm.close(); - - // Start new datanode with the same configuration. - dsm = new DatanodeStateMachine(dd, conf); - StorageVolumeUtil.getHddsVolumesList(dsm.getContainer().getVolumeSet().getVolumesList()) - .forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile())); - int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion(); - if (exactMatch) { - assertEquals(expectedMlv, mlv); - } else { - assertThat(expectedMlv).isLessThanOrEqualTo(mlv); - } - - callVersionEndpointTask(); - } - - /** - * Get the cluster ID and SCM ID from SCM to the datanode. - */ - public void callVersionEndpointTask() throws Exception { - try (EndpointStateMachine esm = ContainerTestUtils.createEndpoint(conf, - address, 1000)) { - VersionEndpointTask vet = new VersionEndpointTask(esm, conf, - dsm.getContainer()); - esm.setState(EndpointStateMachine.EndPointStates.GETVERSION); - vet.call(); - } - } - - public String startScmServer() throws Exception { - String scmID = UUID.randomUUID().toString(); - scmServerImpl = new ScmTestMock(CLUSTER_ID, scmID); + private void startScmServer() throws Exception { + scmServerImpl = new ScmTestMock(CLUSTER_ID); scmRpcServer = SCMTestUtils.startScmRpcServer(conf, scmServerImpl, address, 10); - return scmID; } /** * Updates the SCM ID on the SCM server. Datanode will not be aware of this - * until {@link this#callVersionEndpointTask} is called. + * until {@link UpgradeTestHelper#callVersionEndpointTask} is called. * @return the new scm ID. */ - public String changeScmID() { + private String changeScmID() { String scmID = UUID.randomUUID().toString(); scmServerImpl.setScmId(scmID); return scmID; @@ -579,72 +525,10 @@ public String changeScmID() { /// CONTAINER OPERATIONS /// - public void readChunk(ContainerProtos.WriteChunkRequestProto writeChunk, - Pipeline pipeline) throws Exception { - ContainerProtos.ContainerCommandRequestProto readChunkRequest = - ContainerTestHelper.getReadChunkRequest(pipeline, writeChunk); - - dispatchRequest(readChunkRequest); - } - - public ContainerProtos.WriteChunkRequestProto putBlock(long containerID, - Pipeline pipeline) throws Exception { - ContainerProtos.ContainerCommandRequestProto writeChunkRequest = - getWriteChunk(containerID, pipeline); - dispatchRequest(writeChunkRequest); - - ContainerProtos.ContainerCommandRequestProto putBlockRequest = - ContainerTestHelper.getPutBlockRequest(pipeline, - writeChunkRequest.getWriteChunk()); - dispatchRequest(putBlockRequest); - - return writeChunkRequest.getWriteChunk(); - } - - public ContainerProtos.ContainerCommandRequestProto getWriteChunk( - long containerID, Pipeline pipeline) throws Exception { - return ContainerTestHelper.getWriteChunkRequest(pipeline, - ContainerTestHelper.getTestBlockID(containerID), 100); - } - - public Pipeline getPipeline() { - return MockPipeline.createPipeline( - Collections.singletonList(dsm.getDatanodeDetails())); - } - - public long addContainer(Pipeline pipeline) - throws Exception { - long containerID = random.nextInt(Integer.MAX_VALUE); - ContainerProtos.ContainerCommandRequestProto createContainerRequest = - ContainerTestHelper.getCreateContainerRequest(containerID, pipeline); - dispatchRequest(createContainerRequest); - - return containerID; - } - - public void deleteContainer(long containerID, Pipeline pipeline) - throws Exception { - ContainerProtos.ContainerCommandRequestProto deleteContainerRequest = - ContainerTestHelper.getDeleteContainer(pipeline, containerID, true); - dispatchRequest(deleteContainerRequest); - } - - public void closeContainer(long containerID, Pipeline pipeline) - throws Exception { - closeContainer(containerID, pipeline, ContainerProtos.Result.SUCCESS); - } - - public void closeContainer(long containerID, Pipeline pipeline, - ContainerProtos.Result expectedResult) throws Exception { - ContainerProtos.ContainerCommandRequestProto closeContainerRequest = - ContainerTestHelper.getCloseContainer(pipeline, containerID); - dispatchRequest(closeContainerRequest, expectedResult); - } - /** * Exports the specified container to a temporary file and returns the file. */ - public File exportContainer(long containerId) throws Exception { + private File exportContainer(long containerId) throws Exception { final ContainerReplicationSource replicationSource = new OnDemandContainerReplicationSource( dsm.getContainer().getController()); @@ -663,7 +547,7 @@ public File exportContainer(long containerId) throws Exception { * Imports the container found in {@code source} to the datanode with the ID * {@code containerID}. */ - public void importContainer(long containerID, File source) throws Exception { + private void importContainer(long containerID, File source) throws Exception { ContainerImporter replicator = new ContainerImporter(dsm.getConf(), dsm.getContainer().getContainerSet(), @@ -679,43 +563,8 @@ public void importContainer(long containerID, File source) throws Exception { NO_COMPRESSION); } - public void dispatchRequest( - ContainerProtos.ContainerCommandRequestProto request) { - dispatchRequest(request, ContainerProtos.Result.SUCCESS); - } - - public void dispatchRequest( - ContainerProtos.ContainerCommandRequestProto request, - ContainerProtos.Result expectedResult) { - ContainerProtos.ContainerCommandResponseProto response = - dsm.getContainer().getDispatcher().dispatch(request, null); - assertEquals(expectedResult, response.getResult()); - } - /// VOLUME OPERATIONS /// - /** - * Append a datanode volume to the existing volumes in the configuration. - * @return The root directory for the new volume. - */ - public File addVolume() throws Exception { - - File vol = Files.createDirectory( - tempFolder.resolve(UUID.randomUUID().toString())).toFile(); - String[] existingVolumes = - conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY); - List allVolumes = new ArrayList<>(); - if (existingVolumes != null) { - allVolumes.addAll(Arrays.asList(existingVolumes)); - } - - allVolumes.add(vol.getAbsolutePath()); - conf.setStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, - allVolumes.toArray(new String[0])); - - return vol; - } - /** * Renames the specified volume directory so it will appear as failed to * the datanode. diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/UpgradeTestHelper.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/UpgradeTestHelper.java new file mode 100644 index 00000000000..28b9163f3cf --- /dev/null +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/UpgradeTestHelper.java @@ -0,0 +1,271 @@ +/* + * 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.upgrade; + +import org.apache.hadoop.hdds.HddsConfigKeys; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.common.ContainerTestUtils; +import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; +import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine; +import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask; +import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Helpers for upgrade tests. + */ +public final class UpgradeTestHelper { + private UpgradeTestHelper() { + } + private static final Random RANDOM = new Random(); + + /** + * Starts the datanode with the fore layout version, and calls the version + * endpoint task to get cluster ID and SCM ID. + * + * The daemon for the datanode state machine is not started in this test. + * This greatly speeds up execution time. + * It means we do not have heartbeat functionality or pre-finalize + * upgrade actions, but neither of those things are needed for these tests. + */ + public static DatanodeStateMachine startPreFinalizedDatanode( + OzoneConfiguration conf, Path tempFolder, + DatanodeStateMachine dsm, InetSocketAddress address, + int metadataLayoutVersion) + throws Exception { + // Set layout version. + conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFolder.toString()); + DatanodeLayoutStorage layoutStorage = new DatanodeLayoutStorage(conf, + UUID.randomUUID().toString(), + metadataLayoutVersion); + layoutStorage.initialize(); + if (dsm != null) { + dsm.close(); + } + + // Build and start the datanode. + DatanodeDetails dd = ContainerTestUtils.createDatanodeDetails(); + dsm = new DatanodeStateMachine(dd, conf); + int actualMlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion(); + assertEquals( + metadataLayoutVersion, + actualMlv); + + + callVersionEndpointTask(conf, dsm.getContainer(), address); + return dsm; + } + + public static DatanodeStateMachine restartDatanode( + OzoneConfiguration conf, DatanodeStateMachine dsm, boolean shouldSetDbParentDir, + Path tempFolder, InetSocketAddress address, int expectedMlv, boolean exactMatch) + throws Exception { + // Stop existing datanode. + DatanodeDetails dd = dsm.getDatanodeDetails(); + dsm.close(); + + // Start new datanode with the same configuration. + dsm = new DatanodeStateMachine(dd, conf); + if (shouldSetDbParentDir) { + StorageVolumeUtil.getHddsVolumesList(dsm.getContainer().getVolumeSet().getVolumesList()) + .forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile())); + } + int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion(); + if (exactMatch) { + assertEquals(expectedMlv, mlv); + } else { + assertThat(expectedMlv).isLessThanOrEqualTo(mlv); + } + + callVersionEndpointTask(conf, dsm.getContainer(), address); + return dsm; + } + + /** + * Get the cluster ID and SCM ID from SCM to the datanode. + */ + public static void callVersionEndpointTask( + OzoneConfiguration conf, OzoneContainer container, InetSocketAddress address) + throws Exception { + try (EndpointStateMachine esm = ContainerTestUtils.createEndpoint(conf, + address, 1000)) { + VersionEndpointTask vet = new VersionEndpointTask(esm, conf, + container); + esm.setState(EndpointStateMachine.EndPointStates.GETVERSION); + vet.call(); + } + } + + /** + * Append a datanode volume to the existing volumes in the configuration. + * @return The root directory for the new volume. + */ + public static File addHddsVolume(OzoneConfiguration conf, Path tempFolder) throws IOException { + + File vol = Files.createDirectory(tempFolder.resolve(UUID.randomUUID() + .toString())).toFile(); + String[] existingVolumes = + conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY); + List allVolumes = new ArrayList<>(); + if (existingVolumes != null) { + allVolumes.addAll(Arrays.asList(existingVolumes)); + } + + allVolumes.add(vol.getAbsolutePath()); + conf.setStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, + allVolumes.toArray(new String[0])); + + return vol; + } + + /** + * Append a db volume to the existing volumes in the configuration. + * @return The root directory for the new volume. + */ + public static File addDbVolume(OzoneConfiguration conf, Path tempFolder) throws Exception { + File vol = Files.createDirectory(tempFolder.resolve(UUID.randomUUID() + .toString())).toFile(); + String[] existingVolumes = + conf.getStrings(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR); + List allVolumes = new ArrayList<>(); + if (existingVolumes != null) { + allVolumes.addAll(Arrays.asList(existingVolumes)); + } + + allVolumes.add(vol.getAbsolutePath()); + conf.setStrings(OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR, + allVolumes.toArray(new String[0])); + + return vol; + } + + + public static void dispatchRequest( + ContainerDispatcher dispatcher, + ContainerProtos.ContainerCommandRequestProto request) { + dispatchRequest(dispatcher, request, ContainerProtos.Result.SUCCESS); + } + + public static void dispatchRequest( + ContainerDispatcher dispatcher, ContainerProtos.ContainerCommandRequestProto request, + ContainerProtos.Result expectedResult) { + ContainerProtos.ContainerCommandResponseProto response = + dispatcher.dispatch(request, null); + assertEquals(expectedResult, response.getResult()); + } + + public static void readChunk( + ContainerDispatcher dispatcher, ContainerProtos.WriteChunkRequestProto writeChunk, + Pipeline pipeline) throws Exception { + ContainerProtos.ContainerCommandRequestProto readChunkRequest = + ContainerTestHelper.getReadChunkRequest(pipeline, writeChunk); + dispatchRequest(dispatcher, readChunkRequest); + } + + public static ContainerProtos.WriteChunkRequestProto putBlock( + ContainerDispatcher dispatcher, long containerID, Pipeline pipeline, + boolean incremental) throws Exception { + return putBlock(dispatcher, containerID, pipeline, incremental, ContainerProtos.Result.SUCCESS); + } + + public static ContainerProtos.WriteChunkRequestProto putBlock( + ContainerDispatcher dispatcher, long containerID, Pipeline pipeline) throws Exception { + return putBlock(dispatcher, containerID, pipeline, false, ContainerProtos.Result.SUCCESS); + } + + public static ContainerProtos.WriteChunkRequestProto putBlock( + ContainerDispatcher dispatcher, long containerID, Pipeline pipeline, + boolean incremental, ContainerProtos.Result expectedResult) throws Exception { + ContainerProtos.ContainerCommandRequestProto writeChunkRequest = + ContainerTestHelper.getWriteChunkRequest(pipeline, + ContainerTestHelper.getTestBlockID(containerID), 100); + dispatchRequest(dispatcher, writeChunkRequest); + + ContainerProtos.ContainerCommandRequestProto putBlockRequest = + ContainerTestHelper.getPutBlockRequest(pipeline, + writeChunkRequest.getWriteChunk(), incremental); + dispatchRequest(dispatcher, putBlockRequest, expectedResult); + return writeChunkRequest.getWriteChunk(); + } + + public static long addContainer(ContainerDispatcher dispatcher, Pipeline pipeline) + throws Exception { + long containerID = RANDOM.nextInt(Integer.MAX_VALUE); + ContainerProtos.ContainerCommandRequestProto createContainerRequest = + ContainerTestHelper.getCreateContainerRequest(containerID, pipeline); + dispatchRequest(dispatcher, createContainerRequest); + + return containerID; + } + + public static void deleteContainer(ContainerDispatcher dispatcher, long containerID, Pipeline pipeline) + throws Exception { + ContainerProtos.ContainerCommandRequestProto deleteContainerRequest = + ContainerTestHelper.getDeleteContainer(pipeline, containerID, true); + dispatchRequest(dispatcher, deleteContainerRequest); + } + + public static void closeContainer(ContainerDispatcher dispatcher, long containerID, Pipeline pipeline) + throws Exception { + closeContainer(dispatcher, containerID, pipeline, ContainerProtos.Result.SUCCESS); + } + + public static void closeContainer( + ContainerDispatcher dispatcher, long containerID, Pipeline pipeline, + ContainerProtos.Result expectedResult) throws Exception { + ContainerProtos.ContainerCommandRequestProto closeContainerRequest = + ContainerTestHelper.getCloseContainer(pipeline, containerID); + dispatchRequest(dispatcher, closeContainerRequest, expectedResult); + } + + public static void finalizeBlock( + ContainerDispatcher dispatcher, long containerID, long localID, ContainerProtos.Result expectedResult) { + ContainerInfo container = mock(ContainerInfo.class); + when(container.getContainerID()).thenReturn(containerID); + + ContainerProtos.ContainerCommandRequestProto finalizeBlockRequest = + ContainerTestHelper.getFinalizeBlockRequest(localID, container, UUID.randomUUID().toString()); + + UpgradeTestHelper.dispatchRequest(dispatcher, finalizeBlockRequest, expectedResult); + } +} diff --git a/hadoop-hdds/docs/content/interface/ReconApi.md b/hadoop-hdds/docs/content/interface/ReconApi.md index 485c3b0fd42..b5b885b403e 100644 --- a/hadoop-hdds/docs/content/interface/ReconApi.md +++ b/hadoop-hdds/docs/content/interface/ReconApi.md @@ -23,8 +23,9 @@ summary: Recon server supports HTTP endpoints to help troubleshoot and monitor O limitations under the License. --> -The Recon API v1 is a set of HTTP endpoints that help you understand the current -state of an Ozone cluster and to troubleshoot if needed. +The Recon API v1 offers a collection of HTTP endpoints designed to provide insights into the current state of an Ozone cluster, +facilitating monitoring, management, and troubleshooting. These endpoints allow administrators to access critical cluster +metadata, container status, key management, and more. Endpoints that are marked as *admin only* can only be accessed by Kerberos users specified in the **ozone.administrators** or **ozone.recon.administrators** @@ -37,7 +38,7 @@ ozone.security.enabled| *true* ozone.security.http.kerberos.enabled| *true* ozone.acl.enabled| *true* -Checkout an interactive version of the API powered by Swagger [here]({{< relref "./SwaggerReconApi.md" >}}) +Access an interactive version of the API, complete with detailed descriptions and example requests, powered by Swagger [here]({{< relref "./SwaggerReconApi.md" >}}) ## Containers (admin only) @@ -1062,6 +1063,28 @@ response object being the upper cap for file size range. "count": 2 }] ``` + +### GET /api/v1/utilization/containerCount + +**Parameters** + +* containerSize (optional) + + Filters the results based on the given container size. The smallest container size being tracked for count is 512 MB (512000000 bytes). + +**Returns** + +Returns the container counts within different container size ranges, with `containerSize` representing the size range and `count` representing the number of containers within that range. + +```json + [{ + "containerSize": 2147483648, + "count": 9 + }, { + "containerSize": 1073741824, + "count": 3 + }] +``` ## Metrics diff --git a/hadoop-hdds/erasurecode/pom.xml b/hadoop-hdds/erasurecode/pom.xml index cd0afbca7f8..201336d5ed3 100644 --- a/hadoop-hdds/erasurecode/pom.xml +++ b/hadoop-hdds/erasurecode/pom.xml @@ -38,6 +38,16 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-common + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + org.apache.ozone @@ -49,6 +59,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-test-utils test + + org.apache.ozone + hdds-config + test + diff --git a/hadoop-hdds/framework/pom.xml b/hadoop-hdds/framework/pom.xml index 5eb3d2071b7..70cce849aec 100644 --- a/hadoop-hdds/framework/pom.xml +++ b/hadoop-hdds/framework/pom.xml @@ -33,6 +33,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-interface-client + org.apache.ozone hdds-interface-server @@ -57,6 +65,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ch.qos.reload4j reload4j + + org.slf4j + slf4j-api + org.slf4j slf4j-reload4j @@ -69,11 +81,32 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.commons commons-configuration2 + + org.apache.commons + commons-lang3 + + + commons-codec + commons-codec + commons-fileupload commons-fileupload ${commons-fileupload.version} + + commons-io + commons-io + + + commons-logging + commons-logging + + + commons-validator + commons-validator + + org.apache.logging.log4j log4j-api @@ -88,6 +121,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> disruptor runtime + + org.bouncycastle + bcpkix-jdk18on + + + org.bouncycastle + bcprov-jdk18on + + + org.eclipse.jetty + jetty-http + org.eclipse.jetty jetty-util @@ -104,11 +149,20 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.eclipse.jetty jetty-webapp + + org.glassfish.jersey.core + jersey-server + org.glassfish.jersey.containers jersey-container-servlet-core + org.rocksdb + rocksdbjni + + + ratis-server org.apache.ratis @@ -122,6 +176,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-metrics-api + + + org.apache.ratis + ratis-server-api + ratis-metrics-dropwizard3 org.apache.ratis @@ -132,6 +198,23 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ratis + ratis-thirdparty-misc + + + + io.dropwizard.metrics + metrics-core + + + io.opentracing + opentracing-api + + + io.prometheus + simpleclient + io.prometheus simpleclient_dropwizard @@ -140,14 +223,39 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> io.prometheus simpleclient_common + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + com.fasterxml.jackson.datatype jackson-datatype-jsr310 + + com.github.jnr + jnr-constants + com.github.jnr jnr-posix + + com.google.code.gson + gson + + + com.google.guava + guava + + + com.google.protobuf + protobuf-java + + org.apache.hadoop hadoop-hdfs-client @@ -165,6 +273,19 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ${hdds.version} + + jakarta.annotation + jakarta.annotation-api + + + jakarta.ws.rs + jakarta.ws.rs-api + + + javax.servlet + javax.servlet-api + + org.apache.ozone @@ -182,6 +303,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-test-utils test + + org.eclipse.jetty + jetty-util-ajax + test + diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java index e9f7c4465dd..7ce895760ad 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java @@ -23,10 +23,10 @@ import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto; import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.security.SecurityConfig; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; import org.apache.hadoop.hdds.security.x509.exception.CertificateException; import org.apache.hadoop.security.UserGroupInformation; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,7 +35,6 @@ import java.security.KeyPair; import java.util.function.Consumer; -import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest.getEncodedString; import static org.apache.hadoop.hdds.security.x509.exception.CertificateException.ErrorCode.CSR_ERROR; /** @@ -70,9 +69,9 @@ public DNCertificateClient( * @return CertificateSignRequest.Builder */ @Override - public CertificateSignRequest.Builder getCSRBuilder() - throws CertificateException { - CertificateSignRequest.Builder builder = super.getCSRBuilder(); + public CertificateSignRequest.Builder configureCSRBuilder() + throws SCMSecurityException { + CertificateSignRequest.Builder builder = super.configureCSRBuilder(); try { String hostname = InetAddress.getLocalHost().getCanonicalHostName(); @@ -93,10 +92,8 @@ public CertificateSignRequest.Builder getCSRBuilder() } @Override - public SCMGetCertResponseProto getCertificateSignResponse( - PKCS10CertificationRequest csr) throws IOException { - return getScmSecureClient().getDataNodeCertificateChain( - dn.getProtoBufMessage(), getEncodedString(csr)); + public SCMGetCertResponseProto sign(CertificateSignRequest csr) throws IOException { + return getScmSecureClient().getDataNodeCertificateChain(dn.getProtoBufMessage(), csr.toEncodedFormat()); } @Override diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java index 2fb258e1a29..70a475982bd 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java @@ -76,6 +76,7 @@ import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.ssl.ReloadingX509KeyManager; import org.apache.hadoop.hdds.security.ssl.ReloadingX509TrustManager; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.authority.CAType; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; @@ -99,7 +100,6 @@ import static org.apache.hadoop.hdds.security.x509.exception.CertificateException.ErrorCode.RENEW_ERROR; import static org.apache.hadoop.hdds.security.x509.exception.CertificateException.ErrorCode.ROLLBACK_ERROR; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.slf4j.Logger; /** @@ -567,15 +567,12 @@ private boolean verifySignature(byte[] data, byte[] signature, * @return CertificateSignRequest.Builder */ @Override - public CertificateSignRequest.Builder getCSRBuilder() - throws CertificateException { - CertificateSignRequest.Builder builder = - new CertificateSignRequest.Builder() - .setConfiguration(securityConfig) - .addInetAddresses() - .setDigitalEncryption(true) - .setDigitalSignature(true); - return builder; + public CertificateSignRequest.Builder configureCSRBuilder() throws SCMSecurityException { + return new CertificateSignRequest.Builder() + .setConfiguration(securityConfig) + .addInetAddresses() + .setDigitalEncryption(true) + .setDigitalSignature(true); } /** @@ -805,7 +802,8 @@ protected void recoverStateIfNeeded(InitResponse state) throws IOException { getLogger().info("Initialization successful, case:{}.", state); break; case GETCERT: - String certId = signAndStoreCertificate(getCSRBuilder().build()); + Path certLocation = securityConfig.getCertificateLocation(getComponentName()); + String certId = signAndStoreCertificate(configureCSRBuilder().build(), certLocation, false); if (certIdSaveCallback != null) { certIdSaveCallback.accept(certId); } else { @@ -1152,7 +1150,7 @@ public String renewAndStoreKeyAndCertificate(boolean force) // Get certificate signed String newCertSerialId; try { - CertificateSignRequest.Builder csrBuilder = getCSRBuilder(); + CertificateSignRequest.Builder csrBuilder = configureCSRBuilder(); csrBuilder.setKey(newKeyPair); newCertSerialId = signAndStoreCertificate(csrBuilder.build(), Paths.get(newCertPath), true); @@ -1320,20 +1318,12 @@ private synchronized String updateCertSerialId(String newCertSerialId) { return certSerialId; } - protected String signAndStoreCertificate( - PKCS10CertificationRequest request, Path certificatePath) - throws CertificateException { - return signAndStoreCertificate(request, certificatePath, false); - } - - protected abstract SCMGetCertResponseProto getCertificateSignResponse( - PKCS10CertificationRequest request) throws IOException; + protected abstract SCMGetCertResponseProto sign(CertificateSignRequest request) throws IOException; - protected String signAndStoreCertificate( - PKCS10CertificationRequest request, Path certificatePath, boolean renew) + protected String signAndStoreCertificate(CertificateSignRequest csr, Path certificatePath, boolean renew) throws CertificateException { try { - SCMGetCertResponseProto response = getCertificateSignResponse(request); + SCMGetCertResponseProto response = sign(csr); // Persist certificates. if (response.hasX509CACertificate()) { @@ -1371,12 +1361,6 @@ private void getAndStoreAllRootCAs(CertificateCodec certCodec, boolean renew) } } - public String signAndStoreCertificate( - PKCS10CertificationRequest request) throws CertificateException { - return updateCertSerialId(signAndStoreCertificate(request, - securityConfig.getCertificateLocation(getComponentName()))); - } - public SCMSecurityProtocolClientSideTranslatorPB getScmSecureClient() { return scmSecurityClient; } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/SCMCertificateClient.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/SCMCertificateClient.java index 8bad4f18ade..ae0c0f0db84 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/SCMCertificateClient.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/SCMCertificateClient.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.authority.CAType; import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer; import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateStore; @@ -32,10 +33,8 @@ import org.apache.hadoop.hdds.security.x509.certificate.authority.profile.PKIProfile; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; -import org.apache.hadoop.hdds.security.x509.exception.CertificateException; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneSecurityUtil; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +55,6 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM; import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED; -import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest.getEncodedString; import static org.apache.hadoop.ozone.OzoneConsts.SCM_ROOT_CA_COMPONENT_NAME; import static org.apache.hadoop.ozone.OzoneConsts.SCM_ROOT_CA_PREFIX; import static org.apache.hadoop.ozone.OzoneConsts.SCM_SUB_CA_PREFIX; @@ -137,14 +135,14 @@ public SCMCertificateClient( * * @return CertificateSignRequest.Builder */ - public CertificateSignRequest.Builder getCSRBuilder() - throws CertificateException { + public CertificateSignRequest.Builder configureCSRBuilder() + throws SCMSecurityException { String subject = SCM_SUB_CA_PREFIX + scmHostname; LOG.info("Creating csr for SCM->hostName:{},scmId:{},clusterId:{}," + "subject:{}", scmHostname, scmId, cId, subject); - return super.getCSRBuilder() + return super.configureCSRBuilder() .setSubject(subject) .setScmID(scmId) .setClusterID(cId) @@ -164,15 +162,13 @@ public Logger getLogger() { } @Override - protected SCMGetCertResponseProto getCertificateSignResponse( - PKCS10CertificationRequest request) { - throw new UnsupportedOperationException("getCertSignResponse of " + - " SCMCertificateClient is not supported currently"); + protected SCMGetCertResponseProto sign(CertificateSignRequest request) { + throw new UnsupportedOperationException("Invalid call to SCMCertificateClient#sign(CertificateSignRequest. " + + "SCM certificate client uses a different mechanism to sign the SCMs' certificate."); } @Override - public String signAndStoreCertificate(PKCS10CertificationRequest request, - Path certPath, boolean renew) throws CertificateException { + public String signAndStoreCertificate(CertificateSignRequest csr, Path certPath, boolean renew) { try { HddsProtos.ScmNodeDetailsProto scmNodeDetailsProto = HddsProtos.ScmNodeDetailsProto.newBuilder() @@ -182,8 +178,7 @@ public String signAndStoreCertificate(PKCS10CertificationRequest request, // Get SCM sub CA cert. SCMGetCertResponseProto response = - getScmSecureClient().getSCMCertChain(scmNodeDetailsProto, - getEncodedString(request), true); + getScmSecureClient().getSCMCertChain(scmNodeDetailsProto, csr.toEncodedFormat(), true); CertificateCodec certCodec = new CertificateCodec( getSecurityConfig(), certPath); @@ -310,7 +305,7 @@ protected void recoverStateIfNeeded(InitResponse state) throws IOException { private void getRootCASignedSCMCert() { try { // Generate CSR. - PKCS10CertificationRequest csr = getCSRBuilder().build(); + CertificateSignRequest csr = configureCSRBuilder().build(); HddsProtos.ScmNodeDetailsProto scmNodeDetailsProto = HddsProtos.ScmNodeDetailsProto.newBuilder() .setClusterId(cId) @@ -319,7 +314,7 @@ private void getRootCASignedSCMCert() { // Get SCM sub CA cert. SCMGetCertResponseProto response = getScmSecureClient(). - getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr), false); + getSCMCertChain(scmNodeDetailsProto, csr.toEncodedFormat(), false); String pemEncodedCert = response.getX509Certificate(); // Store SCM sub CA and root CA certificate. @@ -357,9 +352,11 @@ private void getPrimarySCMSelfSignedCert() { String pemEncodedRootCert = CertificateCodec.getPEMEncodedString(rootCACertificatePath); - PKCS10CertificationRequest csr = getCSRBuilder().build(); + CertificateSignRequest csr = configureCSRBuilder().build(); String subCaSerialId = BigInteger.ONE.add(BigInteger.ONE).toString(); - CertPath scmSubCACertPath = rootCAServer.requestCertificate(csr, KERBEROS_TRUSTED, SCM, subCaSerialId).get(); + //TODO: do not use generateCSR() here once the server side change is also done. + CertPath scmSubCACertPath = + rootCAServer.requestCertificate(csr.generateCSR(), KERBEROS_TRUSTED, SCM, subCaSerialId).get(); String pemEncodedCert = CertificateCodec.getPEMEncodedString(scmSubCACertPath); storeCertificate(pemEncodedRootCert, CAType.SUBORDINATE); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStore.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStore.java index 19855625237..3e8ea30a652 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStore.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStore.java @@ -167,6 +167,14 @@ void move(KEY sourceKey, KEY destKey, VALUE value, */ DBCheckpoint getCheckpoint(boolean flush) throws IOException; + /** + * Get current snapshot of DB store as an artifact stored on + * the local filesystem with different parent path. + * @return An object that encapsulates the checkpoint information along with + * location. + */ + DBCheckpoint getCheckpoint(String parentDir, boolean flush) throws IOException; + /** * Get DB Store location. * @return DB file location. diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java index d5aa961b0e9..99924f724d5 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java @@ -338,6 +338,14 @@ public DBCheckpoint getCheckpoint(boolean flush) throws IOException { return checkPointManager.createCheckpoint(checkpointsParentDir); } + @Override + public DBCheckpoint getCheckpoint(String parentPath, boolean flush) throws IOException { + if (flush) { + this.flushDB(); + } + return checkPointManager.createCheckpoint(parentPath, null); + } + public DBCheckpoint getSnapshot(String name) throws IOException { this.flushLog(true); return checkPointManager.createCheckpoint(snapshotsParentDir, name); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java index 042887e4e53..5e8996df172 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java @@ -168,12 +168,20 @@ public void appendOpLatencyNanos(long nanos) { /** * Appends pre-operation operation latency in milliseconds. - * @param millis Latency in nanoseconds. + * @param millis Latency in milliseconds. */ public void appendPreOpLatencyMs(long millis) { append("preOpLatencyMs", millis); } + /** + * Appends pre-operation operation latency in milliseconds. + * @param nanos Latency in nanoseconds. + */ + public void appendPreOpLatencyNano(long nanos) { + append("preOpLatencyMs", TimeUnit.NANOSECONDS.toMillis(nanos)); + } + /** * Appends whole operation latency in milliseconds. * @param millis Latency in milliseconds. diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java index d37d22118da..8dec4f4fc3d 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java @@ -24,7 +24,8 @@ public enum AuditLoggerType { DNLOGGER("DNAudit"), OMLOGGER("OMAudit"), SCMLOGGER("SCMAudit"), - S3GLOGGER("S3GAudit"); + S3GLOGGER("S3GAudit"), + OMSYSTEMLOGGER("OMSystemAudit"); private String type; diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java index 56f84b9e3be..e029006a6af 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java @@ -153,6 +153,7 @@ public void testRequestCertificate() throws Exception { String clusterId = RandomStringUtils.randomAlphabetic(4); KeyPair keyPair = new HDDSKeyGenerator(securityConfig).generateKey(); + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -163,7 +164,8 @@ public void testRequestCertificate() throws Exception { .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); CertificateServer testCA = new DefaultCAServer("testCA", clusterId, scmId, caStore, @@ -204,6 +206,7 @@ public void testRequestCertificate() throws Exception { public void testRequestCertificateWithInvalidSubject() throws Exception { KeyPair keyPair = new HDDSKeyGenerator(securityConfig).generateKey(); + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -211,7 +214,8 @@ public void testRequestCertificateWithInvalidSubject() throws Exception { .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); CertificateServer testCA = new DefaultCAServer("testCA", RandomStringUtils.randomAlphabetic(4), @@ -232,6 +236,7 @@ public void testRequestCertificateWithInvalidSubject() throws Exception { public void testRequestCertificateWithInvalidSubjectFailure() throws Exception { KeyPair keyPair = new HDDSKeyGenerator(securityConfig).generateKey(); + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -241,7 +246,8 @@ public void testRequestCertificateWithInvalidSubjectFailure() throws Exception { .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); CertificateServer testCA = new DefaultCAServer("testCA", RandomStringUtils.randomAlphabetic(4), @@ -344,6 +350,7 @@ public void testInitWithCertChain(@TempDir Path tempDir) throws Exception { LocalDate beginDate = LocalDate.now().atStartOfDay().toLocalDate(); LocalDate endDate = LocalDate.from(LocalDate.now().atStartOfDay().plusDays(10)); + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -354,7 +361,8 @@ public void testInitWithCertChain(@TempDir Path tempDir) throws Exception { .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); X509Certificate externalCert = generateExternalCert(keyPair); X509Certificate signedCert = approver.sign(securityConfig, keyPair.getPrivate(), externalCert, @@ -405,6 +413,7 @@ clusterId, scmId, caStore, new DefaultProfile(), // Generate cert KeyPair keyPair = new HDDSKeyGenerator(securityConfig).generateKey(); + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -412,7 +421,8 @@ clusterId, scmId, caStore, new DefaultProfile(), .setSubject("testCA") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); Future holder = rootCA.requestCertificate(csr, CertificateApprover.ApprovalType.TESTING_AUTOMATIC, SCM, diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultProfile.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultProfile.java index 1204f905213..2f4dd681ea3 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultProfile.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultProfile.java @@ -41,7 +41,6 @@ import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder; import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.bouncycastle.pkcs.PKCS10CertificationRequestBuilder; -import org.bouncycastle.pkcs.PKCSException; import org.bouncycastle.pkcs.jcajce.JcaPKCS10CertificationRequestBuilder; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -50,8 +49,6 @@ import java.io.IOException; import java.nio.file.Path; import java.security.KeyPair; -import java.security.NoSuchAlgorithmException; -import java.security.NoSuchProviderException; import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -94,14 +91,10 @@ public void testisSupportedGeneralName() { /** * Test valid keys are validated correctly. - * - * @throws SCMSecurityException - on Error. - * @throws PKCSException - on Error. - * @throws OperatorCreationException - on Error. */ @Test - public void testVerifyCertificate() throws SCMSecurityException, - PKCSException, OperatorCreationException { + public void testVerifyCertificate() throws Exception { + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -112,7 +105,8 @@ public void testVerifyCertificate() throws SCMSecurityException, .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); assertTrue(approver.verifyPkcs10Request(csr)); } @@ -121,20 +115,13 @@ public void testVerifyCertificate() throws SCMSecurityException, /** * Test invalid keys fail in the validation. - * - * @throws SCMSecurityException - on Error. - * @throws PKCSException - on Error. - * @throws OperatorCreationException - on Error. - * @throws NoSuchProviderException - on Error. - * @throws NoSuchAlgorithmException - on Error. */ @Test - public void testVerifyCertificateInvalidKeys() throws SCMSecurityException, - PKCSException, OperatorCreationException, - NoSuchProviderException, NoSuchAlgorithmException { + public void testVerifyCertificateInvalidKeys() throws Exception { KeyPair newKeyPair = new HDDSKeyGenerator(securityConfig).generateKey(); KeyPair wrongKey = new KeyPair(keyPair.getPublic(), newKeyPair.getPrivate()); + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("8.8.8.8") @@ -144,7 +131,8 @@ public void testVerifyCertificateInvalidKeys() throws SCMSecurityException, .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(wrongKey) - .build(); + .build() + .generateCSR(); // Signature verification should fail here, since the public/private key // does not match. assertFalse(approver.verifyPkcs10Request(csr)); @@ -152,13 +140,10 @@ public void testVerifyCertificateInvalidKeys() throws SCMSecurityException, /** * Tests that normal valid extensions work with the default profile. - * - * @throws SCMSecurityException - on Error. - * @throws PKCSException - on Error. - * @throws OperatorCreationException - on Error. */ @Test - public void testExtensions() throws SCMSecurityException { + public void testExtensions() throws Exception { + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("192.10.234.6") @@ -168,7 +153,8 @@ public void testExtensions() throws SCMSecurityException { .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); assertTrue(approver.verfiyExtensions(csr)); } @@ -180,7 +166,8 @@ public void testExtensions() throws SCMSecurityException { */ @Test - public void testInvalidExtensionsWithCA() throws SCMSecurityException { + public void testInvalidExtensionsWithCA() throws Exception { + //TODO: generateCSR! PKCS10CertificationRequest csr = new CertificateSignRequest.Builder() .addDnsName("hadoop.apache.org") .addIpAddress("192.10.234.6") @@ -190,7 +177,8 @@ public void testInvalidExtensionsWithCA() throws SCMSecurityException { .setSubject("Ozone Cluster") .setConfiguration(securityConfig) .setKey(keyPair) - .build(); + .build() + .generateCSR(); assertFalse(approver.verfiyExtensions(csr)); } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClientTestImpl.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClientTestImpl.java index 00058500f59..fa784b75538 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClientTestImpl.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClientTestImpl.java @@ -48,6 +48,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.ssl.ReloadingX509KeyManager; import org.apache.hadoop.hdds.security.ssl.ReloadingX509TrustManager; @@ -135,13 +136,14 @@ public CertificateClientTestImpl(OzoneConfiguration conf, boolean autoRenew) start = LocalDateTime.now(); String certDuration = conf.get(HDDS_X509_DEFAULT_DURATION, HDDS_X509_DEFAULT_DURATION_DEFAULT); + //TODO: generateCSR should not be called... x509Certificate = approver.sign(securityConfig, rootKeyPair.getPrivate(), - rootCert, - Date.from(start.atZone(ZoneId.systemDefault()).toInstant()), - Date.from(start.plus(Duration.parse(certDuration)) - .atZone(ZoneId.systemDefault()).toInstant()), - csrBuilder.build(), "scm1", "cluster1", - String.valueOf(System.nanoTime())); + rootCert, + Date.from(start.atZone(ZoneId.systemDefault()).toInstant()), + Date.from(start.plus(Duration.parse(certDuration)) + .atZone(ZoneId.systemDefault()).toInstant()), + csrBuilder.build().generateCSR(), "scm1", "cluster1", + String.valueOf(System.nanoTime())); certificateMap.put(x509Certificate.getSerialNumber().toString(), x509Certificate); @@ -227,7 +229,7 @@ public boolean verifySignature(byte[] data, byte[] signature, } @Override - public CertificateSignRequest.Builder getCSRBuilder() { + public CertificateSignRequest.Builder configureCSRBuilder() throws SCMSecurityException { return new CertificateSignRequest.Builder(); } @@ -298,9 +300,10 @@ public void renewKey() throws Exception { Duration certDuration = securityConfig.getDefaultCertDuration(); Date start = new Date(); + //TODO: get rid of generateCSR call here, once the server side changes happened. X509Certificate newX509Certificate = approver.sign(securityConfig, rootKeyPair.getPrivate(), rootCert, start, - new Date(start.getTime() + certDuration.toMillis()), csrBuilder.build(), "scm1", "cluster1", + new Date(start.getTime() + certDuration.toMillis()), csrBuilder.build().generateCSR(), "scm1", "cluster1", String.valueOf(System.nanoTime()) ); diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java index 59c623a53dd..f6827352f00 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java @@ -25,9 +25,9 @@ import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.security.x509.certificate.authority.CAType; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec; +import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; import org.apache.hadoop.hdds.security.x509.exception.CertificateException; import org.apache.hadoop.hdds.security.x509.keys.KeyCodec; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -560,21 +560,12 @@ public void testCloseCertificateClient(@TempDir File metaDir) ) { @Override - protected String signAndStoreCertificate( - PKCS10CertificationRequest request, Path certificatePath) { - return ""; - } - - @Override - protected SCMGetCertResponseProto getCertificateSignResponse( - PKCS10CertificationRequest request) { + protected SCMGetCertResponseProto sign(CertificateSignRequest request) { return null; } @Override - protected String signAndStoreCertificate( - PKCS10CertificationRequest request, Path certificatePath, - boolean renew) { + protected String signAndStoreCertificate(CertificateSignRequest request, Path certificatePath, boolean renew) { return null; } }; diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/utils/TestCertificateSignRequest.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/utils/TestCertificateSignRequest.java index 1d32712fc28..598e74cb36c 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/utils/TestCertificateSignRequest.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/utils/TestCertificateSignRequest.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hdds.security.x509.certificate.utils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator; import org.bouncycastle.asn1.ASN1Encodable; @@ -33,10 +32,8 @@ import org.bouncycastle.asn1.x509.GeneralNames; import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo; import org.bouncycastle.operator.ContentVerifierProvider; -import org.bouncycastle.operator.OperatorCreationException; import org.bouncycastle.operator.jcajce.JcaContentVerifierProviderBuilder; import org.bouncycastle.pkcs.PKCS10CertificationRequest; -import org.bouncycastle.pkcs.PKCSException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -44,8 +41,6 @@ import java.io.IOException; import java.nio.file.Path; import java.security.KeyPair; -import java.security.NoSuchAlgorithmException; -import java.security.NoSuchProviderException; import java.util.Iterator; import java.util.UUID; @@ -73,9 +68,7 @@ public void init(@TempDir Path tempDir) throws IOException { } @Test - public void testGenerateCSR() throws NoSuchProviderException, - NoSuchAlgorithmException, SCMSecurityException, - OperatorCreationException, PKCSException { + public void testGenerateCSR() throws Exception { String clusterID = UUID.randomUUID().toString(); String scmID = UUID.randomUUID().toString(); String subject = "DN001"; @@ -90,7 +83,8 @@ public void testGenerateCSR() throws NoSuchProviderException, .setClusterID(clusterID) .setKey(keyPair) .setConfiguration(securityConfig); - PKCS10CertificationRequest csr = builder.build(); + //TODO: generateCSR! + PKCS10CertificationRequest csr = builder.build().generateCSR(); // Check the Subject Name is in the expected format. String dnName = String.format(getDistinguishedNameFormat(), @@ -124,9 +118,7 @@ public void testGenerateCSR() throws NoSuchProviderException, } @Test - public void testGenerateCSRwithSan() throws NoSuchProviderException, - NoSuchAlgorithmException, SCMSecurityException, - OperatorCreationException, PKCSException { + public void testGenerateCSRwithSan() throws Exception { String clusterID = UUID.randomUUID().toString(); String scmID = UUID.randomUUID().toString(); String subject = "DN001"; @@ -149,7 +141,8 @@ public void testGenerateCSRwithSan() throws NoSuchProviderException, builder.addDnsName("dn1.abc.com"); - PKCS10CertificationRequest csr = builder.build(); + //TODO: generateCSR! + PKCS10CertificationRequest csr = builder.build().generateCSR(); // Check the Subject Name is in the expected format. String dnName = String.format(getDistinguishedNameFormat(), @@ -181,8 +174,7 @@ public void testGenerateCSRwithSan() throws NoSuchProviderException, } @Test - public void testGenerateCSRWithInvalidParams() throws NoSuchProviderException, - NoSuchAlgorithmException, SCMSecurityException { + public void testGenerateCSRWithInvalidParams() throws Exception { String clusterID = UUID.randomUUID().toString(); String scmID = UUID.randomUUID().toString(); String subject = "DN001"; @@ -225,7 +217,8 @@ public void testGenerateCSRWithInvalidParams() throws NoSuchProviderException, builder.build(); }); - PKCS10CertificationRequest csr = builder.build(); + //TODO: generateCSR! + PKCS10CertificationRequest csr = builder.build().generateCSR(); // Check the Subject Name is in the expected format. String dnName = String.format(getDistinguishedNameFormat(), @@ -244,8 +237,7 @@ public void testGenerateCSRWithInvalidParams() throws NoSuchProviderException, } @Test - public void testCsrSerialization() throws NoSuchProviderException, - NoSuchAlgorithmException, SCMSecurityException, IOException { + public void testCsrSerialization() throws Exception { String clusterID = UUID.randomUUID().toString(); String scmID = UUID.randomUUID().toString(); String subject = "DN001"; @@ -261,7 +253,8 @@ public void testCsrSerialization() throws NoSuchProviderException, .setKey(keyPair) .setConfiguration(securityConfig); - PKCS10CertificationRequest csr = builder.build(); + //TODO: generateCSR! + PKCS10CertificationRequest csr = builder.build().generateCSR(); byte[] csrBytes = csr.getEncoded(); // Verify de-serialized CSR matches with the original CSR diff --git a/hadoop-hdds/interface-server/pom.xml b/hadoop-hdds/interface-server/pom.xml index 6c689171c8d..df65c1e2b2a 100644 --- a/hadoop-hdds/interface-server/pom.xml +++ b/hadoop-hdds/interface-server/pom.xml @@ -34,6 +34,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ratis + ratis-thirdparty-misc + com.google.protobuf protobuf-java diff --git a/hadoop-hdds/managed-rocksdb/pom.xml b/hadoop-hdds/managed-rocksdb/pom.xml index b95fcc3479d..125783222e5 100644 --- a/hadoop-hdds/managed-rocksdb/pom.xml +++ b/hadoop-hdds/managed-rocksdb/pom.xml @@ -36,10 +36,30 @@ org.apache.ozone hdds-common + + + org.apache.ratis + ratis-common + + org.rocksdb rocksdbjni + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + + + jakarta.annotation + jakarta.annotation-api + diff --git a/hadoop-hdds/rocks-native/pom.xml b/hadoop-hdds/rocks-native/pom.xml index 509586806c8..5fc9949514b 100644 --- a/hadoop-hdds/rocks-native/pom.xml +++ b/hadoop-hdds/rocks-native/pom.xml @@ -25,14 +25,37 @@ hdds-rocks-native + + org.apache.ozone + hdds-common + org.apache.ozone hdds-managed-rocksdb + + + org.apache.commons + commons-lang3 + + org.eclipse.jetty jetty-io + + org.rocksdb + rocksdbjni + + + org.slf4j + slf4j-api + + + + com.google.guava + guava + diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/pom.xml b/hadoop-hdds/rocksdb-checkpoint-differ/pom.xml index 22f87556054..3e535c5f5f2 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/pom.xml +++ b/hadoop-hdds/rocksdb-checkpoint-differ/pom.xml @@ -41,6 +41,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-common + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-interface-client + org.apache.ozone hdds-managed-rocksdb @@ -49,6 +57,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-rocks-native + com.google.guava guava @@ -57,6 +66,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.commons commons-lang3 + + org.apache.ratis + ratis-common + org.jgrapht jgrapht-core @@ -65,11 +78,29 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.jgrapht jgrapht-ext + + org.slf4j + slf4j-api + org.slf4j slf4j-reload4j + + com.github.vlsi.mxgraph + jgraphx + + + com.google.protobuf + protobuf-java + + + + commons-collections + commons-collections + + org.apache.ozone diff --git a/hadoop-hdds/server-scm/pom.xml b/hadoop-hdds/server-scm/pom.xml index f1a998f27cf..32408e8904b 100644 --- a/hadoop-hdds/server-scm/pom.xml +++ b/hadoop-hdds/server-scm/pom.xml @@ -41,6 +41,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-common + + org.apache.ozone + hdds-config + org.apache.ozone @@ -55,6 +59,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-hadoop-dependency-server + + org.apache.ozone + hdds-interface-admin + + + org.apache.ozone + hdds-interface-client + + + org.apache.ozone + hdds-interface-server + org.apache.ozone hdds-server-framework @@ -65,6 +81,44 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-docs provided + + + org.apache.ratis + ratis-client + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-grpc + + + org.apache.ratis + ratis-netty + + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-server-api + + + org.apache.ratis + ratis-server + + + org.apache.ratis + ratis-thirdparty-misc + + + + org.bouncycastle + bcpkix-jdk18on + org.bouncycastle bcprov-jdk18on @@ -74,6 +128,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> metrics-core + + org.apache.commons + commons-compress + + + org.apache.commons + commons-lang3 + org.apache.commons commons-text @@ -90,6 +152,54 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.eclipse.jetty + jetty-webapp + + + org.slf4j + slf4j-api + + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.google.guava + guava + + + + info.picocli + picocli + + + jakarta.annotation + jakarta.annotation-api + + + javax.servlet + javax.servlet-api + + + + commons-collections + commons-collections + + + commons-io + commons-io + + org.apache.commons diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/reconciliation/ReconcileContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/reconciliation/ReconcileContainerEventHandler.java index f13b37f3ee2..8e8b5bf71c4 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/reconciliation/ReconcileContainerEventHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/reconciliation/ReconcileContainerEventHandler.java @@ -32,7 +32,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -78,9 +77,9 @@ public void onMessage(ContainerID containerID, EventPublisher publisher) { LOG.info("Reconcile container event triggered for container {} with peers {}", containerID, allReplicaNodes); for (DatanodeDetails replica : allReplicaNodes) { - List otherReplicas = allReplicaNodes.stream() + Set otherReplicas = allReplicaNodes.stream() .filter(other -> !other.equals(replica)) - .collect(Collectors.toList()); + .collect(Collectors.toSet()); ReconcileContainerCommand command = new ReconcileContainerCommand(containerID.getId(), otherReplicas); command.setTerm(scmContext.getTermOfLeader()); publisher.fireEvent(DATANODE_COMMAND, new CommandForDatanode<>(replica.getUuid(), command)); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java index ccef5aab24e..229ba6afe0d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/RatisUtil.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import com.google.protobuf.ServiceException; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.conf.RatisConfUtils; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.ratis.RatisHelper; import org.apache.hadoop.hdds.ratis.ServerNotLeaderException; @@ -69,8 +70,9 @@ public static RaftProperties newRaftProperties( // TODO: Check the default values. final RaftProperties properties = new RaftProperties(); setRaftStorageDir(properties, conf); - setRaftRpcProperties(properties, conf); - setRaftLogProperties(properties, conf); + + final int logAppenderBufferByteLimit = setRaftLogProperties(properties, conf); + setRaftRpcProperties(properties, conf, logAppenderBufferByteLimit); setRaftRetryCacheProperties(properties, conf); setRaftSnapshotProperties(properties, conf); setRaftLeadElectionProperties(properties, conf); @@ -100,15 +102,14 @@ public static void setRaftStorageDir(final RaftProperties properties, * @param ozoneConf ConfigurationSource */ private static void setRaftRpcProperties(final RaftProperties properties, - ConfigurationSource ozoneConf) { + ConfigurationSource ozoneConf, int logAppenderBufferByteLimit) { RatisHelper.setRpcType(properties, RpcType.valueOf(ozoneConf.get(ScmConfigKeys.OZONE_SCM_HA_RATIS_RPC_TYPE, ScmConfigKeys.OZONE_SCM_HA_RATIS_RPC_TYPE_DEFAULT))); GrpcConfigKeys.Server.setPort(properties, ozoneConf .getInt(ScmConfigKeys.OZONE_SCM_RATIS_PORT_KEY, ScmConfigKeys.OZONE_SCM_RATIS_PORT_DEFAULT)); - GrpcConfigKeys.setMessageSizeMax(properties, - SizeInBytes.valueOf("32m")); + RatisConfUtils.Grpc.setMessageSizeMax(properties, logAppenderBufferByteLimit); long ratisRequestTimeout = ozoneConf.getTimeDuration( ScmConfigKeys.OZONE_SCM_HA_RATIS_REQUEST_TIMEOUT, ScmConfigKeys.OZONE_SCM_HA_RATIS_REQUEST_TIMEOUT_DEFAULT, @@ -161,7 +162,7 @@ private static void setRaftLeadElectionProperties( * @param properties RaftProperties instance which will be updated * @param ozoneConf ConfigurationSource */ - private static void setRaftLogProperties(final RaftProperties properties, + private static int setRaftLogProperties(final RaftProperties properties, final ConfigurationSource ozoneConf) { Log.setSegmentSizeMax(properties, SizeInBytes.valueOf((long) ozoneConf.getStorageSize( @@ -195,6 +196,7 @@ private static void setRaftLogProperties(final RaftProperties properties, ozoneConf.getInt(ScmConfigKeys.OZONE_SCM_HA_RAFT_LOG_PURGE_GAP, ScmConfigKeys.OZONE_SCM_HA_RAFT_LOG_PURGE_GAP_DEFAULT)); Log.setSegmentCacheNumMax(properties, 2); + return logAppenderQueueByteLimit; } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 3339b27f2ce..05a68628852 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -1155,29 +1155,37 @@ public static String calculateStorageCapacity( } } - double ua = capacityByte; + return convertUnit(capacityByte); + } + + /** + * Convert byte value to other units, such as KB, MB, GB, TB. + * @param value Original value, in byte. + * @return + */ + private static String convertUnit(double value) { StringBuilder unit = new StringBuilder("B"); - if (ua > 1024) { - ua = ua / 1024; + if (value > 1024) { + value = value / 1024; unit.replace(0, 1, "KB"); } - if (ua > 1024) { - ua = ua / 1024; + if (value > 1024) { + value = value / 1024; unit.replace(0, 2, "MB"); } - if (ua > 1024) { - ua = ua / 1024; + if (value > 1024) { + value = value / 1024; unit.replace(0, 2, "GB"); } - if (ua > 1024) { - ua = ua / 1024; + if (value > 1024) { + value = value / 1024; unit.replace(0, 2, "TB"); } DecimalFormat decimalFormat = new DecimalFormat("#0.0"); decimalFormat.setRoundingMode(RoundingMode.HALF_UP); - String capacity = decimalFormat.format(ua); - return capacity + unit.toString(); + String newValue = decimalFormat.format(value); + return newValue + unit.toString(); } /** @@ -1225,6 +1233,8 @@ public Map getNodeStatistics() { nodeUsageStatistics(nodeStatistics); // Statistics node states nodeStateStatistics(nodeStatistics); + // Statistics node space + nodeSpaceStatistics(nodeStatistics); // todo: Statistics of other instances return nodeStatistics; } @@ -1280,6 +1290,38 @@ private void nodeStateStatistics(Map nodeStatics) { nodeStatics.put(StateStatistics.VOLUME_FAILURES.getLabel(), String.valueOf(volumeFailuresNodeCount)); } + private void nodeSpaceStatistics(Map nodeStatics) { + if (nodeStateManager.getAllNodes().size() < 1) { + return; + } + long capacityByte = 0; + long scmUsedByte = 0; + long remainingByte = 0; + for (DatanodeInfo dni : nodeStateManager.getAllNodes()) { + List storageReports = dni.getStorageReports(); + if (storageReports != null && !storageReports.isEmpty()) { + for (StorageReportProto storageReport : storageReports) { + capacityByte += storageReport.getCapacity(); + scmUsedByte += storageReport.getScmUsed(); + remainingByte += storageReport.getRemaining(); + } + } + } + + long nonScmUsedByte = capacityByte - scmUsedByte - remainingByte; + if (nonScmUsedByte < 0) { + nonScmUsedByte = 0; + } + String capacity = convertUnit(capacityByte); + String scmUsed = convertUnit(scmUsedByte); + String remaining = convertUnit(remainingByte); + String nonScmUsed = convertUnit(nonScmUsedByte); + nodeStatics.put(SpaceStatistics.CAPACITY.getLabel(), capacity); + nodeStatics.put(SpaceStatistics.SCM_USED.getLabel(), scmUsed); + nodeStatics.put(SpaceStatistics.REMAINING.getLabel(), remaining); + nodeStatics.put(SpaceStatistics.NON_SCM_USED.getLabel(), nonScmUsed); + } + /** * Based on the current time and the last heartbeat, calculate the time difference * and get a string of the relative value. E.g. "2s ago", "1m 2s ago", etc. @@ -1376,6 +1418,20 @@ public String getLabel() { } } + private enum SpaceStatistics { + CAPACITY("Capacity"), + SCM_USED("Scmused"), + NON_SCM_USED("NonScmused"), + REMAINING("Remaining"); + private String label; + public String getLabel() { + return label; + } + SpaceStatistics(String label) { + this.label = label; + } + } + /** * Returns the min of no healthy volumes reported out of the set * of datanodes constituting the pipeline. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/security/RootCARotationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/security/RootCARotationManager.java index c88abb5b8d0..d38a904d09c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/security/RootCARotationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/security/RootCARotationManager.java @@ -584,7 +584,7 @@ public void run() { String newCertSerialId = ""; try { CertificateSignRequest.Builder csrBuilder = - scmCertClient.getCSRBuilder(); + scmCertClient.configureCSRBuilder(); csrBuilder.setKey(newKeyPair); newCertSerialId = scmCertClient.signAndStoreCertificate( csrBuilder.build(), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java index a035751796e..7ed23b110c7 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java @@ -73,6 +73,8 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_BLOCK_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_BLOCK_READ_THREADPOOL_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_BLOCK_READ_THREADPOOL_DEFAULT; import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.IO_EXCEPTION; import static org.apache.hadoop.hdds.scm.net.NetConstants.NODE_COST_DEFAULT; import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT; @@ -117,6 +119,8 @@ public SCMBlockProtocolServer(OzoneConfiguration conf, final int handlerCount = conf.getInt(OZONE_SCM_BLOCK_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_DEFAULT, LOG::info); + final int readThreads = conf.getInt(OZONE_SCM_BLOCK_READ_THREADPOOL_KEY, + OZONE_SCM_BLOCK_READ_THREADPOOL_DEFAULT); RPC.setProtocolEngine(conf, ScmBlockLocationProtocolPB.class, ProtobufRpcEngine.class); @@ -142,7 +146,8 @@ public SCMBlockProtocolServer(OzoneConfiguration conf, scmBlockAddress, ScmBlockLocationProtocolPB.class, blockProtoPbService, - handlerCount); + handlerCount, + readThreads); blockRpcAddress = updateRPCListenAddress( conf, scm.getScmNodeDetails().getBlockProtocolServerAddressKey(), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index 4bba6b7ac29..3bf8d9c55ca 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -122,6 +122,8 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_READ_THREADPOOL_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_READ_THREADPOOL_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmUtils.checkIfCertSignRequestAllowed; import static org.apache.hadoop.hdds.scm.ha.HASecurityUtils.createSCMRatisTLSConfig; import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.startRpcServer; @@ -152,6 +154,8 @@ public SCMClientProtocolServer(OzoneConfiguration conf, final int handlerCount = conf.getInt(OZONE_SCM_CLIENT_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_DEFAULT, LOG::info); + final int readThreads = conf.getInt(OZONE_SCM_CLIENT_READ_THREADPOOL_KEY, + OZONE_SCM_CLIENT_READ_THREADPOOL_DEFAULT); RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class, ProtobufRpcEngine.class); @@ -175,7 +179,8 @@ public SCMClientProtocolServer(OzoneConfiguration conf, scmAddress, StorageContainerLocationProtocolPB.class, storageProtoPbService, - handlerCount); + handlerCount, + readThreads); // Add reconfigureProtocolService. ReconfigureProtocolServerSideTranslatorPB reconfigureServerProtocol diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java index 98a7aa22f3e..e74a83e394f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java @@ -104,6 +104,8 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_READ_THREADPOOL_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_READ_THREADPOOL_DEFAULT; import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT; import static org.apache.hadoop.hdds.scm.events.SCMEvents.PIPELINE_REPORT; import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.startRpcServer; @@ -163,6 +165,8 @@ public SCMDatanodeProtocolServer(final OzoneConfiguration conf, final int handlerCount = conf.getInt(OZONE_SCM_DATANODE_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_DEFAULT, LOG::info); + final int readThreads = conf.getInt(OZONE_SCM_DATANODE_READ_THREADPOOL_KEY, + OZONE_SCM_DATANODE_READ_THREADPOOL_DEFAULT); RPC.setProtocolEngine(conf, getProtocolClass(), ProtobufRpcEngine.class); @@ -178,7 +182,8 @@ public SCMDatanodeProtocolServer(final OzoneConfiguration conf, datanodeRpcAddr, getProtocolClass(), dnProtoPbService, - handlerCount); + handlerCount, + readThreads); datanodeRpcAddress = updateRPCListenAddress( conf, getDatanodeAddressKey(), datanodeRpcAddr, diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java index 6fea1f106f0..88b3c887746 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java @@ -129,6 +129,8 @@ public class SCMSecurityProtocolServer implements SCMSecurityProtocol, final int handlerCount = conf.getInt(ScmConfigKeys.OZONE_SCM_SECURITY_HANDLER_COUNT_KEY, ScmConfigKeys.OZONE_SCM_SECURITY_HANDLER_COUNT_DEFAULT); + final int readThreads = conf.getInt(ScmConfigKeys.OZONE_SCM_SECURITY_READ_THREADPOOL_KEY, + ScmConfigKeys.OZONE_SCM_SECURITY_READ_THREADPOOL_DEFAULT); rpcAddress = HddsServerUtil .getScmSecurityInetAddress(conf); // SCM security service RPC service. @@ -157,7 +159,8 @@ public class SCMSecurityProtocolServer implements SCMSecurityProtocol, rpcAddress, SCMSecurityProtocolPB.class, secureProtoPbService, - handlerCount); + handlerCount, + readThreads); HddsServerUtil.addPBProtocol(conf, SecretKeyProtocolDatanodePB.class, secretKeyService, rpcServer); HddsServerUtil.addPBProtocol(conf, SecretKeyProtocolOmPB.class, diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 7ee17d5837d..876c499113d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -1108,7 +1108,8 @@ public static RPC.Server startRpcServer( InetSocketAddress addr, Class protocol, BlockingService instance, - int handlerCount) + int handlerCount, + int readThreads) throws IOException { RPC.Server rpcServer = preserveThreadName(() -> new RPC.Builder(conf) @@ -1117,6 +1118,7 @@ public static RPC.Server startRpcServer( .setBindAddress(addr.getHostString()) .setPort(addr.getPort()) .setNumHandlers(handlerCount) + .setNumReaders(readThreads) .setVerbose(false) .setSecretManager(null) .build()); diff --git a/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html b/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html index 5a4f2ff633c..3f825d4e25f 100644 --- a/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html +++ b/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html @@ -84,6 +84,32 @@

State Statistics

+

Space Statistics

+ + + + + + + + + + + + + + + + + + + + + + + +
Datanode SpaceSize
Capacity{{statistics.nodes.space.capacity}}
ScmUsed{{statistics.nodes.space.scmused}}
Remaining{{statistics.nodes.space.remaining}}
Non ScmUsed{{statistics.nodes.space.nonscmused}}
+

Node Status

diff --git a/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm.js b/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm.js index 41dc25cb650..6fac6849530 100644 --- a/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm.js +++ b/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm.js @@ -46,6 +46,12 @@ decommissioning : "N/A", enteringmaintenance : "N/A", volumefailures : "N/A" + }, + space : { + capacity : "N/A", + scmused : "N/A", + remaining : "N/A", + nonscmused : "N/A" } } } @@ -118,6 +124,14 @@ $scope.statistics.nodes.state.enteringmaintenance = value; } else if(key == "VolumeFailures") { $scope.statistics.nodes.state.volumefailures = value; + } else if(key == "Capacity") { + $scope.statistics.nodes.space.capacity = value; + } else if(key == "Scmused") { + $scope.statistics.nodes.space.scmused = value; + } else if(key == "Remaining") { + $scope.statistics.nodes.space.remaining = value; + } else if(key == "NonScmused") { + $scope.statistics.nodes.space.nonscmused = value; } }); }); diff --git a/hadoop-hdds/tools/pom.xml b/hadoop-hdds/tools/pom.xml index 5e7238f5f3e..daf6f3d40f4 100644 --- a/hadoop-hdds/tools/pom.xml +++ b/hadoop-hdds/tools/pom.xml @@ -37,6 +37,22 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-common + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-container-service + + + org.apache.ozone + hdds-interface-admin + + + org.apache.ozone + hdds-interface-client + org.apache.ozone hdds-server-framework @@ -45,6 +61,15 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-client + + + org.apache.commons + commons-lang3 + + + org.apache.ratis + ratis-common + ratis-tools org.apache.ratis @@ -86,6 +111,37 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-server-scm + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-jsr310 + + + com.google.guava + guava + + + + info.picocli + picocli + + + + commons-io + commons-io + + org.apache.ozone @@ -109,6 +165,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-test-utils test + + commons-codec + commons-codec + test + diff --git a/hadoop-ozone/client/pom.xml b/hadoop-ozone/client/pom.xml index dac609caa46..545faba51ac 100644 --- a/hadoop-ozone/client/pom.xml +++ b/hadoop-ozone/client/pom.xml @@ -31,19 +31,73 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-client + org.apache.ozone hdds-erasurecode + + org.apache.ozone + hdds-interface-client + org.apache.ozone ozone-common + + org.apache.ozone + ozone-interface-client + + + org.apache.commons + commons-lang3 + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-thirdparty-misc + + + org.slf4j + slf4j-api + + + + com.fasterxml.jackson.core + jackson-annotations + com.github.stephenc.jcip jcip-annotations + + com.google.guava + guava + + + + commons-collections + commons-collections + + + jakarta.annotation + jakarta.annotation-api + diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java index e77f5580cca..65dce09cba1 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java @@ -34,6 +34,7 @@ import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BucketLayout; @@ -391,6 +392,10 @@ public void deleteVolume(String volumeName) throws IOException { proxy.deleteVolume(volumeName); } + public OzoneFsServerDefaults getServerDefaults() throws IOException { + return proxy.getServerDefaults(); + } + public KeyProvider getKeyProvider() throws IOException { return proxy.getKeyProvider(); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java index 18a9231c66f..2ae3e475531 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java @@ -22,6 +22,8 @@ import java.util.Collection; import java.util.Collections; import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; import java.util.function.Supplier; import org.apache.hadoop.fs.Syncable; @@ -41,6 +43,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.ratis.util.JavaUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A BlockOutputStreamEntry manages the data writes into the DataNodes. @@ -51,9 +55,9 @@ * but there can be other implementations that are using a different way. */ public class BlockOutputStreamEntry extends OutputStream { - + public static final Logger LOG = LoggerFactory.getLogger(BlockOutputStreamEntry.class); private final OzoneClientConfig config; - private OutputStream outputStream; + private BlockOutputStream outputStream; private BlockID blockID; private final String key; private final XceiverClientFactory xceiverClientManager; @@ -69,6 +73,18 @@ public class BlockOutputStreamEntry extends OutputStream { private final StreamBufferArgs streamBufferArgs; private final Supplier executorServiceSupplier; + /** + * An indicator that this BlockOutputStream is created to handoff writes from another faulty BlockOutputStream. + * Once this flag is on, this BlockOutputStream can only handle writeOnRetry. + */ + private volatile boolean isHandlingRetry; + + /** + * To record how many calls(write, flush) are being handled by this block. + */ + private AtomicInteger inflightCalls = new AtomicInteger(); + + BlockOutputStreamEntry(Builder b) { this.config = b.config; this.outputStream = null; @@ -83,6 +99,7 @@ public class BlockOutputStreamEntry extends OutputStream { this.clientMetrics = b.clientMetrics; this.streamBufferArgs = b.streamBufferArgs; this.executorServiceSupplier = b.executorServiceSupplier; + this.isHandlingRetry = b.forRetry; } @Override @@ -102,13 +119,44 @@ void checkStream() throws IOException { } } + /** Register when a call (write or flush) is received on this block. */ + void registerCallReceived() { + inflightCalls.incrementAndGet(); + } + + /** + * Register when a call (write or flush) is finished on this block. + * @return true if all the calls are done. + */ + boolean registerCallFinished() { + return inflightCalls.decrementAndGet() == 0; + } + + void waitForRetryHandling(Condition retryHandlingCond) throws InterruptedException { + while (isHandlingRetry) { + LOG.info("{} : Block to wait for retry handling.", this); + retryHandlingCond.await(); + LOG.info("{} : Done waiting for retry handling.", this); + } + } + + void finishRetryHandling(Condition retryHandlingCond) { + LOG.info("{}: Exiting retry handling mode", this); + isHandlingRetry = false; + retryHandlingCond.signalAll(); + } + + void waitForAllPendingFlushes() throws IOException { + outputStream.waitForAllPendingFlushes(); + } + /** * Creates the outputStreams that are necessary to start the write. * Implementors can override this to instantiate multiple streams instead. * @throws IOException */ void createOutputStream() throws IOException { - outputStream = new RatisBlockOutputStream(blockID, xceiverClientManager, + outputStream = new RatisBlockOutputStream(blockID, length, xceiverClientManager, pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs, executorServiceSupplier); } @@ -144,6 +192,7 @@ void writeOnRetry(long len) throws IOException { BlockOutputStream out = (BlockOutputStream) getOutputStream(); out.writeOnRetry(len); incCurrentPosition(len); + LOG.info("{}: Finish retrying with len {}, currentPosition {}", this, len, currentPosition); } @Override @@ -368,6 +417,7 @@ public static class Builder { private ContainerClientMetrics clientMetrics; private StreamBufferArgs streamBufferArgs; private Supplier executorServiceSupplier; + private boolean forRetry; public Pipeline getPipeline() { return pipeline; @@ -433,6 +483,11 @@ public Builder setExecutorServiceSupplier(Supplier executorServ return this; } + public Builder setForRetry(boolean forRetry) { + this.forRetry = forRetry; + return this; + } + public BlockOutputStreamEntry build() { return new BlockOutputStreamEntry(this); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java index 99899c6874e..3705a136377 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java @@ -141,7 +141,7 @@ public synchronized void addPreallocateBlocks(OmKeyLocationInfoGroup version, lo // only the blocks allocated in this open session (block createVersion // equals to open session version) for (OmKeyLocationInfo subKeyInfo : version.getLocationList(openVersion)) { - addKeyLocationInfo(subKeyInfo); + addKeyLocationInfo(subKeyInfo, false); } } @@ -154,7 +154,7 @@ public synchronized void addPreallocateBlocks(OmKeyLocationInfoGroup version, lo * key to be written. * @return a BlockOutputStreamEntry instance that handles how data is written. */ - BlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo) { + BlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo, boolean forRetry) { return new BlockOutputStreamEntry.Builder() .setBlockID(subKeyInfo.getBlockID()) @@ -168,12 +168,13 @@ BlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo) { .setClientMetrics(clientMetrics) .setStreamBufferArgs(streamBufferArgs) .setExecutorServiceSupplier(executorServiceSupplier) + .setForRetry(forRetry) .build(); } - private synchronized void addKeyLocationInfo(OmKeyLocationInfo subKeyInfo) { + private synchronized void addKeyLocationInfo(OmKeyLocationInfo subKeyInfo, boolean forRetry) { Preconditions.checkNotNull(subKeyInfo.getPipeline()); - streamEntries.add(createStreamEntry(subKeyInfo)); + streamEntries.add(createStreamEntry(subKeyInfo, forRetry)); } /** @@ -295,13 +296,13 @@ synchronized long getKeyLength() { * * @throws IOException */ - private void allocateNewBlock() throws IOException { + private void allocateNewBlock(boolean forRetry) throws IOException { if (!excludeList.isEmpty()) { LOG.debug("Allocating block with {}", excludeList); } OmKeyLocationInfo subKeyInfo = omClient.allocateBlock(keyArgs, openID, excludeList); - addKeyLocationInfo(subKeyInfo); + addKeyLocationInfo(subKeyInfo, forRetry); } /** @@ -379,7 +380,7 @@ BlockOutputStreamEntry getCurrentStreamEntry() { * @return the new current open stream to write to * @throws IOException if the block allocation failed. */ - synchronized BlockOutputStreamEntry allocateBlockIfNeeded() throws IOException { + synchronized BlockOutputStreamEntry allocateBlockIfNeeded(boolean forRetry) throws IOException { BlockOutputStreamEntry streamEntry = getCurrentStreamEntry(); if (streamEntry != null && streamEntry.isClosed()) { // a stream entry gets closed either by : @@ -391,7 +392,7 @@ synchronized BlockOutputStreamEntry allocateBlockIfNeeded() throws IOException { Preconditions.checkNotNull(omClient); // allocate a new block, if a exception happens, log an error and // throw exception to the caller directly, and the write fails. - allocateNewBlock(); + allocateNewBlock(forRetry); } // in theory, this condition should never violate due the check above // still do a sanity check. diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java index 6eb9aed0d3a..f891724270e 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java @@ -37,7 +37,7 @@ public ECBlockOutputStreamEntryPool(ECKeyOutputStream.Builder builder) { } @Override - ECBlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo) { + ECBlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo, boolean forRetry) { final ECBlockOutputStreamEntry.Builder b = new ECBlockOutputStreamEntry.Builder(); b.setBlockID(subKeyInfo.getBlockID()) .setKey(getKeyName()) diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java index 0cb3973e041..ea3a3592a55 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java @@ -19,6 +19,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -314,7 +315,7 @@ private void generateParityCells() throws IOException { private void writeDataCells(ECChunkBuffers stripe) throws IOException { final ECBlockOutputStreamEntryPool blockOutputStreamEntryPool = getBlockOutputStreamEntryPool(); - blockOutputStreamEntryPool.allocateBlockIfNeeded(); + blockOutputStreamEntryPool.allocateBlockIfNeeded(false); ByteBuffer[] dataCells = stripe.getDataBuffers(); for (int i = 0; i < numDataBlks; i++) { if (dataCells[i].limit() > 0) { @@ -420,6 +421,16 @@ public void flush() { LOG.debug("ECKeyOutputStream does not support flush."); } + @Override + public void hflush() { + throw new NotImplementedException("ECKeyOutputStream does not support hflush."); + } + + @Override + public void hsync() { + throw new NotImplementedException("ECKeyOutputStream does not support hsync."); + } + private void closeCurrentStreamEntry() throws IOException { final ECBlockOutputStreamEntryPool blockOutputStreamEntryPool = getBlockOutputStreamEntryPool(); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java index 59c0fa134aa..4f9e5db49a9 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java @@ -25,6 +25,9 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ExecutorService; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -58,6 +61,7 @@ import com.google.common.base.Preconditions; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.protocol.exceptions.RaftRetryFailureException; +import org.apache.ratis.util.function.CheckedRunnable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -109,6 +113,27 @@ enum StreamAction { private boolean atomicKeyCreation; private ContainerClientMetrics clientMetrics; private OzoneManagerVersion ozoneManagerVersion; + private final Lock writeLock = new ReentrantLock(); + private final Condition retryHandlingCondition = writeLock.newCondition(); + + private final int maxConcurrentWritePerKey; + private final KeyOutputStreamSemaphore keyOutputStreamSemaphore; + + @VisibleForTesting + KeyOutputStreamSemaphore getRequestSemaphore() { + return keyOutputStreamSemaphore; + } + + /** Required to spy the object in testing. */ + @VisibleForTesting + @SuppressWarnings("unused") + KeyOutputStream() { + maxConcurrentWritePerKey = 0; + keyOutputStreamSemaphore = null; + blockOutputStreamEntryPool = null; + retryPolicyMap = null; + replication = null; + } public KeyOutputStream(ReplicationConfig replicationConfig, BlockOutputStreamEntryPool blockOutputStreamEntryPool) { this.replication = replicationConfig; @@ -120,6 +145,10 @@ public KeyOutputStream(ReplicationConfig replicationConfig, BlockOutputStreamEnt retryCount = 0; offset = 0; this.blockOutputStreamEntryPool = blockOutputStreamEntryPool; + // Force write concurrency to 1 per key when using this constructor. + // At the moment, this constructor is only used by ECKeyOutputStream. + this.maxConcurrentWritePerKey = 1; + this.keyOutputStreamSemaphore = new KeyOutputStreamSemaphore(maxConcurrentWritePerKey); } protected BlockOutputStreamEntryPool getBlockOutputStreamEntryPool() { @@ -150,6 +179,8 @@ public KeyOutputStream(Builder b) { this.replication = b.replicationConfig; this.blockOutputStreamEntryPool = new BlockOutputStreamEntryPool(b); final OzoneClientConfig config = b.getClientConfig(); + this.maxConcurrentWritePerKey = config.getMaxConcurrentWritePerKey(); + this.keyOutputStreamSemaphore = new KeyOutputStreamSemaphore(maxConcurrentWritePerKey); this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException( config.getMaxRetryCount(), config.getRetryInterval()); this.retryCount = 0; @@ -174,12 +205,12 @@ public KeyOutputStream(Builder b) { * @param version the set of blocks that are pre-allocated. * @param openVersion the version corresponding to the pre-allocation. */ - public synchronized void addPreallocateBlocks(OmKeyLocationInfoGroup version, long openVersion) { + public void addPreallocateBlocks(OmKeyLocationInfoGroup version, long openVersion) { blockOutputStreamEntryPool.addPreallocateBlocks(version, openVersion); } @Override - public synchronized void write(int b) throws IOException { + public void write(int b) throws IOException { byte[] buf = new byte[1]; buf[0] = (byte) b; write(buf, 0, 1); @@ -200,29 +231,45 @@ public synchronized void write(int b) throws IOException { @Override public void write(byte[] b, int off, int len) throws IOException { - checkNotClosed(); - if (b == null) { - throw new NullPointerException(); - } - if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) - || ((off + len) < 0)) { - throw new IndexOutOfBoundsException(); - } - if (len == 0) { - return; + try { + getRequestSemaphore().acquire(); + checkNotClosed(); + if (b == null) { + throw new NullPointerException(); + } + if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) + || ((off + len) < 0)) { + throw new IndexOutOfBoundsException(); + } + if (len == 0) { + return; + } + + doInWriteLock(() -> { + handleWrite(b, off, len, false); + writeOffset += len; + }); + } finally { + getRequestSemaphore().release(); } - synchronized (this) { - handleWrite(b, off, len, false); - writeOffset += len; + } + + private void doInWriteLock(CheckedRunnable block) throws E { + writeLock.lock(); + try { + block.run(); + } finally { + writeLock.unlock(); } } - private void handleWrite(byte[] b, int off, long len, boolean retry) + @VisibleForTesting + void handleWrite(byte[] b, int off, long len, boolean retry) throws IOException { while (len > 0) { try { BlockOutputStreamEntry current = - blockOutputStreamEntryPool.allocateBlockIfNeeded(); + blockOutputStreamEntryPool.allocateBlockIfNeeded(retry); // length(len) will be in int range if the call is happening through // write API of blockOutputStream. Length can be in long range if it // comes via Exception path. @@ -252,12 +299,18 @@ private int writeToOutputStream(BlockOutputStreamEntry current, boolean retry, long len, byte[] b, int writeLen, int off, long currentPos) throws IOException { try { + current.registerCallReceived(); if (retry) { current.writeOnRetry(len); } else { + waitForRetryHandling(current); current.write(b, off, writeLen); offset += writeLen; } + current.registerCallFinished(); + } catch (InterruptedException e) { + current.registerCallFinished(); + throw new InterruptedIOException(); } catch (IOException ioe) { // for the current iteration, totalDataWritten - currentPos gives the // amount of data already written to the buffer @@ -275,11 +328,24 @@ private int writeToOutputStream(BlockOutputStreamEntry current, offset += writeLen; } LOG.debug("writeLen {}, total len {}", writeLen, len); - handleException(current, ioe); + handleException(current, ioe, retry); } return writeLen; } + private void handleException(BlockOutputStreamEntry entry, IOException exception, boolean fromRetry) + throws IOException { + doInWriteLock(() -> { + handleExceptionInternal(entry, exception); + BlockOutputStreamEntry current = blockOutputStreamEntryPool.getCurrentStreamEntry(); + if (!fromRetry && entry.registerCallFinished()) { + // When the faulty block finishes handling all its pending call, the current block can exit retry + // handling mode and unblock normal calls. + current.finishRetryHandling(retryHandlingCondition); + } + }); + } + /** * It performs following actions : * a. Updates the committed length at datanode for the current stream in @@ -290,8 +356,15 @@ private int writeToOutputStream(BlockOutputStreamEntry current, * @param exception actual exception that occurred * @throws IOException Throws IOException if Write fails */ - private synchronized void handleException(BlockOutputStreamEntry streamEntry, - IOException exception) throws IOException { + private void handleExceptionInternal(BlockOutputStreamEntry streamEntry, IOException exception) throws IOException { + try { + // Wait for all pending flushes in the faulty stream. It's possible that a prior write is pending completion + // successfully. Errors are ignored here and will be handled by the individual flush call. We just want to ensure + // all the pending are complete before handling exception. + streamEntry.waitForAllPendingFlushes(); + } catch (IOException ignored) { + } + Throwable t = HddsClientUtils.checkForException(exception); Preconditions.checkNotNull(t); boolean retryFailure = checkForRetryFailure(t); @@ -318,8 +391,6 @@ private synchronized void handleException(BlockOutputStreamEntry streamEntry, } Preconditions.checkArgument( bufferedDataLen <= streamBufferArgs.getStreamBufferMaxSize()); - Preconditions.checkArgument( - offset - blockOutputStreamEntryPool.getKeyLength() == bufferedDataLen); long containerId = streamEntry.getBlockID().getContainerID(); Collection failedServers = streamEntry.getFailedServers(); Preconditions.checkNotNull(failedServers); @@ -441,40 +512,51 @@ protected boolean checkIfContainerToExclude(Throwable t) { @Override public void flush() throws IOException { - checkNotClosed(); - handleFlushOrClose(StreamAction.FLUSH); + try { + getRequestSemaphore().acquire(); + checkNotClosed(); + handleFlushOrClose(StreamAction.FLUSH); + } finally { + getRequestSemaphore().release(); + } } @Override public void hflush() throws IOException { + // Note: Semaphore acquired and released inside hsync(). hsync(); } @Override public void hsync() throws IOException { - if (replication.getReplicationType() != ReplicationType.RATIS) { - throw new UnsupportedOperationException( - "Replication type is not " + ReplicationType.RATIS); - } - if (replication.getRequiredNodes() <= 1) { - throw new UnsupportedOperationException("The replication factor = " - + replication.getRequiredNodes() + " <= 1"); - } - if (ozoneManagerVersion.compareTo(OzoneManagerVersion.HBASE_SUPPORT) < 0) { - throw new UnsupportedOperationException("Hsync API requires OM version " - + OzoneManagerVersion.HBASE_SUPPORT + " or later. Current OM version " - + ozoneManagerVersion); - } - checkNotClosed(); - final long hsyncPos = writeOffset; - - handleFlushOrClose(StreamAction.HSYNC); + try { + getRequestSemaphore().acquire(); - synchronized (this) { - Preconditions.checkState(offset >= hsyncPos, - "offset = %s < hsyncPos = %s", offset, hsyncPos); - MetricUtil.captureLatencyNs(clientMetrics::addHsyncLatency, - () -> blockOutputStreamEntryPool.hsyncKey(hsyncPos)); + if (replication.getReplicationType() != ReplicationType.RATIS) { + throw new UnsupportedOperationException( + "Replication type is not " + ReplicationType.RATIS); + } + if (replication.getRequiredNodes() <= 1) { + throw new UnsupportedOperationException("The replication factor = " + + replication.getRequiredNodes() + " <= 1"); + } + if (ozoneManagerVersion.compareTo(OzoneManagerVersion.HBASE_SUPPORT) < 0) { + throw new UnsupportedOperationException("Hsync API requires OM version " + + OzoneManagerVersion.HBASE_SUPPORT + " or later. Current OM version " + + ozoneManagerVersion); + } + checkNotClosed(); + final long hsyncPos = writeOffset; + handleFlushOrClose(StreamAction.HSYNC); + + doInWriteLock(() -> { + Preconditions.checkState(offset >= hsyncPos, + "offset = %s < hsyncPos = %s", offset, hsyncPos); + MetricUtil.captureLatencyNs(clientMetrics::addHsyncLatency, + () -> blockOutputStreamEntryPool.hsyncKey(hsyncPos)); + }); + } finally { + getRequestSemaphore().release(); } } @@ -501,14 +583,23 @@ private void handleFlushOrClose(StreamAction op) throws IOException { BlockOutputStreamEntry entry = blockOutputStreamEntryPool.getCurrentStreamEntry(); if (entry != null) { + // If the current block is to handle retries, wait until all the retries are done. + waitForRetryHandling(entry); + entry.registerCallReceived(); try { handleStreamAction(entry, op); + entry.registerCallFinished(); } catch (IOException ioe) { - handleException(entry, ioe); + handleException(entry, ioe, false); continue; + } catch (Exception e) { + entry.registerCallFinished(); + throw e; } } return; + } catch (InterruptedException e) { + throw new InterruptedIOException(); } catch (Exception e) { markStreamClosed(); throw e; @@ -517,6 +608,10 @@ private void handleFlushOrClose(StreamAction op) throws IOException { } } + private void waitForRetryHandling(BlockOutputStreamEntry currentEntry) throws InterruptedException { + doInWriteLock(() -> currentEntry.waitForRetryHandling(retryHandlingCondition)); + } + private void handleStreamAction(BlockOutputStreamEntry entry, StreamAction op) throws IOException { Collection failedServers = entry.getFailedServers(); @@ -552,7 +647,11 @@ private void handleStreamAction(BlockOutputStreamEntry entry, * @throws IOException */ @Override - public synchronized void close() throws IOException { + public void close() throws IOException { + doInWriteLock(this::closeInternal); + } + + private void closeInternal() throws IOException { if (closed) { return; } @@ -750,7 +849,7 @@ public KeyOutputStream build() { * the last state of the volatile {@link #closed} field. * @throws IOException if the connection is closed. */ - private synchronized void checkNotClosed() throws IOException { + private void checkNotClosed() throws IOException { if (closed) { throw new IOException( ": " + FSExceptionMessages.STREAM_IS_CLOSED + " Key: " diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStreamSemaphore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStreamSemaphore.java new file mode 100644 index 00000000000..36031a9cf4d --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStreamSemaphore.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.client.io; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.concurrent.Semaphore; + +/** + * Helper class that streamlines request semaphore usage in KeyOutputStream. + */ +public class KeyOutputStreamSemaphore { + + public static final Logger LOG = LoggerFactory.getLogger(KeyOutputStreamSemaphore.class); + private final Semaphore requestSemaphore; + + KeyOutputStreamSemaphore(int maxConcurrentWritePerKey) { + LOG.info("Initializing semaphore with maxConcurrentWritePerKey = {}", maxConcurrentWritePerKey); + if (maxConcurrentWritePerKey > 0) { + requestSemaphore = new Semaphore(maxConcurrentWritePerKey); + } else if (maxConcurrentWritePerKey == 0) { + throw new IllegalArgumentException("Invalid config. ozone.client.key.write.concurrency cannot be set to 0"); + } else { + requestSemaphore = null; + } + } + + public int getQueueLength() { + return requestSemaphore != null ? requestSemaphore.getQueueLength() : 0; + } + + public void acquire() throws IOException { + if (requestSemaphore != null) { + try { + LOG.debug("Acquiring semaphore"); + requestSemaphore.acquire(); + LOG.debug("Acquired semaphore"); + } catch (InterruptedException e) { + final String errMsg = "Write aborted. Interrupted waiting for KeyOutputStream semaphore: " + e.getMessage(); + LOG.error(errMsg); + Thread.currentThread().interrupt(); + throw new InterruptedIOException(errMsg); + } + } + } + + public void release() { + if (requestSemaphore != null) { + LOG.debug("Releasing semaphore"); + requestSemaphore.release(); + LOG.debug("Released semaphore"); + } + } +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java index 68812a7eb4d..16211ebbb8e 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.io.Text; import org.apache.hadoop.ozone.OzoneAcl; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.client.BucketArgs; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneKey; @@ -858,6 +859,13 @@ TenantUserList listUsersInTenant(String tenantId, String prefix) */ TenantStateList listTenant() throws IOException; + /** + * Get server default values for a number of configuration params. + * @return Default configuration from the server. + * @throws IOException + */ + OzoneFsServerDefaults getServerDefaults() throws IOException; + /** * Get KMS client provider. * @return KMS client provider. diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 8445f9e6954..bfeb9c1e6c1 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -34,6 +34,7 @@ import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.Syncable; +import org.apache.hadoop.util.Time; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; @@ -61,6 +62,7 @@ import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.io.Text; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; @@ -178,6 +180,8 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_KEY_PROVIDER_CACHE_EXPIRY; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_REQUIRED_OM_VERSION_MIN_KEY; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT; import static org.apache.hadoop.ozone.OzoneConsts.MAXIMUM_NUMBER_OF_PARTS_PER_UPLOAD; import static org.apache.hadoop.ozone.OzoneConsts.OLD_QUOTA_DEFAULT; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_MAXIMUM_ACCESS_ID_LENGTH; @@ -224,6 +228,9 @@ public class RpcClient implements ClientProtocol { private final ContainerClientMetrics clientMetrics; private final MemoizedSupplier writeExecutor; private final AtomicBoolean isS3GRequest = new AtomicBoolean(false); + private volatile OzoneFsServerDefaults serverDefaults; + private volatile long serverDefaultsLastUpdate; + private final long serverDefaultsValidityPeriod; /** * Creates RpcClient instance with the given configuration. @@ -330,6 +337,11 @@ public void onRemoval( .getInstance(byteBufferPool, ecReconstructExecutor); this.clientMetrics = ContainerClientMetrics.acquire(); + this.serverDefaultsValidityPeriod = conf.getTimeDuration( + OZONE_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS, + OZONE_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT, + TimeUnit.MILLISECONDS); + TracingUtil.initTracing("client", conf); } @@ -2591,11 +2603,22 @@ public KeyProvider call() throws Exception { } } + @Override + public OzoneFsServerDefaults getServerDefaults() throws IOException { + long now = Time.monotonicNow(); + if ((serverDefaults == null) || + (now - serverDefaultsLastUpdate > serverDefaultsValidityPeriod)) { + serverDefaults = ozoneManagerClient.getServerDefaults(); + serverDefaultsLastUpdate = now; + } + assert serverDefaults != null; + return serverDefaults; + } + @Override public URI getKeyProviderUri() throws IOException { - // TODO: fix me to support kms instances for difference OMs return OzoneKMSUtil.getKeyProviderUri(ugi, - null, null, conf); + null, getServerDefaults().getKeyProviderUri(), conf); } @Override diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java index 0d82f0f8bbb..b14582c8ea9 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java @@ -187,11 +187,6 @@ public ReplicationType getPipelineType() { return pipeline.getType(); } - @Override - public XceiverClientReply watchForCommit(long index) { - return null; - } - @Override public long getReplicatedMinCommitIndex() { return 0; diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestKeyOutputStream.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestKeyOutputStream.java new file mode 100644 index 00000000000..d90a335321b --- /dev/null +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestKeyOutputStream.java @@ -0,0 +1,126 @@ +/** + * 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.client.io; + +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.event.Level; + +import java.io.IOException; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests KeyOutputStream. + * This is a unit test meant to verify specific behaviors of KeyOutputStream. + */ +public class TestKeyOutputStream { + + @BeforeAll + static void init() { + GenericTestUtils.setLogLevel(KeyOutputStreamSemaphore.LOG, Level.DEBUG); + } + + @Test + void testConcurrentWriteLimitOne() throws Exception { + // Verify the semaphore is working to limit the number of concurrent writes allowed. + KeyOutputStreamSemaphore sema1 = new KeyOutputStreamSemaphore(1); + KeyOutputStream keyOutputStream = spy(KeyOutputStream.class); + when(keyOutputStream.getRequestSemaphore()).thenReturn(sema1); + + final AtomicInteger countWrite = new AtomicInteger(0); + // mock write() + doAnswer(invocation -> { + countWrite.getAndIncrement(); + return invocation.callRealMethod(); + }).when(keyOutputStream).write(any(), anyInt(), anyInt()); + + final ConcurrentHashMap mapNotifiers = new ConcurrentHashMap<>(); + + final AtomicInteger countHandleWrite = new AtomicInteger(0); + // mock handleWrite() + doAnswer(invocation -> { + final long tid = Thread.currentThread().getId(); + System.out.println("handleWrite() called from tid " + tid); + final CountDownLatch latch = mapNotifiers.compute(tid, (k, v) -> + v != null ? v : new CountDownLatch(1)); + countHandleWrite.getAndIncrement(); + // doing some "work" + latch.await(); + return null; + }).when(keyOutputStream).handleWrite(any(), anyInt(), anyLong(), anyBoolean()); + + final Runnable writeRunnable = () -> { + try { + keyOutputStream.write(new byte[4], 0, 4); + } catch (IOException e) { + throw new RuntimeException(e); + } + }; + + final Thread thread1 = new Thread(writeRunnable); + thread1.start(); + + final Thread thread2 = new Thread(writeRunnable); + thread2.start(); + + // Wait for both threads to enter write() + GenericTestUtils.waitFor(() -> countWrite.get() == 2, 100, 3000); + // One thread should enter handleWrite() + GenericTestUtils.waitFor(() -> countHandleWrite.get() == 1, 100, 3000); + // The other thread is waiting on the semaphore + GenericTestUtils.waitFor(() -> sema1.getQueueLength() == 1, 100, 3000); + + // handleWrite is triggered only once because of the semaphore and the synchronized block + verify(keyOutputStream, times(1)).handleWrite(any(), anyInt(), anyLong(), anyBoolean()); + + // Now, allow the current thread to finish handleWrite + // There is only one thread in handleWrite() so mapNotifiers should have only one entry. + assertEquals(1, mapNotifiers.size()); + Entry entry = mapNotifiers.entrySet().stream().findFirst().get(); + mapNotifiers.remove(entry.getKey()); + entry.getValue().countDown(); + + // Wait for the other thread to proceed + GenericTestUtils.waitFor(() -> countHandleWrite.get() == 2, 100, 3000); + verify(keyOutputStream, times(2)).handleWrite(any(), anyInt(), anyLong(), anyBoolean()); + + // Allow the other thread to finish handleWrite + entry = mapNotifiers.entrySet().stream().findFirst().get(); + mapNotifiers.remove(entry.getKey()); + entry.getValue().countDown(); + + // Let threads finish + thread2.join(); + thread1.join(); + } +} diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml index 15d961e0949..bd16a0a5dfe 100644 --- a/hadoop-ozone/common/pom.xml +++ b/hadoop-ozone/common/pom.xml @@ -61,21 +61,105 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-client + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-interface-client + org.apache.ozone ozone-interface-client + + + org.apache.commons + commons-lang3 + org.apache.hadoop hadoop-hdfs-client + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-thirdparty-misc + + + org.slf4j + slf4j-api + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + com.github.stephenc.jcip jcip-annotations + + com.google.guava + guava + + + com.google.protobuf + protobuf-java + + + io.grpc + grpc-api + + + com.google.code.findbugs + jsr305 + + + + + io.grpc + grpc-stub + + + io.netty + netty-common + + + io.netty + netty-handler + + + jakarta.annotation + jakarta.annotation-api + + + io.grpc + grpc-inprocess + test + io.grpc grpc-testing diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java index 82030669c93..11f176362a6 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java @@ -274,6 +274,7 @@ public static boolean isReadOnly( case SetSafeMode: case PrintCompactionLogDag: case GetSnapshotInfo: + case GetServerDefaults: return true; case CreateVolume: case SetVolumeProperty: @@ -327,6 +328,7 @@ public static boolean isReadOnly( case SetTimes: case AbortExpiredMultiPartUploads: case SetSnapshotProperty: + case QuotaRepair: case UnknownCommand: return false; case EchoRPC: diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OzoneFsServerDefaults.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OzoneFsServerDefaults.java new file mode 100644 index 00000000000..782fa88e8d2 --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OzoneFsServerDefaults.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone; + +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.hdds.annotation.InterfaceAudience; +import org.apache.hadoop.hdds.annotation.InterfaceStability; + +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.FsServerDefaultsProto; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.FsServerDefaultsProto.Builder; + + +/**************************************************** + * Provides server default configuration values to clients. + * + ****************************************************/ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class OzoneFsServerDefaults extends FsServerDefaults { + + public OzoneFsServerDefaults() { + } + + public OzoneFsServerDefaults(String keyProviderUri) { + super(0L, 0, 0, (short)0, 0, false, 0L, null, keyProviderUri); + } + + public FsServerDefaultsProto getProtobuf() { + Builder builder = FsServerDefaultsProto.newBuilder(); + if (getKeyProviderUri() != null) { + builder.setKeyProviderUri(getKeyProviderUri()); + } + return builder.build(); + } + + public static OzoneFsServerDefaults getFromProtobuf( + FsServerDefaultsProto serverDefaults) { + String keyProviderUri = null; + if (serverDefaults.hasKeyProviderUri()) { + keyProviderUri = serverDefaults.getKeyProviderUri(); + } + return new OzoneFsServerDefaults(keyProviderUri); + } +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java index 5f592663dad..45922c107cb 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.SafeModeAction; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.ozone.OzoneAcl; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.exceptions.OMException; @@ -1178,4 +1179,12 @@ void setTimes(OmKeyArgs keyArgs, long mtime, long atime) */ boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException; + + /** + * Get server default configurations. + * + * @return OzoneFsServerDefaults some default configurations from server. + * @throws IOException + */ + OzoneFsServerDefaults getServerDefaults() throws IOException; } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java index 9a965b7c3d0..f70beed5f25 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java @@ -41,6 +41,7 @@ import org.apache.hadoop.ipc.CallerContext; import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.OzoneAcl; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BasicOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.ErrorInfo; @@ -197,6 +198,8 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3Authentication; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3Secret; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SafeMode; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServerDefaultsRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServerDefaultsResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetAclRequest; @@ -2644,6 +2647,22 @@ public boolean setSafeMode(SafeModeAction action, boolean isChecked) return setSafeModeResponse.getResponse(); } + @Override + public OzoneFsServerDefaults getServerDefaults() + throws IOException { + ServerDefaultsRequest serverDefaultsRequest = + ServerDefaultsRequest.newBuilder().build(); + + OMRequest omRequest = createOMRequest(Type.GetServerDefaults) + .setServerDefaultsRequest(serverDefaultsRequest).build(); + + ServerDefaultsResponse serverDefaultsResponse = + handleError(submitRequest(omRequest)).getServerDefaultsResponse(); + + return OzoneFsServerDefaults.getFromProtobuf( + serverDefaultsResponse.getServerDefaults()); + } + private SafeMode toProtoBuf(SafeModeAction action) { switch (action) { case ENTER: diff --git a/hadoop-ozone/csi/pom.xml b/hadoop-ozone/csi/pom.xml index 04c153f3988..a0565d7e890 100644 --- a/hadoop-ozone/csi/pom.xml +++ b/hadoop-ozone/csi/pom.xml @@ -33,6 +33,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ozone + hdds-common + com.google.protobuf protobuf-java-util @@ -70,6 +74,15 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-server-framework + + org.apache.ozone + ozone-common + + + + commons-io + commons-io + com.google.code.findbugs jsr305 @@ -90,6 +103,14 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> io.grpc grpc-netty + + io.netty + netty-transport + + + io.netty + netty-transport-classes-epoll + io.netty netty-transport-native-epoll @@ -111,10 +132,24 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> ch.qos.reload4j reload4j + + org.slf4j + slf4j-api + org.slf4j slf4j-reload4j + + io.grpc + grpc-api + + + com.google.code.findbugs + jsr305 + + + io.grpc grpc-protobuf @@ -133,6 +168,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> io.grpc grpc-stub + + info.picocli + picocli + org.apache.ozone ozone-client diff --git a/hadoop-ozone/dev-support/checks/acceptance.sh b/hadoop-ozone/dev-support/checks/acceptance.sh index 1e16b277aff..3425f66605e 100755 --- a/hadoop-ozone/dev-support/checks/acceptance.sh +++ b/hadoop-ozone/dev-support/checks/acceptance.sh @@ -52,11 +52,6 @@ if [[ "${OZONE_ACCEPTANCE_SUITE}" == "s3a" ]]; then download_hadoop_aws "${HADOOP_AWS_DIR}" fi -if [[ "${OZONE_ACCEPTANCE_TEST_TYPE}" == "robot" ]]; then - install_virtualenv - install_robot -fi - export OZONE_ACCEPTANCE_SUITE OZONE_ACCEPTANCE_TEST_TYPE cd "$DIST_DIR/compose" || exit 1 diff --git a/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Datanode Chunk Read_Write Dashboard.json b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Datanode Chunk Read_Write Dashboard.json index 104382f9850..44749f902cf 100644 --- a/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Datanode Chunk Read_Write Dashboard.json +++ b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Datanode Chunk Read_Write Dashboard.json @@ -1,7 +1,6 @@ { "__inputs": [ { - "name": "DS_PROMETHEUS", "label": "prometheus", "description": "", "type": "datasource", @@ -15,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "10.4.2" + "version": "11.1.4" }, { "type": "datasource", @@ -65,8 +64,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -145,15 +143,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(volume_io_stats_read_bytes[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(volume_io_stats_read_bytes[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{label_name}} Volume={{storagedirectory}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Data read per Volume", @@ -161,8 +162,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -240,15 +240,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(volume_io_stats_read_op_count[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(volume_io_stats_read_op_count[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}} Volume={{storagedirectory}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Read Ops", @@ -256,8 +259,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -336,15 +338,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(volume_io_stats_write_bytes[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(volume_io_stats_write_bytes[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{label_name}} Volume={{storagedirectory}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Data write per Volume", @@ -352,8 +357,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -431,15 +435,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(volume_io_stats_write_op_count[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(volume_io_stats_write_op_count[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}} Volume={{storagedirectory}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Write Ops", @@ -460,8 +467,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -540,15 +546,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(storage_container_metrics_bytes_write_chunk[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(storage_container_metrics_bytes_write_chunk[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Write Chunk Traffic", @@ -556,8 +565,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -636,15 +644,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "sum(rate(storage_container_metrics_bytes_write_chunk[$__rate_interval]))", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "sum(rate(storage_container_metrics_bytes_write_chunk[$__interval]))", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Total Write Chunk Traffic", @@ -652,8 +663,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -731,15 +741,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(storage_container_metrics_num_write_chunk[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(storage_container_metrics_num_write_chunk[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Write Chunks Ops", @@ -747,8 +760,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -826,15 +838,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(storage_container_metrics_bytes_put_block[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(storage_container_metrics_bytes_put_block[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "__auto", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Put Blocks Ops", @@ -855,8 +870,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -935,15 +949,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "rate(storage_container_metrics_bytes_read_chunk[$__rate_interval])", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rate(storage_container_metrics_bytes_read_chunk[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Read Chunk Traffic", @@ -951,8 +968,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -1031,15 +1047,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, - "editorMode": "code", - "expr": "sum(rate(storage_container_metrics_bytes_read_chunk[$__rate_interval]))", + "disableTextWrap": false, + "editorMode": "builder", + "expr": "sum(rate(storage_container_metrics_bytes_read_chunk[$__interval]))", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Total Read Chunk Traffic", @@ -1047,8 +1066,7 @@ }, { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, "fieldConfig": { "defaults": { @@ -1126,15 +1144,18 @@ "targets": [ { "datasource": { - "type": "prometheus", - "uid": "${DS_PROMETHEUS}" + "type": "prometheus" }, + "disableTextWrap": false, "editorMode": "builder", - "expr": "rate(storage_container_metrics_num_read_chunk[$__rate_interval])", + "expr": "rate(storage_container_metrics_num_read_chunk[$__interval])", + "fullMetaSearch": false, + "includeNullMetadata": true, "instant": false, "legendFormat": "Datanode={{hostname}}", "range": true, - "refId": "A" + "refId": "A", + "useBackend": false } ], "title": "Read Chunks Ops", @@ -1148,13 +1169,13 @@ "list": [] }, "time": { - "from": "now-3h", + "from": "now-7d", "to": "now" }, "timepicker": {}, "timezone": "browser", "title": "Datanode Chunk Read/Write Dashboard", "uid": "edj2lc6lfn5s0a", - "version": 15, + "version": 4, "weekStart": "" } \ No newline at end of file diff --git a/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json index 78b027afd2e..3ec057b205f 100644 --- a/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json +++ b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json @@ -15,7 +15,6 @@ "editable": true, "fiscalYearStartMonth": 0, "graphTooltip": 0, - "id": null, "links": [], "liveNow": false, "panels": [ @@ -27,6 +26,308 @@ "x": 0, "y": 0 }, + "id": 65, + "panels": [], + "title": "OM Operations", + "type": "row" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 1 + }, + "id": 68, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "rate(om_metrics_num_key_commits[1m])", + "fullMetaSearch": false, + "includeNullMetadata": false, + "instant": false, + "legendFormat": "{{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Rate of Key Commits", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 1 + }, + "id": 67, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "rate(om_metrics_ec_key_create_fails_total[1m])", + "fullMetaSearch": false, + "includeNullMetadata": false, + "instant": false, + "legendFormat": "{{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Rate of Key Commit Failures", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "description": "Rate at which data is committed across objects in OM. ", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 9 + }, + "id": 66, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "editorMode": "code", + "expr": "rate(om_metrics_total_data_committed[60s])", + "instant": false, + "legendFormat": "{{hostname}}", + "range": true, + "refId": "A" + } + ], + "title": "Aggregate Rate of Data Commit", + "type": "timeseries" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 17 + }, "id": 52, "panels": [ { @@ -75,8 +376,7 @@ "mode": "absolute", "steps": [ { - "color": "green", - "value": null + "color": "green" }, { "color": "red", @@ -117,7 +417,7 @@ "h": 7, "w": 8, "x": 0, - "y": 1 + "y": 25 }, "id": 11, "options": { @@ -199,8 +499,7 @@ "mode": "absolute", "steps": [ { - "color": "green", - "value": null + "color": "green" }, { "color": "red", @@ -215,7 +514,7 @@ "h": 7, "w": 8, "x": 8, - "y": 1 + "y": 25 }, "id": 12, "options": { @@ -261,7 +560,7 @@ "h": 1, "w": 24, "x": 0, - "y": 1 + "y": 18 }, "id": 51, "panels": [ @@ -275,6 +574,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "no. of keys", @@ -325,7 +625,7 @@ "h": 7, "w": 8, "x": 0, - "y": 2 + "y": 26 }, "id": 13, "options": { @@ -386,7 +686,7 @@ "h": 1, "w": 24, "x": 0, - "y": 2 + "y": 19 }, "id": 50, "panels": [ @@ -400,6 +700,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "time (ns)", @@ -476,7 +777,7 @@ "h": 8, "w": 8, "x": 0, - "y": 3 + "y": 27 }, "id": 15, "options": { @@ -531,6 +832,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "no. of ops", @@ -581,7 +883,7 @@ "h": 8, "w": 8, "x": 8, - "y": 3 + "y": 27 }, "id": 14, "options": { @@ -658,7 +960,7 @@ "h": 1, "w": 24, "x": 0, - "y": 3 + "y": 20 }, "id": 8, "panels": [ @@ -673,6 +975,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "", @@ -748,7 +1051,7 @@ "h": 8, "w": 8, "x": 0, - "y": 4 + "y": 28 }, "id": 16, "options": { @@ -793,6 +1096,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "", @@ -843,7 +1147,7 @@ "h": 8, "w": 8, "x": 8, - "y": 4 + "y": 28 }, "id": 17, "options": { @@ -904,7 +1208,7 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 21 }, "id": 64, "panels": [ @@ -918,6 +1222,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "no. of ops", @@ -993,7 +1298,7 @@ "h": 8, "w": 8, "x": 0, - "y": 13 + "y": 29 }, "id": 10, "options": { @@ -1054,7 +1359,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 22 }, "id": 63, "panels": [ @@ -1068,6 +1373,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "no. of keys", @@ -1143,7 +1449,7 @@ "h": 8, "w": 8, "x": 0, - "y": 1 + "y": 30 }, "id": 49, "options": { @@ -1189,7 +1495,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 23 }, "id": 53, "panels": [ @@ -1253,7 +1559,7 @@ "h": 7, "w": 8, "x": 0, - "y": 20 + "y": 44 }, "id": 9, "options": { @@ -1410,7 +1716,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 24 }, "id": 54, "panels": [ @@ -1424,6 +1730,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "", @@ -1474,7 +1781,7 @@ "h": 8, "w": 8, "x": 0, - "y": 21 + "y": 32 }, "id": 48, "options": { @@ -1583,6 +1890,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "", @@ -1658,7 +1966,7 @@ "h": 8, "w": 8, "x": 8, - "y": 21 + "y": 32 }, "id": 24, "options": { @@ -1719,6 +2027,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "Time (ns)", @@ -1770,7 +2079,7 @@ "h": 8, "w": 8, "x": 16, - "y": 21 + "y": 32 }, "id": 45, "options": { @@ -1879,7 +2188,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 25 }, "id": 56, "panels": [ @@ -1943,7 +2252,7 @@ "h": 8, "w": 8, "x": 0, - "y": 9 + "y": 33 }, "id": 29, "options": { @@ -2038,7 +2347,7 @@ "h": 8, "w": 8, "x": 8, - "y": 9 + "y": 33 }, "id": 30, "options": { @@ -2083,7 +2392,7 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 26 }, "id": 7, "panels": [ @@ -2147,7 +2456,7 @@ "h": 8, "w": 8, "x": 0, - "y": 8 + "y": 32 }, "id": 26, "options": { @@ -2259,7 +2568,7 @@ "h": 8, "w": 8, "x": 8, - "y": 8 + "y": 32 }, "id": 25, "options": { @@ -2320,7 +2629,7 @@ "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 27 }, "id": 57, "panels": [ @@ -2384,7 +2693,7 @@ "h": 7, "w": 8, "x": 0, - "y": 9 + "y": 33 }, "id": 34, "options": { @@ -2495,7 +2804,7 @@ "h": 7, "w": 8, "x": 8, - "y": 9 + "y": 33 }, "id": 32, "options": { @@ -2572,7 +2881,7 @@ "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 28 }, "id": 59, "panels": [ @@ -2636,7 +2945,7 @@ "h": 7, "w": 8, "x": 0, - "y": 11 + "y": 35 }, "id": 37, "options": { @@ -2731,7 +3040,7 @@ "h": 7, "w": 8, "x": 8, - "y": 11 + "y": 35 }, "id": 36, "options": { @@ -2776,7 +3085,7 @@ "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 29 }, "id": 60, "panels": [ @@ -2840,7 +3149,7 @@ "h": 8, "w": 8, "x": 0, - "y": 12 + "y": 36 }, "id": 33, "options": { @@ -2885,7 +3194,7 @@ "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 30 }, "id": 40, "panels": [ @@ -2899,6 +3208,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "no. of ops", @@ -2949,7 +3259,7 @@ "h": 7, "w": 8, "x": 0, - "y": 13 + "y": 38 }, "id": 44, "options": { @@ -3026,7 +3336,7 @@ "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 31 }, "id": 61, "panels": [ @@ -3040,6 +3350,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "time (ns)", @@ -3091,7 +3402,7 @@ "h": 8, "w": 8, "x": 0, - "y": 14 + "y": 39 }, "id": 41, "options": { @@ -3152,6 +3463,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "no. of ops", @@ -3202,7 +3514,7 @@ "h": 8, "w": 12, "x": 8, - "y": 14 + "y": 39 }, "id": 42, "options": { @@ -3247,7 +3559,7 @@ "h": 1, "w": 24, "x": 0, - "y": 16 + "y": 32 }, "id": 62, "panels": [ @@ -3261,6 +3573,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "Time (ns)", @@ -3312,7 +3625,7 @@ "h": 9, "w": 8, "x": 0, - "y": 15 + "y": 40 }, "id": 46, "options": { @@ -3421,6 +3734,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "", @@ -3496,7 +3810,7 @@ "h": 9, "w": 8, "x": 8, - "y": 15 + "y": 40 }, "id": 47, "options": { @@ -3557,6 +3871,7 @@ "mode": "palette-classic" }, "custom": { + "axisBorderShow": false, "axisCenteredZero": false, "axisColorMode": "text", "axisLabel": "", @@ -3607,7 +3922,7 @@ "h": 9, "w": 8, "x": 16, - "y": 15 + "y": 40 }, "id": 22, "options": { @@ -3718,13 +4033,12 @@ "list": [] }, "time": { - "from": "now-12h", + "from": "now-15m", "to": "now" }, "timeRangeUpdatedDuringEditOrView": false, "timepicker": {}, "timezone": "", "title": "Create key Dashboard", - "version": 36, "weekStart": "" } \ No newline at end of file diff --git a/hadoop-ozone/dist/src/main/compose/testlib.sh b/hadoop-ozone/dist/src/main/compose/testlib.sh index 44b4f303807..1e9cc85781a 100755 --- a/hadoop-ozone/dist/src/main/compose/testlib.sh +++ b/hadoop-ozone/dist/src/main/compose/testlib.sh @@ -37,8 +37,6 @@ create_results_dir() { #delete previous results [[ "${OZONE_KEEP_RESULTS:-}" == "true" ]] || rm -rf "$RESULT_DIR" mkdir -p "$RESULT_DIR" - #Should be writeable from the docker containers where user is different. - chmod ogu+w "$RESULT_DIR" } ## @description find all the test*.sh scripts in the immediate child dirs @@ -390,22 +388,44 @@ cleanup_docker_images() { fi } +## @description Run Robot Framework report generator (rebot) in ozone-runner container. +## @param input directory where source Robot XML files are +## @param output directory where report should be placed +## @param rebot options and arguments +run_rebot() { + local input_dir="$(realpath "$1")" + local output_dir="$(realpath "$2")" + + shift 2 + + local tempdir="$(mktemp -d --suffix rebot -p "${output_dir}")" + #Should be writeable from the docker containers where user is different. + chmod a+wx "${tempdir}" + if docker run --rm -v "${input_dir}":/rebot-input -v "${tempdir}":/rebot-output -w /rebot-input \ + $(get_runner_image_spec) \ + bash -c "rebot --nostatusrc -d /rebot-output $@"; then + mv -v "${tempdir}"/* "${output_dir}"/ + fi + rmdir "${tempdir}" +} + ## @description Generate robot framework reports based on the saved results. generate_report(){ local title="${1:-${COMPOSE_ENV_NAME}}" local dir="${2:-${RESULT_DIR}}" local xunitdir="${3:-}" - if command -v rebot > /dev/null 2>&1; then - #Generate the combined output and return with the right exit code (note: robot = execute test, rebot = generate output) - if [ -z "${xunitdir}" ]; then - rebot --reporttitle "${title}" -N "${title}" -d "${dir}" "${dir}/*.xml" - else - rebot --reporttitle "${title}" -N "${title}" --xunit ${xunitdir}/TEST-ozone.xml -d "${dir}" "${dir}/*.xml" - fi - else - echo "Robot framework is not installed, the reports cannot be generated (sudo pip install robotframework)." - exit 1 + if [[ -n "$(find "${dir}" -mindepth 1 -maxdepth 1 -name "*.xml")" ]]; then + xunit_args="" + if [[ -n "${xunitdir}" ]] && [[ -e "${xunitdir}" ]]; then + xunit_args="--xunit TEST-ozone.xml" + fi + + run_rebot "$dir" "$dir" "--reporttitle '${title}' -N '${title}' ${xunit_args} *.xml" + + if [[ -n "${xunit_args}" ]]; then + mv -v "${dir}"/TEST-ozone.xml "${xunitdir}"/ || rm -f "${dir}"/TEST-ozone.xml + fi fi } @@ -429,8 +449,8 @@ copy_results() { target_dir="${target_dir}/${test_script_name}" fi - if command -v rebot > /dev/null 2>&1 && [[ -n "$(find "${result_dir}" -name "*.xml")" ]]; then - rebot --nostatusrc -N "${test_name}" -l NONE -r NONE -o "${all_result_dir}/${test_name}.xml" "${result_dir}"/*.xml \ + if [[ -n "$(find "${result_dir}" -mindepth 1 -maxdepth 1 -name "*.xml")" ]]; then + run_rebot "${result_dir}" "${all_result_dir}" "-N '${test_name}' -l NONE -r NONE -o '${test_name}.xml' *.xml" \ && rm -fv "${result_dir}"/*.xml "${result_dir}"/log.html "${result_dir}"/report.html fi @@ -505,14 +525,21 @@ prepare_for_binary_image() { ## @description Define variables required for using `ozone-runner` docker image ## (no binaries included) ## @param `ozone-runner` image version (optional) -prepare_for_runner_image() { +get_runner_image_spec() { local default_version=${docker.ozone-runner.version} # set at build-time from Maven property local runner_version=${OZONE_RUNNER_VERSION:-${default_version}} # may be specified by user running the test local runner_image=${OZONE_RUNNER_IMAGE:-apache/ozone-runner} # may be specified by user running the test local v=${1:-${runner_version}} # prefer explicit argument + echo "${runner_image}:${v}" +} + +## @description Define variables required for using `ozone-runner` docker image +## (no binaries included) +## @param `ozone-runner` image version (optional) +prepare_for_runner_image() { export OZONE_DIR=/opt/hadoop - export OZONE_IMAGE="${runner_image}:${v}" + export OZONE_IMAGE="$(get_runner_image_spec "$@")" } ## @description Executing the Ozone Debug CLI related robot tests diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh index 9d7ec5d4e60..6fc4763631b 100755 --- a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh +++ b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh @@ -35,7 +35,7 @@ RESULT_DIR="$ALL_RESULT_DIR" create_results_dir # This is the version of Ozone that should use the runner image to run the # code that was built. Other versions will pull images from docker hub. -export OZONE_CURRENT_VERSION=1.5.0 +export OZONE_CURRENT_VERSION="${ozone.version}" run_test ha non-rolling-upgrade 1.4.0 "$OZONE_CURRENT_VERSION" # run_test ha non-rolling-upgrade 1.3.0 "$OZONE_CURRENT_VERSION" # run_test ha non-rolling-upgrade 1.2.1 "$OZONE_CURRENT_VERSION" diff --git a/hadoop-ozone/dist/src/main/compose/xcompat/test.sh b/hadoop-ozone/dist/src/main/compose/xcompat/test.sh index 419d397c19e..695d8bf06ab 100755 --- a/hadoop-ozone/dist/src/main/compose/xcompat/test.sh +++ b/hadoop-ozone/dist/src/main/compose/xcompat/test.sh @@ -21,7 +21,7 @@ COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" export COMPOSE_DIR basename=$(basename ${COMPOSE_DIR}) -current_version=1.5.0 +current_version="${ozone.version}" old_versions="1.0.0 1.1.0 1.2.1 1.3.0 1.4.0" # container is needed for each version in clients.yaml # shellcheck source=hadoop-ozone/dist/src/main/compose/testlib.sh @@ -77,7 +77,8 @@ test_cross_compatibility() { test_ec_cross_compatibility() { echo "Running Erasure Coded storage backward compatibility tests." - local cluster_versions_with_ec="1.3.0 1.4.0" + # local cluster_versions_with_ec="1.3.0 1.4.0 ${current_version}" + local cluster_versions_with_ec="${current_version}" # until HDDS-11334 local non_ec_client_versions="1.0.0 1.1.0 1.2.1" for cluster_version in ${cluster_versions_with_ec}; do diff --git a/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot index 83f0a1b69e2..719cdaf83f3 100644 --- a/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot +++ b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot @@ -92,18 +92,43 @@ Test ozone shell Test ozone shell errors [arguments] ${protocol} ${server} ${volume} - ${result} = Execute and checkrc ozone sh volume create ${protocol}${server}/${volume} --space-quota invalid 255 - Should contain ${result} invalid + ${result} = Execute and checkrc ozone sh volume create ${protocol}${server}/${volume} --space-quota 1.5GB 255 + Should contain ${result} 1.5GB is invalid + ${result} = Execute and checkrc ozone sh volume create ${protocol}${server}/${volume} --namespace-quota 1.5 255 + Should contain ${result} 1.5 is invalid Execute and checkrc ozone sh volume create ${protocol}${server}/${volume} 0 ${result} = Execute and checkrc ozone sh bucket create ${protocol}${server}/${volume}/bucket_1 255 Should contain ${result} INVALID_BUCKET_NAME + ${result} = Execute and checkrc ozone sh bucket create ${protocol}${server}/${volume}/bucket1 --space-quota 1.5GB 255 + Should contain ${result} 1.5GB is invalid + ${result} = Execute and checkrc ozone sh bucket create ${protocol}${server}/${volume}/bucket1 --namespace-quota 1.5 255 + Should contain ${result} 1.5 is invalid ${result} = Execute and checkrc ozone sh bucket create ${protocol}${server}/${volume}/bucket1 --layout Invalid 2 Should contain ${result} Usage Execute and checkrc ozone sh bucket create ${protocol}${server}/${volume}/bucket1 0 ${result} = Execute and checkrc ozone sh key info ${protocol}${server}/${volume}/bucket1/non-existing 255 Should contain ${result} KEY_NOT_FOUND ${result} = Execute and checkrc ozone sh key put ${protocol}${server}/${volume}/bucket1/key1 unexisting --type invalid 2 + ${result} = Execute and checkrc ozone sh bucket setquota ${volume}/bucket1 --space-quota 1.5 255 + Should contain ${result} 1.5 is invalid + ${result} = Execute and checkrc ozone sh bucket setquota ${volume}/bucket1 --namespace-quota 1.5 255 + Should contain ${result} 1.5 is invalid + ${result} = Execute and checkrc ozone sh volume setquota ${volume} --space-quota 1.5 255 + Should contain ${result} 1.5 is invalid + ${result} = Execute and checkrc ozone sh volume setquota ${volume} --namespace-quota 1.5 255 + Should contain ${result} 1.5 is invalid + Execute and checkrc ozone sh bucket setquota ${volume}/bucket1 --space-quota 2KB 0 + ${result} = Execute and checkrc ozone sh key put ${volume}/bucket1/key1 /opt/hadoop/NOTICE.txt 255 + Should contain ${result} QUOTA_EXCEEDED + ${result} = Execute and checkrc ozone sh volume setquota ${volume} --space-quota 1KB 255 + Should contain ${result} QUOTA_EXCEEDED + Execute and checkrc ozone sh bucket clrquota ${volume}/bucket1 --space-quota 0 + ${result} = Execute and checkrc ozone sh volume setquota ${volume} --space-quota 1GB 255 + Should contain ${result} QUOTA_ERROR Execute and checkrc ozone sh bucket delete ${protocol}${server}/${volume}/bucket1 0 + Execute and checkrc ozone sh volume setquota ${volume} --space-quota 1GB 0 + ${result} = Execute and checkrc ozone sh bucket create ${protocol}${server}/${volume}/bucket1 255 + Should contain ${result} QUOTA_ERROR Execute and checkrc ozone sh volume delete ${protocol}${server}/${volume} 0 diff --git a/hadoop-ozone/dist/src/shell/conf/om-audit-log4j2.properties b/hadoop-ozone/dist/src/shell/conf/om-audit-log4j2.properties index b9b11bb6289..40d02bae2c3 100644 --- a/hadoop-ozone/dist/src/shell/conf/om-audit-log4j2.properties +++ b/hadoop-ozone/dist/src/shell/conf/om-audit-log4j2.properties @@ -60,7 +60,7 @@ filter.write.onMismatch=NEUTRAL #appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n # Comment this line when using both console and rolling appenders -appenders=rolling +appenders=rolling,sysrolling # Rolling File Appender with size & time thresholds. # Rolling is triggered when either threshold is breached. @@ -88,13 +88,39 @@ appender.rolling.strategy.delete.ifFileName.glob=om-audit-*.log.gz appender.rolling.strategy.delete.ifLastModified.type=IfLastModified appender.rolling.strategy.delete.ifLastModified.age=30d -loggers=audit +appender.sysrolling.type=RollingFile +appender.sysrolling.name=SysRollingFile +appender.sysrolling.fileName =${sys:hadoop.log.dir}/om-sys-audit-${hostName}.log +appender.sysrolling.filePattern=${sys:hadoop.log.dir}/om-sys-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz +appender.sysrolling.layout.type=PatternLayout +appender.sysrolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n +appender.sysrolling.policies.type=Policies +appender.sysrolling.policies.time.type=TimeBasedTriggeringPolicy +appender.sysrolling.policies.time.interval=86400 +appender.sysrolling.policies.size.type=SizeBasedTriggeringPolicy +appender.sysrolling.policies.size.size=64MB +appender.sysrolling.strategy.type=DefaultRolloverStrategy +appender.sysrolling.strategy.delete.type=Delete +appender.sysrolling.strategy.delete.basePath=${sys:hadoop.log.dir} +appender.sysrolling.strategy.delete.maxDepth=1 +appender.sysrolling.strategy.delete.ifFileName.type=IfFileName +appender.sysrolling.strategy.delete.ifFileName.glob=om-sys-audit-*.log.gz +appender.sysrolling.strategy.delete.ifLastModified.type=IfLastModified +appender.sysrolling.strategy.delete.ifLastModified.age=30d + +loggers=audit,sysaudit logger.audit.type=AsyncLogger logger.audit.name=OMAudit logger.audit.level=INFO logger.audit.appenderRefs=rolling logger.audit.appenderRef.file.ref=RollingFile +logger.sysaudit.type=AsyncLogger +logger.sysaudit.name=OMSystemAudit +logger.sysaudit.level=INFO +logger.sysaudit.appenderRefs=sysrolling +logger.sysaudit.appenderRef.file.ref=SysRollingFile + rootLogger.level=INFO #rootLogger.appenderRefs=stdout #rootLogger.appenderRef.stdout.ref=STDOUT diff --git a/hadoop-ozone/dist/src/shell/ozone/ozone b/hadoop-ozone/dist/src/shell/ozone/ozone index 99fce8c2d44..22ceed9ed3c 100755 --- a/hadoop-ozone/dist/src/shell/ozone/ozone +++ b/hadoop-ozone/dist/src/shell/ozone/ozone @@ -250,6 +250,7 @@ function ozone_suppress_shell_log && [[ -z "${OZONE_ORIGINAL_ROOT_LOGGER}" ]]; then OZONE_LOGLEVEL=OFF OZONE_ROOT_LOGGER="${OZONE_LOGLEVEL},console" + OZONE_OPTS="${OZONE_OPTS} -Dslf4j.internal.verbosity=ERROR" fi } diff --git a/hadoop-ozone/httpfsgateway/pom.xml b/hadoop-ozone/httpfsgateway/pom.xml index 5be2a4be83f..7664643b153 100644 --- a/hadoop-ozone/httpfsgateway/pom.xml +++ b/hadoop-ozone/httpfsgateway/pom.xml @@ -40,6 +40,14 @@ + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + org.apache.ozone hdds-server-framework @@ -86,6 +94,10 @@ commons-codec runtime + + org.apache.commons + commons-lang3 + ch.qos.reload4j reload4j @@ -138,6 +150,18 @@ + + com.fasterxml.jackson.core + jackson-databind + + + com.google.guava + guava + + + jakarta.ws.rs + jakarta.ws.rs-api + jakarta.xml.bind jakarta.xml.bind-api @@ -146,6 +170,10 @@ org.glassfish.hk2 hk2-api + + org.glassfish.jersey.core + jersey-server + org.glassfish.jersey.inject jersey-hk2 diff --git a/hadoop-ozone/insight/pom.xml b/hadoop-ozone/insight/pom.xml index ae0bc5cd3aa..bcfb1660244 100644 --- a/hadoop-ozone/insight/pom.xml +++ b/hadoop-ozone/insight/pom.xml @@ -31,6 +31,26 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-container-service + + + org.apache.ozone + hdds-interface-admin + + + org.apache.ozone + hdds-interface-client + org.apache.ozone ozone-manager @@ -51,6 +71,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone ozone-filesystem + + org.apache.ozone + ozone-interface-client + org.apache.ozone hdds-server-framework @@ -59,6 +83,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-tools + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + info.picocli + picocli + jakarta.xml.bind jakarta.xml.bind-api diff --git a/hadoop-ozone/integration-test/pom.xml b/hadoop-ozone/integration-test/pom.xml index e1ba4af83db..f66f64d2874 100644 --- a/hadoop-ozone/integration-test/pom.xml +++ b/hadoop-ozone/integration-test/pom.xml @@ -238,6 +238,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ratis + ratis-server + + org.hamcrest hamcrest diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java index 466490290b7..49b515d53c5 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java @@ -32,6 +32,7 @@ import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Stream; @@ -44,6 +45,9 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.scm.ErrorInjector; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockOutputStream; import org.apache.hadoop.hdds.scm.storage.BufferPool; import org.apache.hadoop.hdds.utils.IOUtils; @@ -90,13 +94,15 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; -import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.ozone.om.service.OpenKeyCleanupService; -import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature; -import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Time; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.Message; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeerId; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.MethodOrderer.OrderAnnotation; @@ -121,8 +127,6 @@ import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME; import static org.apache.hadoop.ozone.TestDataUtil.cleanupDeletedTable; import static org.apache.hadoop.ozone.TestDataUtil.cleanupOpenKeyTable; -import static org.apache.hadoop.ozone.admin.scm.FinalizeUpgradeCommandUtil.isDone; -import static org.apache.hadoop.ozone.admin.scm.FinalizeUpgradeCommandUtil.isStarting; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY; @@ -130,9 +134,6 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_CLEANUP_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_LEASE_HARD_LIMIT; -import static org.apache.hadoop.ozone.om.OmUpgradeConfig.ConfigStrings.OZONE_OM_INIT_DEFAULT_LAYOUT_VERSION; -import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION; -import static org.apache.ozone.test.LambdaTestUtils.await; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertArrayEquals; @@ -170,9 +171,6 @@ public class TestHSync { private static OpenKeyCleanupService openKeyCleanupService; - private static final int POLL_INTERVAL_MILLIS = 500; - private static final int POLL_MAX_WAIT_MILLIS = 120_000; - @BeforeAll public static void init() throws Exception { final BucketLayout layout = BUCKET_LAYOUT; @@ -186,6 +184,8 @@ public static void init() throws Exception { CONF.setTimeDuration(OZONE_DIR_DELETING_SERVICE_INTERVAL, 100, TimeUnit.MILLISECONDS); CONF.setBoolean("ozone.client.incremental.chunk.list", true); CONF.setBoolean("ozone.client.stream.putblock.piggybacking", true); + // Unlimited key write concurrency + CONF.setInt("ozone.client.key.write.concurrency", -1); CONF.setTimeDuration(OZONE_OM_OPEN_KEY_CLEANUP_SERVICE_INTERVAL, SERVICE_INTERVAL, TimeUnit.MILLISECONDS); CONF.setTimeDuration(OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD, @@ -193,7 +193,6 @@ public static void init() throws Exception { CONF.setTimeDuration(OZONE_OM_LEASE_HARD_LIMIT, EXPIRE_THRESHOLD_MS, TimeUnit.MILLISECONDS); CONF.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); - CONF.setInt(OZONE_OM_INIT_DEFAULT_LAYOUT_VERSION, OMLayoutFeature.QUOTA.layoutVersion()); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) .setBlockSize(BLOCK_SIZE) @@ -226,9 +225,6 @@ public static void init() throws Exception { openKeyCleanupService = (OpenKeyCleanupService) cluster.getOzoneManager().getKeyManager().getOpenKeyCleanupService(); openKeyCleanupService.suspend(); - - preFinalizationChecks(); - finalizeOMUpgrade(); } @AfterAll @@ -239,72 +235,6 @@ public static void teardown() { } } - private static void preFinalizationChecks() throws IOException { - final String rootPath = String.format("%s://%s/", - OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); - CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); - - final String dir = OZONE_ROOT + bucket.getVolumeName() - + OZONE_URI_DELIMITER + bucket.getName(); - - final Path file = new Path(dir, "pre-finalization"); - try (RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(CONF)) { - try (FSDataOutputStream outputStream = fs.create(file, true)) { - OMException omException = assertThrows(OMException.class, outputStream::hsync); - assertFinalizationExceptionForHsyncLeaseRecovery(omException); - } - final OzoneManagerProtocol omClient = client.getObjectStore() - .getClientProxy().getOzoneManagerClient(); - OMException omException = assertThrows(OMException.class, - () -> omClient.listOpenFiles("", 100, "")); - assertFinalizationException(omException); - - omException = assertThrows(OMException.class, - () -> fs.recoverLease(file)); - assertFinalizationException(omException); - - fs.delete(file, false); - } - } - - private static void assertFinalizationExceptionForHsyncLeaseRecovery(OMException omException) { - assertEquals(NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION, - omException.getResult()); - assertThat(omException.getMessage()) - .contains("Cluster does not have the HBase support feature finalized yet"); - } - - private static void assertFinalizationException(OMException omException) { - assertEquals(NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION, - omException.getResult()); - assertThat(omException.getMessage()) - .contains("cannot be invoked before finalization."); - } - - /** - * Trigger OM upgrade finalization from the client and block until completion - * (status FINALIZATION_DONE). - */ - private static void finalizeOMUpgrade() throws Exception { - // Trigger OM upgrade finalization. Ref: FinalizeUpgradeSubCommand#call - final OzoneManagerProtocol omClient = client.getObjectStore() - .getClientProxy().getOzoneManagerClient(); - final String upgradeClientID = "Test-Upgrade-Client-" + UUID.randomUUID(); - UpgradeFinalizer.StatusAndMessages finalizationResponse = - omClient.finalizeUpgrade(upgradeClientID); - - // The status should transition as soon as the client call above returns - assertTrue(isStarting(finalizationResponse.status())); - // Wait for the finalization to be marked as done. - // 10s timeout should be plenty. - await(POLL_MAX_WAIT_MILLIS, POLL_INTERVAL_MILLIS, () -> { - final UpgradeFinalizer.StatusAndMessages progress = - omClient.queryUpgradeFinalizationProgress( - upgradeClientID, false, false); - return isDone(progress.status()); - }); - } - @Test // Making this the first test to be run to avoid db key composition headaches @Order(1) @@ -369,19 +299,49 @@ public void testKeyMetadata() throws Exception { // Clean up assertTrue(fs.delete(file, false)); - // Wait for KeyDeletingService to finish to avoid interfering other tests - Table deletedTable = omMetadataManager.getDeletedTable(); - GenericTestUtils.waitFor( - () -> { - try { - return deletedTable.isEmpty(); - } catch (IOException e) { - return false; - } - }, 250, 10000); + waitForEmptyDeletedTable(); } } + private void waitForEmptyDeletedTable() + throws TimeoutException, InterruptedException { + // Wait for KeyDeletingService to finish to avoid interfering other tests + OMMetadataManager omMetadataManager = + cluster.getOzoneManager().getMetadataManager(); + Table deletedTable = omMetadataManager.getDeletedTable(); + GenericTestUtils.waitFor( + () -> { + try { + return deletedTable.isEmpty(); + } catch (IOException e) { + return false; + } + }, 250, 10000); + } + + @Test + public void testEmptyHsync() throws Exception { + // Check that deletedTable should not have keys with the same block as in + // keyTable's when a key is hsync()'ed then close()'d. + + // Set the fs.defaultFS + final String rootPath = String.format("%s://%s/", + OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); + CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + + final String dir = OZONE_ROOT + bucket.getVolumeName() + + OZONE_URI_DELIMITER + bucket.getName(); + + final Path file = new Path(dir, "file-hsync-empty"); + try (FileSystem fs = FileSystem.get(CONF)) { + try (FSDataOutputStream outputStream = fs.create(file, true)) { + outputStream.write(new byte[0], 0, 0); + outputStream.hsync(); + } + } + waitForEmptyDeletedTable(); + } + @Test public void testKeyHSyncThenClose() throws Exception { // Check that deletedTable should not have keys with the same block as in @@ -566,6 +526,7 @@ private List getOpenKeyInfo(BucketLayout bucketLayout) { @Test public void testUncommittedBlocks() throws Exception { + waitForEmptyDeletedTable(); // Set the fs.defaultFS final String rootPath = String.format("%s://%s/", OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); @@ -746,6 +707,99 @@ static void runTestHSync(FileSystem fs, Path file, }, 500, 3000); } + + public static Stream concurrentExceptionHandling() { + return Stream.of( + Arguments.of(4, 1), + Arguments.of(4, 4), + Arguments.of(8, 4) + ); + } + + @ParameterizedTest + @MethodSource("concurrentExceptionHandling") + public void testConcurrentExceptionHandling(int syncerThreads, int errors) throws Exception { + final String rootPath = String.format("%s://%s/", OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); + CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + + ErrorInjectorImpl errorInjector = new ErrorInjectorImpl(); + XceiverClientManager.enableErrorInjection(errorInjector); + + final String dir = OZONE_ROOT + bucket.getVolumeName() + OZONE_URI_DELIMITER + bucket.getName(); + + try (FileSystem fs = FileSystem.get(CONF)) { + final Path file = new Path(dir, "exceptionhandling"); + byte[] data = new byte[8]; + ThreadLocalRandom.current().nextBytes(data); + int writes; + try (FSDataOutputStream out = fs.create(file, true)) { + writes = runConcurrentWriteHSyncWithException(file, out, data, syncerThreads, errors, errorInjector); + } + validateWrittenFile(file, fs, data, writes); + fs.delete(file, false); + } + } + + private int runConcurrentWriteHSyncWithException(Path file, + final FSDataOutputStream out, byte[] data, int syncThreadsCount, int errors, + ErrorInjectorImpl errorInjector) throws Exception { + + AtomicReference writerException = new AtomicReference<>(); + AtomicReference syncerException = new AtomicReference<>(); + + LOG.info("runConcurrentWriteHSyncWithException {} with size {}", file, data.length); + AtomicInteger writes = new AtomicInteger(); + final long start = Time.monotonicNow(); + + Runnable syncer = () -> { + while ((Time.monotonicNow() - start < 10000)) { + try { + out.write(data); + writes.incrementAndGet(); + out.hsync(); + } catch (Exception e) { + LOG.error("Error calling hsync", e); + syncerException.compareAndSet(null, e); + throw new RuntimeException(e); + } + } + }; + + Thread[] syncThreads = new Thread[syncThreadsCount]; + for (int i = 0; i < syncThreadsCount; i++) { + syncThreads[i] = new Thread(syncer); + syncThreads[i].setName("Syncer-" + i); + syncThreads[i].start(); + } + + // Inject error at 3rd second. + Runnable startErrorInjector = () -> { + while ((Time.monotonicNow() - start <= 3000)) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + errorInjector.start(errors); + LOG.info("Enabled error injection in XceiverClientRatis"); + }; + + new Thread(startErrorInjector).start(); + + for (Thread sync : syncThreads) { + sync.join(); + } + + if (syncerException.get() != null) { + throw syncerException.get(); + } + if (writerException.get() != null) { + throw writerException.get(); + } + return writes.get(); + } + private int runConcurrentWriteHSync(Path file, final FSDataOutputStream out, byte[] data, int syncThreadsCount) throws Exception { @@ -1368,4 +1422,33 @@ private Map getAllDeletedKeys(Table= 0) { + ContainerProtos.ContainerCommandResponseProto proto = ContainerProtos.ContainerCommandResponseProto.newBuilder() + .setResult(ContainerProtos.Result.CLOSED_CONTAINER_IO) + .setMessage("Simulated error #" + errorNum) + .setCmdType(request.getCmdType()) + .build(); + RaftClientReply reply = RaftClientReply.newBuilder() + .setSuccess(true) + .setMessage(Message.valueOf(proto.toByteString())) + .setClientId(clientId) + .setServerId(RaftPeerId.getRaftPeerId(pipeline.getLeaderId().toString())) + .setGroupId(RaftGroupId.randomId()) + .build(); + return reply; + } + + return null; + } + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSyncUpgrade.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSyncUpgrade.java new file mode 100644 index 00000000000..917ce57fe7d --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSyncUpgrade.java @@ -0,0 +1,240 @@ +/* + * 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.fs.ozone; + +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; +import org.apache.hadoop.hdds.scm.storage.BlockOutputStream; +import org.apache.hadoop.hdds.scm.storage.BufferPool; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.ozone.ClientConfigForTesting; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.TestDataUtil; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; +import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl; +import org.apache.hadoop.ozone.container.metadata.AbstractDatanodeStore; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.om.service.OpenKeyCleanupService; +import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature; +import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.MethodOrderer.OrderAnnotation; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; +import org.junit.jupiter.api.Timeout; +import org.slf4j.event.Level; + +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL; +import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME; +import static org.apache.hadoop.ozone.OzoneConsts.OZONE_ROOT; +import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER; +import static org.apache.hadoop.ozone.admin.scm.FinalizeUpgradeCommandUtil.isDone; +import static org.apache.hadoop.ozone.admin.scm.FinalizeUpgradeCommandUtil.isStarting; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DIR_DELETING_SERVICE_INTERVAL; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_LEASE_HARD_LIMIT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_CLEANUP_SERVICE_INTERVAL; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY; +import static org.apache.hadoop.ozone.om.OmUpgradeConfig.ConfigStrings.OZONE_OM_INIT_DEFAULT_LAYOUT_VERSION; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION; +import static org.apache.ozone.test.LambdaTestUtils.await; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test HSync upgrade. + */ +@Timeout(value = 300) +@TestMethodOrder(OrderAnnotation.class) +public class TestHSyncUpgrade { + private MiniOzoneCluster cluster; + private OzoneBucket bucket; + + private final OzoneConfiguration conf = new OzoneConfiguration(); + private OzoneClient client; + private static final BucketLayout BUCKET_LAYOUT = BucketLayout.FILE_SYSTEM_OPTIMIZED; + + private static final int CHUNK_SIZE = 4 << 12; + private static final int FLUSH_SIZE = 3 * CHUNK_SIZE; + private static final int MAX_FLUSH_SIZE = 2 * FLUSH_SIZE; + private static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; + private static final int SERVICE_INTERVAL = 100; + private static final int EXPIRE_THRESHOLD_MS = 140; + + private static final int POLL_INTERVAL_MILLIS = 500; + private static final int POLL_MAX_WAIT_MILLIS = 120_000; + + @BeforeEach + public void init() throws Exception { + final BucketLayout layout = BUCKET_LAYOUT; + + conf.setBoolean(OZONE_OM_RATIS_ENABLE_KEY, false); + conf.set(OZONE_DEFAULT_BUCKET_LAYOUT, layout.name()); + conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); + conf.setInt(OZONE_SCM_RATIS_PIPELINE_LIMIT, 10); + // Reduce KeyDeletingService interval + conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_DIR_DELETING_SERVICE_INTERVAL, 100, TimeUnit.MILLISECONDS); + conf.setBoolean("ozone.client.incremental.chunk.list", true); + conf.setBoolean("ozone.client.stream.putblock.piggybacking", true); + conf.setTimeDuration(OZONE_OM_OPEN_KEY_CLEANUP_SERVICE_INTERVAL, + SERVICE_INTERVAL, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD, + EXPIRE_THRESHOLD_MS, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_OM_LEASE_HARD_LIMIT, + EXPIRE_THRESHOLD_MS, TimeUnit.MILLISECONDS); + conf.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); + conf.setInt(OZONE_OM_INIT_DEFAULT_LAYOUT_VERSION, OMLayoutFeature.MULTITENANCY_SCHEMA.layoutVersion()); + + ClientConfigForTesting.newBuilder(StorageUnit.BYTES) + .setBlockSize(BLOCK_SIZE) + .setChunkSize(CHUNK_SIZE) + .setStreamBufferFlushSize(FLUSH_SIZE) + .setStreamBufferMaxSize(MAX_FLUSH_SIZE) + .setDataStreamBufferFlushSize(MAX_FLUSH_SIZE) + .setDataStreamMinPacketSize(CHUNK_SIZE) + .setDataStreamWindowSize(5 * CHUNK_SIZE) + .applyTo(conf); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(5) + .build(); + cluster.waitForClusterToBeReady(); + client = cluster.newClient(); + + // create a volume and a bucket to be used by OzoneFileSystem + bucket = TestDataUtil.createVolumeAndBucket(client, layout); + + // Enable DEBUG level logging for relevant classes + GenericTestUtils.setLogLevel(BlockManagerImpl.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(AbstractDatanodeStore.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockOutputStream.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(KeyValueHandler.LOG, Level.DEBUG); + + GenericTestUtils.setLogLevel(BufferPool.LOG, Level.DEBUG); + + OpenKeyCleanupService openKeyCleanupService = + (OpenKeyCleanupService) cluster.getOzoneManager().getKeyManager() + .getOpenKeyCleanupService(); + openKeyCleanupService.suspend(); + } + + @AfterEach + public void teardown() { + IOUtils.closeQuietly(client); + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void upgrade() throws Exception { + preFinalizationChecks(); + finalizeOMUpgrade(); + } + + private void preFinalizationChecks() throws IOException { + final String rootPath = String.format("%s://%s/", + OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY)); + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + + final String dir = OZONE_ROOT + bucket.getVolumeName() + + OZONE_URI_DELIMITER + bucket.getName(); + + final Path file = new Path(dir, "pre-finalization"); + try (RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf)) { + try (FSDataOutputStream outputStream = fs.create(file, true)) { + OMException omException = assertThrows(OMException.class, outputStream::hsync); + assertFinalizationExceptionForHsync(omException); + } + final OzoneManagerProtocol omClient = client.getObjectStore() + .getClientProxy().getOzoneManagerClient(); + OMException omException = assertThrows(OMException.class, + () -> omClient.listOpenFiles("", 100, "")); + assertFinalizationException(omException); + + omException = assertThrows(OMException.class, + () -> fs.recoverLease(file)); + assertFinalizationException(omException); + + fs.delete(file, false); + } + } + + private void assertFinalizationExceptionForHsync(OMException omException) { + assertEquals(NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION, + omException.getResult()); + assertThat(omException.getMessage()) + .contains("Cluster does not have the hsync support feature finalized yet"); + } + + private void assertFinalizationException(OMException omException) { + assertEquals(NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION, + omException.getResult()); + assertThat(omException.getMessage()) + .contains("cannot be invoked before finalization."); + } + + /** + * Trigger OM upgrade finalization from the client and block until completion + * (status FINALIZATION_DONE). + */ + private void finalizeOMUpgrade() throws Exception { + // Trigger OM upgrade finalization. Ref: FinalizeUpgradeSubCommand#call + final OzoneManagerProtocol omClient = client.getObjectStore() + .getClientProxy().getOzoneManagerClient(); + final String upgradeClientID = "Test-Upgrade-Client-" + UUID.randomUUID(); + UpgradeFinalizer.StatusAndMessages finalizationResponse = + omClient.finalizeUpgrade(upgradeClientID); + + // The status should transition as soon as the client call above returns + assertTrue(isStarting(finalizationResponse.status())); + // Wait for the finalization to be marked as done. + // 10s timeout should be plenty. + await(POLL_MAX_WAIT_MILLIS, POLL_INTERVAL_MILLIS, () -> { + final UpgradeFinalizer.StatusAndMessages progress = + omClient.queryUpgradeFinalizationProgress( + upgradeClientID, false, false); + return isDone(progress.status()); + }); + } + +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java index 6ec233fc35b..a4a9bcff470 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java @@ -45,6 +45,7 @@ import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.utils.FaultInjectorImpl; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.tag.Flaky; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -82,6 +83,7 @@ * Test cases for recoverLease() API. */ @Timeout(300) +@Flaky("HDDS-11323") public class TestLeaseRecovery { private MiniOzoneCluster cluster; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerMXBean.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerMXBean.java index b6e85ab942d..75d860d951b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerMXBean.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerMXBean.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.tag.Flaky; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -61,6 +62,7 @@ public void init() * * @throws Exception */ + @Flaky("HDDS-11359") @Test public void testPipelineInfo() throws Exception { ObjectName bean = new ObjectName( diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/utils/ClusterContainersUtil.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/utils/ClusterContainersUtil.java index e7e0337b5f9..bf20b4ecc0b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/utils/ClusterContainersUtil.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/utils/ClusterContainersUtil.java @@ -19,12 +19,12 @@ import com.google.common.base.Preconditions; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneKey; import org.apache.hadoop.ozone.client.OzoneKeyDetails; import org.apache.hadoop.ozone.container.common.helpers.BlockData; -import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.DBHandle; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; @@ -65,18 +65,10 @@ public static File getChunksLocationPath(MiniOzoneCluster cluster, Container con // the container. KeyValueContainerData containerData = (KeyValueContainerData) container.getContainerData(); - try (DBHandle db = BlockUtils.getDB(containerData, cluster.getConf()); - BlockIterator keyValueBlockIterator = - db.getStore().getBlockIterator(containerID)) { - // Find the block corresponding to the key we put. We use the localID of - // the BlockData to identify out key. - BlockData blockData = null; - while (keyValueBlockIterator.hasNext()) { - blockData = keyValueBlockIterator.nextBlock(); - if (blockData.getBlockID().getLocalID() == localID) { - break; - } - } + try (DBHandle db = BlockUtils.getDB(containerData, cluster.getConf())) { + BlockID blockID = new BlockID(containerID, localID); + String blockKey = containerData.getBlockKey(localID); + BlockData blockData = db.getStore().getBlockByID(blockID, blockKey); assertNotNull(blockData, "Block not found"); // Get the location of the chunk file diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java index 2b29701cf79..4f41d516153 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java @@ -1449,11 +1449,12 @@ private static X509Certificate signX509Cert( addIpAndDnsDataToBuilder(csrBuilder); LocalDateTime start = LocalDateTime.now(); Duration certDuration = conf.getDefaultCertDuration(); + //TODO: generateCSR! return approver.sign(conf, rootKeyPair.getPrivate(), rootCert, Date.from(start.atZone(ZoneId.systemDefault()).toInstant()), Date.from(start.plus(certDuration) .atZone(ZoneId.systemDefault()).toInstant()), - csrBuilder.build(), "test", clusterId, + csrBuilder.build().generateCSR(), "test", clusterId, String.valueOf(System.nanoTime())); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java index b40b0bbcc62..766ed09bccd 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java @@ -16,6 +16,7 @@ */ package org.apache.hadoop.ozone.client.rpc; +import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; import org.apache.hadoop.hdds.client.ECReplicationConfig; @@ -67,6 +68,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.fail; /** @@ -123,6 +125,8 @@ protected static void init(boolean zeroCopyEnabled) throws Exception { conf.setBoolean(OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED, zeroCopyEnabled); conf.setInt(ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT, 10); + // "Enable" hsync to verify that hsync would be blocked by ECKeyOutputStream + conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) .setBlockSize(blockSize) @@ -469,4 +473,18 @@ private byte[] getInputBytes(int offset, int bufferChunks, int numChunks) { return inputData; } + @Test + public void testBlockedHflushAndHsync() throws Exception { + // Expect ECKeyOutputStream hflush and hsync calls to throw exception + try (OzoneOutputStream oOut = TestHelper.createKey( + keyString, new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS, chunkSize), + inputSize, objectStore, volumeName, bucketName)) { + assertInstanceOf(ECKeyOutputStream.class, oOut.getOutputStream()); + KeyOutputStream kOut = (KeyOutputStream) oOut.getOutputStream(); + + assertThrows(NotImplementedException.class, () -> kOut.hflush()); + assertThrows(NotImplementedException.class, () -> kOut.hsync()); + } + } + } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java index 30597fc3e77..eb9f35f518c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java @@ -4908,6 +4908,12 @@ public void testParallelDeleteBucketAndCreateKey() throws IOException, assertThat(omSMLog.getOutput()).contains("Failed to write, Exception occurred"); } + @Test + public void testGetServerDefaults() throws IOException { + assertNotNull(getClient().getProxy().getServerDefaults()); + assertNull(getClient().getProxy().getServerDefaults().getKeyProviderUri()); + } + private static class OMRequestHandlerPauseInjector extends FaultInjector { private CountDownLatch ready; private CountDownLatch wait; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java index 2a6b2246b9c..8810bab5190 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java @@ -48,6 +48,7 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.TestHelper; +import org.apache.ozone.test.tag.Flaky; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestInstance; @@ -661,6 +662,7 @@ void testWriteExactlyMaxFlushSize(boolean flushDelay, boolean enablePiggybacking @ParameterizedTest @MethodSource("clientParameters") + @Flaky("HDDS-11325") void testWriteMoreThanMaxFlushSize(boolean flushDelay, boolean enablePiggybacking) throws Exception { OzoneClientConfig config = newClientConfig(cluster.getConf(), flushDelay, enablePiggybacking); try (OzoneClient client = newClient(cluster.getConf(), config)) { @@ -698,7 +700,8 @@ void testWriteMoreThanMaxFlushSize(boolean flushDelay, boolean enablePiggybackin assertInstanceOf(RatisBlockOutputStream.class, keyOutputStream.getStreamEntries().get(0).getOutputStream()); - assertEquals(4, blockOutputStream.getBufferPool().getSize()); + assertThat(blockOutputStream.getBufferPool().getSize()) + .isLessThanOrEqualTo(4); // writtenDataLength as well flushedDataLength will be updated here assertEquals(dataLength, blockOutputStream.getWrittenDataLength()); @@ -727,7 +730,8 @@ void testWriteMoreThanMaxFlushSize(boolean flushDelay, boolean enablePiggybackin // Since the data in the buffer is already flushed, flush here will have // no impact on the counters and data structures - assertEquals(4, blockOutputStream.getBufferPool().getSize()); + assertThat(blockOutputStream.getBufferPool().getSize()) + .isLessThanOrEqualTo(4); assertEquals(dataLength, blockOutputStream.getWrittenDataLength()); // dataLength > MAX_FLUSH_SIZE assertEquals(flushDelay ? MAX_FLUSH_SIZE : dataLength, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStreamWithFailures.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStreamWithFailures.java index 8d69da3ef3e..f823add57bd 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStreamWithFailures.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStreamWithFailures.java @@ -93,6 +93,7 @@ private static Stream clientParameters() { @ParameterizedTest @MethodSource("clientParameters") + @Flaky("HDDS-11325") void testContainerClose(boolean flushDelay, boolean enablePiggybacking) throws Exception { OzoneClientConfig config = newClientConfig(cluster.getConf(), flushDelay, enablePiggybacking); try (OzoneClient client = newClient(cluster.getConf(), config)) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCommitInRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCommitInRatis.java index f7fbbf37c52..4ff671df616 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCommitInRatis.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCommitInRatis.java @@ -163,7 +163,7 @@ public void test2WayCommitForRetryfailure(RaftProtos.ReplicationLevel watchType) reply.getResponse().get(); assertEquals(3, ratisClient.getCommitInfoMap().size()); // wait for the container to be created on all the nodes - xceiverClient.watchForCommit(reply.getLogIndex()); + xceiverClient.watchForCommit(reply.getLogIndex()).get(); for (HddsDatanodeService dn : cluster.getHddsDatanodes()) { // shutdown the ratis follower if (RatisTestHelper.isRatisFollower(dn, pipeline)) { @@ -175,7 +175,7 @@ public void test2WayCommitForRetryfailure(RaftProtos.ReplicationLevel watchType) .getCloseContainer(pipeline, container1.getContainerInfo().getContainerID())); reply.getResponse().get(); - xceiverClient.watchForCommit(reply.getLogIndex()); + xceiverClient.watchForCommit(reply.getLogIndex()).get(); if (watchType == RaftProtos.ReplicationLevel.ALL_COMMITTED) { // commitInfo Map will be reduced to 2 here diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestMultiBlockWritesWithDnFailures.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestMultiBlockWritesWithDnFailures.java index 34f85d8e992..4c558c3d77a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestMultiBlockWritesWithDnFailures.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestMultiBlockWritesWithDnFailures.java @@ -42,6 +42,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.ozone.test.tag.Flaky; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -178,6 +179,7 @@ public void testMultiBlockWritesWithDnFailures() throws Exception { validateData(keyName, data.concat(data).getBytes(UTF_8)); } + @Flaky("HDDS-11355") @Test public void testMultiBlockWritesWithIntermittentDnFailures() throws Exception { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java index 773f6be966b..958a37380cf 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java @@ -99,6 +99,8 @@ */ class TestSecureOzoneRpcClient extends OzoneRpcClientTests { + private static String keyProviderUri = "kms://http@kms:9600/kms"; + @BeforeAll public static void init() throws Exception { File testDir = GenericTestUtils.getTestDir( @@ -120,6 +122,8 @@ public static void init() throws Exception { conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, OMConfigKeys.OZONE_BUCKET_LAYOUT_OBJECT_STORE); conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, + keyProviderUri); MiniOzoneCluster.Builder builder = MiniOzoneCluster.newBuilder(conf) .setCertificateClient(certificateClientTest) .setSecretKeyClient(new SecretKeyTestClient()); @@ -433,6 +437,13 @@ public void testS3Auth() throws Exception { public void testZReadKeyWithUnhealthyContainerReplica() { } + @Test + public void testGetServerDefaults() throws IOException { + assertNotNull(getClient().getProxy().getServerDefaults()); + assertEquals(keyProviderUri, + getClient().getProxy().getServerDefaults().getKeyProviderUri()); + } + @AfterAll public static void shutdown() throws IOException { shutdownCluster(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java index f42969e67f0..bec14b23b0f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java @@ -291,7 +291,7 @@ public void testWatchForCommitForRetryfailure(RaftProtos.ReplicationLevel watchT // as well as there is no logIndex generate in Ratis. // The basic idea here is just to test if its throws an exception. ExecutionException e = assertThrows(ExecutionException.class, - () -> xceiverClient.watchForCommit(index + RandomUtils.nextInt(0, 100) + 10)); + () -> xceiverClient.watchForCommit(index + RandomUtils.nextInt(0, 100) + 10).get()); // since the timeout value is quite long, the watch request will either // fail with NotReplicated exceptio, RetryFailureException or // RuntimeException @@ -348,7 +348,7 @@ public void test2WayCommitForTimeoutException(RaftProtos.ReplicationLevel watchT .getCloseContainer(pipeline, container1.getContainerInfo().getContainerID())); reply.getResponse().get(); - xceiverClient.watchForCommit(reply.getLogIndex()); + xceiverClient.watchForCommit(reply.getLogIndex()).get(); // commitInfo Map will be reduced to 2 here if (watchType == RaftProtos.ReplicationLevel.ALL_COMMITTED) { @@ -392,9 +392,8 @@ public void testWatchForCommitForGroupMismatchException() throws Exception { // just watch for a log index which in not updated in the commitInfo Map // as well as there is no logIndex generate in Ratis. // The basic idea here is just to test if its throws an exception. - Exception e = - assertThrows(Exception.class, - () -> xceiverClient.watchForCommit(reply.getLogIndex() + RandomUtils.nextInt(0, 100) + 10)); + final Exception e = assertThrows(Exception.class, + () -> xceiverClient.watchForCommit(reply.getLogIndex() + RandomUtils.nextInt(0, 100) + 10).get()); assertInstanceOf(GroupMismatchException.class, HddsClientUtils.checkForException(e)); } finally { clientManager.releaseClient(xceiverClient, false); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java index 00654d943f7..192c933f53c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java @@ -557,23 +557,32 @@ private void clearBlocksTable(Container container) throws IOException { = BlockUtils.getDB( (KeyValueContainerData) container.getContainerData(), conf)) { - List> - blocks = dbHandle.getStore().getBlockDataTable().getRangeKVs( - ((KeyValueContainerData) container.getContainerData()). - startKeyEmpty(), - Integer.MAX_VALUE, - ((KeyValueContainerData) container.getContainerData()). - containerPrefix(), - ((KeyValueContainerData) container.getContainerData()). - getUnprefixedKeyFilter()); - try (BatchOperation batch = dbHandle.getStore().getBatchHandler() - .initBatchOperation()) { - for (Table.KeyValue kv : blocks) { - String blk = kv.getKey(); - dbHandle.getStore().getBlockDataTable().deleteWithBatch(batch, blk); - } - dbHandle.getStore().getBatchHandler().commitBatchOperation(batch); + Table table = dbHandle.getStore().getBlockDataTable(); + clearTable(dbHandle, table, container); + + table = dbHandle.getStore().getLastChunkInfoTable(); + clearTable(dbHandle, table, container); + } + } + + private void clearTable(DBHandle dbHandle, Table table, Container container) + throws IOException { + List> + blocks = table.getRangeKVs( + ((KeyValueContainerData) container.getContainerData()). + startKeyEmpty(), + Integer.MAX_VALUE, + ((KeyValueContainerData) container.getContainerData()). + containerPrefix(), + ((KeyValueContainerData) container.getContainerData()). + getUnprefixedKeyFilter()); + try (BatchOperation batch = dbHandle.getStore().getBatchHandler() + .initBatchOperation()) { + for (Table.KeyValue kv : blocks) { + String blk = kv.getKey(); + table.deleteWithBatch(batch, blk); } + dbHandle.getStore().getBatchHandler().commitBatchOperation(batch); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java index 0fd31bb4b72..e68831b494f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java @@ -189,7 +189,7 @@ static XceiverServerRatis newXceiverServerRatis( conf.set(OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir); final ContainerDispatcher dispatcher = new TestContainerDispatcher(); - return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher, + return XceiverServerRatis.newXceiverServerRatis(null, dn, conf, dispatcher, new ContainerController(new ContainerSet(1000), Maps.newHashMap()), null, null); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java index 068cb01a967..f55912b26b0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.nio.file.Path; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.commons.io.FileUtils; @@ -48,7 +47,6 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; -import org.apache.hadoop.ozone.container.common.interfaces.Handler; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; @@ -148,18 +146,7 @@ private HddsDispatcher createDispatcher(DatanodeDetails dd, VolumeSet volumeSet) ContainerSet containerSet = new ContainerSet(1000); StateContext context = ContainerTestUtils.getMockContext( dd, CONF); - ContainerMetrics metrics = ContainerMetrics.create(CONF); - Map handlers = Maps.newHashMap(); - for (ContainerProtos.ContainerType containerType : - ContainerProtos.ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType(containerType, CONF, - context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, - c -> { })); - } - HddsDispatcher dispatcher = new HddsDispatcher(CONF, containerSet, - volumeSet, handlers, context, metrics, null); + HddsDispatcher dispatcher = ContainerTestUtils.getHddsDispatcher(CONF, containerSet, volumeSet, context); StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()) .forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile())); dispatcher.setClusterId(UUID.randomUUID().toString()); @@ -253,7 +240,7 @@ private XceiverServerSpi newXceiverServerRatis(DatanodeDetails dn, MutableVolume CONF.set(OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir); final ContainerDispatcher dispatcher = createDispatcher(dn, volumeSet); - return XceiverServerRatis.newXceiverServerRatis(dn, CONF, dispatcher, + return XceiverServerRatis.newXceiverServerRatis(null, dn, CONF, dispatcher, new ContainerController(new ContainerSet(1000), Maps.newHashMap()), null, null); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java index e06e8b5a00f..3ac0cd4346e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java @@ -386,7 +386,7 @@ private OzoneContainer createAndStartOzoneContainerInstance() { try { StateContext stateContext = ContainerTestUtils.getMockContext(dn, conf); container = new OzoneContainer( - dn, conf, stateContext, caClient, keyClient); + null, dn, conf, stateContext, caClient, keyClient); MutableVolumeSet volumeSet = container.getVolumeSet(); StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()) .forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile())); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestSecureOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestSecureOzoneContainer.java index 5585696dfc3..92d716f7a40 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestSecureOzoneContainer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestSecureOzoneContainer.java @@ -137,7 +137,7 @@ void testCreateOzoneContainer(boolean requireToken, boolean hasToken, conf.setBoolean(OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT, false); DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails(); - container = new OzoneContainer(dn, conf, ContainerTestUtils + container = new OzoneContainer(null, dn, conf, ContainerTestUtils .getMockContext(dn, conf), caClient, secretKeyClient); MutableVolumeSet volumeSet = container.getVolumeSet(); StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java index 630c4d31495..27e85501662 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java @@ -48,7 +48,6 @@ import org.apache.hadoop.ozone.RatisTestHelper; import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; -import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; @@ -132,7 +131,7 @@ static XceiverServerRatis newXceiverServerRatis( conf.set(OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir); final ContainerDispatcher dispatcher = new TestContainerDispatcher(); - return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher, + return XceiverServerRatis.newXceiverServerRatis(null, dn, conf, dispatcher, new ContainerController(new ContainerSet(1000), Maps.newHashMap()), caClient, null); } @@ -200,19 +199,7 @@ private HddsDispatcher createDispatcher(DatanodeDetails dd, UUID scmId, StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()) .forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile())); StateContext context = ContainerTestUtils.getMockContext(dd, conf); - ContainerMetrics metrics = ContainerMetrics.create(conf); - Map handlers = Maps.newHashMap(); - for (ContainerProtos.ContainerType containerType : - ContainerProtos.ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType(containerType, conf, - dd.getUuid().toString(), - containerSet, volumeSet, metrics, - c -> { - })); - } - HddsDispatcher hddsDispatcher = new HddsDispatcher( - conf, containerSet, volumeSet, handlers, context, metrics, null); + HddsDispatcher hddsDispatcher = ContainerTestUtils.getHddsDispatcher(conf, containerSet, volumeSet, context); hddsDispatcher.setClusterId(scmId.toString()); return hddsDispatcher; } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java index 8044685bb74..cae7f6bb59e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.EnumSet; import java.util.List; -import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -57,11 +56,9 @@ import org.apache.hadoop.ozone.RatisTestHelper; import org.apache.hadoop.ozone.client.SecretKeyTestClient; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; -import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; -import org.apache.hadoop.ozone.container.common.interfaces.Handler; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; @@ -183,18 +180,7 @@ private HddsDispatcher createDispatcher(DatanodeDetails dd, UUID scmId, StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()) .forEach(hddsVolume -> hddsVolume.setDbParentDir(tempDir.toFile())); StateContext context = ContainerTestUtils.getMockContext(dd, conf); - ContainerMetrics metrics = ContainerMetrics.create(conf); - Map handlers = Maps.newHashMap(); - for (ContainerProtos.ContainerType containerType : - ContainerProtos.ContainerType.values()) { - handlers.put(containerType, - Handler.getHandlerForContainerType(containerType, conf, - dd.getUuid().toString(), - containerSet, volumeSet, metrics, - c -> { })); - } - HddsDispatcher hddsDispatcher = new HddsDispatcher( - conf, containerSet, volumeSet, handlers, context, metrics, + HddsDispatcher hddsDispatcher = ContainerTestUtils.getHddsDispatcher(conf, containerSet, volumeSet, context, TokenVerifier.create(new SecurityConfig(conf), secretKeyClient)); hddsDispatcher.setClusterId(scmId.toString()); return hddsDispatcher; @@ -218,7 +204,7 @@ XceiverServerRatis newXceiverServerRatis( conf.set(OzoneConfigKeys.HDDS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir); final ContainerDispatcher dispatcher = createDispatcher(dn, UUID.randomUUID(), conf); - return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher, + return XceiverServerRatis.newXceiverServerRatis(null, dn, conf, dispatcher, new ContainerController(new ContainerSet(1000), Maps.newHashMap()), caClient, null); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java index f209783c745..33d59f101eb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java @@ -22,7 +22,8 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig; -import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientCreator; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; @@ -79,11 +80,11 @@ private static void startCluster(OzoneConfiguration conf) throws Exception { storageContainerLocationClient.allocateContainer( SCMTestUtils.getReplicationType(conf), HddsProtos.ReplicationFactor.ONE, OzoneConsts.OZONE); - XceiverClientManager xceiverClientManager = new XceiverClientManager(conf); - XceiverClientSpi client = xceiverClientManager - .acquireClient(container.getPipeline()); - ContainerProtocolCalls.createContainer(client, - container.getContainerInfo().getContainerID(), null); + try (XceiverClientFactory factory = new XceiverClientCreator(conf); + XceiverClientSpi client = factory.acquireClient(container.getPipeline())) { + ContainerProtocolCalls.createContainer(client, + container.getContainerInfo().getContainerID(), null); + } } static void shutdownCluster() { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestHadoopDirTreeGenerator.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestHadoopDirTreeGenerator.java index 3140681d3dc..9137eca6c0e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestHadoopDirTreeGenerator.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestHadoopDirTreeGenerator.java @@ -152,10 +152,10 @@ private void verifyDirTree(String volumeName, String bucketName, int depth, Path rootDir = new Path(rootPath.concat("/")); // verify root path details FileStatus[] fileStatuses = fileSystem.listStatus(rootDir); + // verify the num of peer directories, expected span count is 1 + // as it has only one dir at root. + verifyActualSpan(1, fileStatuses); for (FileStatus fileStatus : fileStatuses) { - // verify the num of peer directories, expected span count is 1 - // as it has only one dir at root. - verifyActualSpan(1, fileStatuses); int actualDepth = traverseToLeaf(fileSystem, fileStatus.getPath(), 1, depth, span, fileCount, StorageSize.parse(perFileSize, StorageUnit.BYTES)); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestAddRemoveOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestAddRemoveOzoneManager.java index dfc3045be8f..c7cc83c61c4 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestAddRemoveOzoneManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestAddRemoveOzoneManager.java @@ -47,6 +47,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.tag.Flaky; import org.apache.ratis.grpc.server.GrpcLogAppender; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; @@ -285,6 +286,7 @@ public void testBootstrapWithoutConfigUpdate() throws Exception { * 1. Stop 1 OM and update configs on rest, bootstrap new node -> fail * 2. Force bootstrap (with 1 node down and updated configs on rest) -> pass */ + @Flaky("HDDS-11358") @Test public void testForceBootstrap() throws Exception { GenericTestUtils.setLogLevel(GrpcLogAppender.LOG, Level.ERROR); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java index e773bf7ed7f..f25bb47f0db 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java @@ -18,10 +18,12 @@ package org.apache.hadoop.ozone.om; +import com.google.common.collect.ImmutableMap; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.client.ContainerBlockID; +import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -89,8 +91,11 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -145,10 +150,17 @@ public class TestOmContainerLocationCache { private static ObjectStore objectStore; private static XceiverClientGrpc mockDn1Protocol; private static XceiverClientGrpc mockDn2Protocol; + private static XceiverClientGrpc mockDnEcProtocol; private static final DatanodeDetails DN1 = MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID()); private static final DatanodeDetails DN2 = MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID()); + private static final DatanodeDetails DN3 = + MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID()); + private static final DatanodeDetails DN4 = + MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID()); + private static final DatanodeDetails DN5 = + MockDatanodeDetails.createDatanodeDetails(UUID.randomUUID()); private static final AtomicLong CONTAINER_ID = new AtomicLong(1); @@ -200,6 +212,8 @@ private static XceiverClientManager mockDataNodeClientFactory() throws IOException { mockDn1Protocol = spy(new XceiverClientGrpc(createPipeline(DN1), conf)); mockDn2Protocol = spy(new XceiverClientGrpc(createPipeline(DN2), conf)); + mockDnEcProtocol = spy(new XceiverClientGrpc(createEcPipeline( + ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4, DN5, 5)), conf)); XceiverClientManager manager = mock(XceiverClientManager.class); when(manager.acquireClient(argThat(matchEmptyPipeline()))) .thenCallRealMethod(); @@ -217,6 +231,11 @@ private static XceiverClientManager mockDataNodeClientFactory() .thenReturn(mockDn2Protocol); when(manager.acquireClientForReadData(argThat(matchPipeline(DN2)))) .thenReturn(mockDn2Protocol); + + when(manager.acquireClient(argThat(matchEcPipeline()))) + .thenReturn(mockDnEcProtocol); + when(manager.acquireClientForReadData(argThat(matchEcPipeline()))) + .thenReturn(mockDnEcProtocol); return manager; } @@ -231,6 +250,11 @@ private static ArgumentMatcher matchPipeline(DatanodeDetails dn) { && argument.getNodes().get(0).getUuid().equals(dn.getUuid()); } + private static ArgumentMatcher matchEcPipeline() { + return argument -> argument != null && !argument.getNodes().isEmpty() + && argument.getReplicationConfig() instanceof ECReplicationConfig; + } + private static void createBucket(String volumeName, String bucketName, boolean isVersionEnabled) throws IOException { @@ -256,12 +280,14 @@ private static void createVolume(String volumeName) throws IOException { public void beforeEach() throws IOException { CONTAINER_ID.getAndIncrement(); reset(mockScmBlockLocationProtocol, mockScmContainerClient, - mockDn1Protocol, mockDn2Protocol); + mockDn1Protocol, mockDn2Protocol, mockDnEcProtocol); InnerNode.Factory factory = InnerNodeImpl.FACTORY; when(mockScmBlockLocationProtocol.getNetworkTopology()).thenReturn( factory.newInnerNode("", "", null, NetConstants.ROOT_LEVEL, 1)); when(mockDn1Protocol.getPipeline()).thenReturn(createPipeline(DN1)); when(mockDn2Protocol.getPipeline()).thenReturn(createPipeline(DN2)); + when(mockDnEcProtocol.getPipeline()).thenReturn(createEcPipeline( + ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4, DN5, 5))); } /** @@ -575,6 +601,48 @@ public void containerRefreshedOnEmptyPipelines() throws Exception { .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); } + @Test + public void containerRefreshedOnInsufficientEcPipelines() throws Exception { + int chunkSize = 1024 * 1024; + int dataBlocks = 3; + int parityBlocks = 2; + int inputSize = chunkSize * dataBlocks; + byte[][] inputChunks = new byte[dataBlocks][chunkSize]; + + mockScmAllocationEcPipeline(CONTAINER_ID.get(), 1L); + mockWriteChunkResponse(mockDnEcProtocol); + mockPutBlockResponse(mockDnEcProtocol, CONTAINER_ID.get(), 1L, null); + + OzoneBucket bucket = objectStore.getVolume(VOLUME_NAME).getBucket(BUCKET_NAME); + + String keyName = "ecKey"; + try (OzoneOutputStream os = bucket.createKey(keyName, inputSize, + new ECReplicationConfig(dataBlocks, parityBlocks, ECReplicationConfig.EcCodec.RS, + chunkSize), new HashMap<>())) { + for (int i = 0; i < dataBlocks; i++) { + os.write(inputChunks[i]); + } + } + + // case1: pipeline replicaIndexes missing some data indexes, should not cache + mockScmGetContainerEcPipeline(CONTAINER_ID.get(), ImmutableMap.of(DN1, 1, DN2, 2, DN4, 4)); + bucket.getKey(keyName); + verify(mockScmContainerClient, times(1)) + .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); + bucket.getKey(keyName); + verify(mockScmContainerClient, times(2)) + .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); + + // case2: pipeline replicaIndexes contain all data indexes, should cache + mockScmGetContainerEcPipeline(CONTAINER_ID.get(), ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4)); + bucket.getKey(keyName); + verify(mockScmContainerClient, times(3)) + .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); + bucket.getKey(keyName); + verify(mockScmContainerClient, times(3)) + .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); + } + private void mockPutBlockResponse(XceiverClientSpi mockDnProtocol, long containerId, long localId, byte[] data) @@ -661,6 +729,22 @@ private void mockScmAllocationOnDn1(long containerID, .thenReturn(Collections.singletonList(block)); } + private void mockScmAllocationEcPipeline(long containerID, long localId) + throws IOException { + ContainerBlockID blockId = new ContainerBlockID(containerID, localId); + AllocatedBlock block = new AllocatedBlock.Builder() + .setPipeline(createEcPipeline(ImmutableMap.of(DN1, 1, DN2, 2, DN3, 3, DN4, 4, DN5, 5))) + .setContainerBlockID(blockId) + .build(); + when(mockScmBlockLocationProtocol + .allocateBlock(anyLong(), anyInt(), + any(ECReplicationConfig.class), + anyString(), + any(ExcludeList.class), + anyString())) + .thenReturn(Collections.singletonList(block)); + } + private void mockScmGetContainerPipeline(long containerId, DatanodeDetails dn) throws IOException { @@ -690,6 +774,20 @@ private void mockScmGetContainerPipelineEmpty(long containerId) newHashSet(containerId))).thenReturn(containerWithPipelines); } + private void mockScmGetContainerEcPipeline(long containerId, Map indexes) + throws IOException { + Pipeline pipeline = createEcPipeline(indexes); + ContainerInfo containerInfo = new ContainerInfo.Builder() + .setContainerID(containerId) + .setPipelineID(pipeline.getId()).build(); + List containerWithPipelines = + Collections.singletonList( + new ContainerWithPipeline(containerInfo, pipeline)); + + when(mockScmContainerClient.getContainerWithPipelineBatch( + newHashSet(containerId))).thenReturn(containerWithPipelines); + } + private void mockGetBlock(XceiverClientGrpc mockDnProtocol, long containerId, long localId, byte[] data, @@ -788,4 +886,14 @@ private static Pipeline createPipeline(List nodes) { .setNodes(nodes) .build(); } + + private static Pipeline createEcPipeline(Map indexes) { + return Pipeline.newBuilder() + .setState(Pipeline.PipelineState.OPEN) + .setId(PipelineID.randomId()) + .setReplicationConfig(new ECReplicationConfig(3, 2)) + .setReplicaIndexes(indexes) + .setNodes(new ArrayList<>(indexes.keySet())) + .build(); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithStoppedNodes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithStoppedNodes.java index 2d34f5fc403..6937c52c712 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithStoppedNodes.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithStoppedNodes.java @@ -42,6 +42,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.log4j.Logger; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.tag.Flaky; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.util.TimeDuration; @@ -76,6 +77,7 @@ * Ozone Manager HA tests that stop/restart one or more OM nodes. * @see TestOzoneManagerHAWithAllRunning */ +@Flaky("HDDS-11352") @TestMethodOrder(MethodOrderer.OrderAnnotation.class) public class TestOzoneManagerHAWithStoppedNodes extends TestOzoneManagerHA { @@ -484,6 +486,8 @@ void testIncrementalWaitTimeWithSameNodeFailover() throws Exception { assertEquals((numTimesTriedToSameNode + 1) * waitBetweenRetries, omFailoverProxyProvider.getWaitTime()); } + + @Flaky("HDDS-11353") @Test void testOMHAMetrics() throws Exception { // Get leader OM diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOzoneManagerSnapshotAcl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOzoneManagerSnapshotAcl.java index 5694edd773e..96099498438 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOzoneManagerSnapshotAcl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOzoneManagerSnapshotAcl.java @@ -48,6 +48,7 @@ import org.apache.hadoop.ozone.security.acl.OzoneObj; import org.apache.hadoop.ozone.security.acl.OzoneObjInfo; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.ozone.test.tag.Flaky; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Timeout; @@ -402,6 +403,7 @@ public void testLookupKeyWithAllowedUserForPrefixAcl(BucketLayout bucketLayout) assertDoesNotThrow(() -> ozoneManager.lookupKey(snapshotKeyArgs)); } + @Flaky("HDDS-11354") @ParameterizedTest @EnumSource(BucketLayout.class) public void testLookupKeyWithNotAllowedUserForPrefixAcl(BucketLayout bucketLayout) throws Exception { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java index 5a46d571c6b..89f068cdedf 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java @@ -1568,6 +1568,39 @@ public void testShQuota() throws Exception { .contains("Missing required parameter"); out.reset(); + // Test incompatible volume-bucket quota + args = new String[]{"volume", "create", "vol6"}; + execute(ozoneShell, args); + out.reset(); + + args = new String[]{"bucket", "create", "vol6/buck6"}; + execute(ozoneShell, args); + out.reset(); + + args = new String[]{"volume", "setquota", "vol6", "--space-quota", "1000B"}; + executeWithError(ozoneShell, args, "Can not set volume space quota " + + "on volume as some of buckets in this volume have no quota set"); + out.reset(); + + args = new String[]{"bucket", "setquota", "vol6/buck6", "--space-quota", "1000B"}; + execute(ozoneShell, args); + out.reset(); + + args = new String[]{"volume", "setquota", "vol6", "--space-quota", "2000B"}; + execute(ozoneShell, args); + out.reset(); + + args = new String[]{"bucket", "create", "vol6/buck62"}; + executeWithError(ozoneShell, args, "Bucket space quota in this " + + "volume should be set as volume space quota is already set."); + out.reset(); + + args = new String[]{"bucket", "create", "vol6/buck62", "--space-quota", "2000B"}; + executeWithError(ozoneShell, args, "Total buckets quota in this volume " + + "should not be greater than volume quota : the total space quota is set to:3000. " + + "But the volume space quota is:2000"); + out.reset(); + // Test set bucket spaceQuota or nameSpaceQuota to normal value. String[] bucketArgs8 = new String[]{"bucket", "setquota", "vol4/buck4", "--space-quota", "1000B"}; diff --git a/hadoop-ozone/interface-client/pom.xml b/hadoop-ozone/interface-client/pom.xml index c196dca045d..18d9584fbc8 100644 --- a/hadoop-ozone/interface-client/pom.xml +++ b/hadoop-ozone/interface-client/pom.xml @@ -33,6 +33,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + com.google.guava + guava + com.google.protobuf protobuf-java @@ -41,6 +45,22 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone hdds-interface-client + + + org.apache.hadoop.thirdparty + hadoop-shaded-protobuf_3_7 + + + + io.grpc + grpc-api + + + com.google.code.findbugs + jsr305 + + + io.grpc grpc-protobuf diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index 32bba266080..9e0f729be40 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -147,8 +147,9 @@ enum Type { ListStatusLight = 129; GetSnapshotInfo = 130; RenameSnapshot = 131; - ListOpenFiles = 132; + QuotaRepair = 133; + GetServerDefaults = 134; } enum SafeMode { @@ -285,8 +286,9 @@ message OMRequest { optional SetSnapshotPropertyRequest SetSnapshotPropertyRequest = 127; optional SnapshotInfoRequest SnapshotInfoRequest = 128; optional RenameSnapshotRequest RenameSnapshotRequest = 129; - optional ListOpenFilesRequest ListOpenFilesRequest = 130; + optional QuotaRepairRequest QuotaRepairRequest = 131; + optional ServerDefaultsRequest ServerDefaultsRequest = 132; } message OMResponse { @@ -410,8 +412,9 @@ message OMResponse { optional SnapshotInfoResponse SnapshotInfoResponse = 130; optional OMLockDetailsProto omLockDetails = 131; optional RenameSnapshotResponse RenameSnapshotResponse = 132; - optional ListOpenFilesResponse ListOpenFilesResponse = 133; + optional QuotaRepairResponse QuotaRepairResponse = 134; + optional ServerDefaultsResponse ServerDefaultsResponse = 135; } enum Status { @@ -2187,6 +2190,32 @@ message SetSafeModeResponse { optional bool response = 1; } +message QuotaRepairRequest { + repeated BucketQuotaCount bucketCount = 1; + required bool supportVolumeOldQuota = 2 [default=false]; +} +message BucketQuotaCount { + required string volName = 1; + required string bucketName = 2; + required int64 diffUsedBytes = 3; + required int64 diffUsedNamespace = 4; + required bool supportOldQuota = 5 [default=false]; +} + +message QuotaRepairResponse { +} + +message ServerDefaultsRequest { +} + +message FsServerDefaultsProto { + optional string keyProviderUri = 1; +} + +message ServerDefaultsResponse { + required FsServerDefaultsProto serverDefaults = 1; +} + message OMLockDetailsProto { optional bool isLockAcquired = 1; optional uint64 waitLockNanos = 2; diff --git a/hadoop-ozone/interface-storage/pom.xml b/hadoop-ozone/interface-storage/pom.xml index ca7f14f1fa5..ab1cc275ac1 100644 --- a/hadoop-ozone/interface-storage/pom.xml +++ b/hadoop-ozone/interface-storage/pom.xml @@ -31,11 +31,32 @@ + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-interface-client + org.apache.ozone ozone-common + + org.apache.ozone + rocksdb-checkpoint-differ + + + org.apache.ratis + ratis-common + + + + com.google.guava + guava + com.google.protobuf protobuf-java diff --git a/hadoop-ozone/ozone-manager/pom.xml b/hadoop-ozone/ozone-manager/pom.xml index 5f6bc54c5af..ae427727def 100644 --- a/hadoop-ozone/ozone-manager/pom.xml +++ b/hadoop-ozone/ozone-manager/pom.xml @@ -32,6 +32,35 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ozone + hdds-annotation-processing + test + + + org.apache.ozone + hdds-client + + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-interface-server + + + org.apache.ozone + hdds-managed-rocksdb + + + org.apache.ozone + ozone-interface-client + org.aspectj @@ -85,25 +114,132 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> rocksdb-checkpoint-differ + + org.apache.commons + commons-compress + + + org.apache.commons + commons-lang3 + + + org.apache.kerby + kerby-util + + + org.apache.logging.log4j + log4j-api + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-grpc + + + org.apache.ratis + ratis-netty + + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-server-api + + + org.apache.ratis + ratis-server + + + org.apache.ratis + ratis-thirdparty-misc + + org.bouncycastle bcprov-jdk18on + + io.grpc + grpc-api + + + com.google.code.findbugs + jsr305 + + + + + io.grpc + grpc-netty + + + io.grpc + grpc-stub + + + io.netty + netty-common + + + io.netty + netty-handler + io.netty netty-tcnative-boringssl-static runtime + + io.netty + netty-transport + org.reflections reflections + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.google.guava + guava + + + com.google.protobuf + protobuf-java + com.sun.jersey jersey-client + + info.picocli + picocli + + + jakarta.annotation + jakarta.annotation-api + + + jakarta.xml.bind + jakarta.xml.bind-api + + + javax.servlet + javax.servlet-api + @@ -134,7 +270,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ranger - ranger-intg + ranger-plugins-common ${ranger.version} compile diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMSystemAction.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMSystemAction.java new file mode 100644 index 00000000000..9f5b6ccebcf --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMSystemAction.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.ozone.audit; + +/** + * Enum to define Audit Action types for system audit in OzoneManager. This will in addition to OMAction + * as present for request. + */ +public enum OMSystemAction implements AuditAction { + STARTUP; + + @Override + public String getAction() { + return this.toString(); + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 03896a042ec..a514262cae2 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -26,6 +26,7 @@ import java.io.UncheckedIOException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.URI; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; @@ -90,7 +91,9 @@ import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.Table.KeyValue; import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.OzoneManagerVersion; +import org.apache.hadoop.ozone.audit.OMSystemAction; import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.SnapshotDiffJob; @@ -186,6 +189,7 @@ import org.apache.hadoop.ozone.om.protocolPB.OMAdminProtocolPB; import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; import org.apache.hadoop.hdds.security.exception.OzoneSecurityException; +import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource; import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; @@ -351,6 +355,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl private static final AuditLogger AUDIT = new AuditLogger( AuditLoggerType.OMLOGGER); + private static final AuditLogger SYSTEMAUDIT = new AuditLogger( + AuditLoggerType.OMSYSTEMLOGGER); + private static final String OM_DAEMON = "om"; // This is set for read requests when OMRequest has S3Authentication set, @@ -430,6 +437,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl private List ratisReporterList = null; private KeyProviderCryptoExtension kmsProvider; + private OzoneFsServerDefaults serverDefaults; private final OMLayoutVersionManager versionManager; private final ReplicationConfigValidator replicationConfigValidator; @@ -647,6 +655,14 @@ private OzoneManager(OzoneConfiguration conf, StartupOption startupOption) kmsProvider = null; LOG.error("Fail to create Key Provider"); } + Configuration hadoopConfig = + LegacyHadoopConfigurationSource.asHadoopConfiguration(configuration); + URI keyProviderUri = KMSUtil.getKeyProviderUri( + hadoopConfig, + CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH); + String keyProviderUriStr = + (keyProviderUri != null) ? keyProviderUri.toString() : null; + serverDefaults = new OzoneFsServerDefaults(keyProviderUriStr); if (secConfig.isSecurityEnabled()) { omComponent = OM_DAEMON + "-" + omId; HddsProtos.OzoneManagerDetailsProto omInfo = @@ -1644,11 +1660,15 @@ public OMPerformanceMetrics getPerfMetrics() { * Start service. */ public void start() throws IOException { + Map auditMap = new HashMap(); + auditMap.put("OmState", omState.name()); if (omState == State.BOOTSTRAPPING) { if (isBootstrapping) { + auditMap.put("Bootstrap", "normal"); // Check that all OM configs have been updated with the new OM info. checkConfigBeforeBootstrap(); } else if (isForcedBootstrapping) { + auditMap.put("Bootstrap", "force"); LOG.warn("Skipped checking whether existing OM configs have been " + "updated with this OM information as force bootstrap is called."); } @@ -1736,12 +1756,17 @@ public void start() throws IOException { } omState = State.RUNNING; + auditMap.put("NewOmState", omState.name()); + SYSTEMAUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMSystemAction.STARTUP, auditMap)); } /** * Restarts the service. This method re-initializes the rpc server. */ public void restart() throws IOException { + Map auditMap = new HashMap(); + auditMap.put("OmState", omState.name()); + auditMap.put("Trigger", "restart"); setInstanceVariablesFromConf(); LOG.info(buildRpcServerStartMessage("OzoneManager RPC server", @@ -1808,6 +1833,8 @@ public void restart() throws IOException { startJVMPauseMonitor(); setStartTime(); omState = State.RUNNING; + auditMap.put("NewOmState", omState.name()); + SYSTEMAUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMSystemAction.STARTUP, auditMap)); } /** @@ -3037,6 +3064,10 @@ public AuditLogger getAuditLogger() { return AUDIT; } + public AuditLogger getSystemAuditLogger() { + return SYSTEMAUDIT; + } + @Override public AuditMessage buildAuditMessageForSuccess(AuditAction op, Map auditMap) { @@ -4749,6 +4780,11 @@ public boolean setSafeMode(SafeModeAction action, boolean isChecked) } } + @Override + public OzoneFsServerDefaults getServerDefaults() { + return serverDefaults; + } + /** * Write down Layout version of a finalized feature to DB on finalization. * @param lvm OMLayoutVersionManager diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ScmClient.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ScmClient.java index 77ed7f63ad5..318decfa70f 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ScmClient.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ScmClient.java @@ -21,6 +21,8 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.CacheLoader.InvalidCacheLoadException; import com.google.common.cache.LoadingCache; +import org.apache.hadoop.hdds.client.ECReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; @@ -113,12 +115,29 @@ public Map getContainerLocations(Iterable containerIds, } try { Map result = containerLocationCache.getAll(containerIds); - // Don't keep empty pipelines in the cache. - List emptyPipelines = result.entrySet().stream() - .filter(e -> e.getValue().isEmpty()) + // Don't keep empty pipelines or insufficient EC pipelines in the cache. + List uncachePipelines = result.entrySet().stream() + .filter(e -> { + Pipeline pipeline = e.getValue(); + // filter empty pipelines + if (pipeline.isEmpty()) { + return true; + } + // filter insufficient EC pipelines which missing any data index + ReplicationConfig repConfig = pipeline.getReplicationConfig(); + if (repConfig instanceof ECReplicationConfig) { + int d = ((ECReplicationConfig) repConfig).getData(); + for (int i = 1; i <= d; i++) { + if (!pipeline.getReplicaIndexes().containsValue(i)) { + return true; + } + } + } + return false; + }) .map(Map.Entry::getKey) .collect(Collectors.toList()); - containerLocationCache.invalidateAll(emptyPipelines); + containerLocationCache.invalidateAll(uncachePipelines); return result; } catch (ExecutionException e) { return handleCacheExecutionException(e); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java index aa9612ef805..78d6ed89d2d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.conf.RatisConfUtils; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.ratis.RatisHelper; import org.apache.hadoop.hdds.security.SecurityConfig; @@ -624,17 +625,16 @@ public static RaftProperties newRaftProperties(ConfigurationSource conf, // Set Ratis storage directory RaftServerConfigKeys.setStorageDir(properties, Collections.singletonList(new File(ratisStorageDir))); - final int logAppenderQueueByteLimit = (int) conf.getStorageSize( + final int logAppenderBufferByteLimit = (int) conf.getStorageSize( OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT, OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT, StorageUnit.BYTES); + setRaftLogProperties(properties, logAppenderBufferByteLimit, conf); // For grpc config - setGrpcConfig(properties, logAppenderQueueByteLimit); + RatisConfUtils.Grpc.setMessageSizeMax(properties, logAppenderBufferByteLimit); setRaftLeaderElectionProperties(properties, conf); - setRaftLogProperties(properties, logAppenderQueueByteLimit, conf); - setRaftRpcProperties(properties, conf); setRaftRetryCacheProperties(properties, conf); @@ -693,12 +693,6 @@ private static void setRaftLogProperties(RaftProperties properties, RaftServerConfigKeys.Log.setSegmentCacheNumMax(properties, 2); } - private static void setGrpcConfig(RaftProperties properties, int logAppenderQueueByteLimit) { - // For grpc set the maximum message size - // TODO: calculate the optimal max message size - GrpcConfigKeys.setMessageSizeMax(properties, SizeInBytes.valueOf(logAppenderQueueByteLimit)); - } - private static void setRaftRpcProperties(RaftProperties properties, ConfigurationSource conf) { // Set the server request timeout TimeUnit serverRequestTimeoutUnit = OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getUnit(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java index 8ff59e091d8..5dc640c742c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java @@ -84,6 +84,7 @@ import org.apache.hadoop.ozone.om.request.upgrade.OMFinalizeUpgradeRequest; import org.apache.hadoop.ozone.om.request.upgrade.OMPrepareRequest; import org.apache.hadoop.ozone.om.request.util.OMEchoRPCWriteRequest; +import org.apache.hadoop.ozone.om.request.volume.OMQuotaRepairRequest; import org.apache.hadoop.ozone.om.request.volume.OMVolumeCreateRequest; import org.apache.hadoop.ozone.om.request.volume.OMVolumeDeleteRequest; import org.apache.hadoop.ozone.om.request.volume.OMVolumeSetOwnerRequest; @@ -331,6 +332,8 @@ public static OMClientRequest createClientRequest(OMRequest omRequest, return new OMEchoRPCWriteRequest(omRequest); case AbortExpiredMultiPartUploads: return new S3ExpiredMultipartUploadsAbortRequest(omRequest); + case QuotaRepair: + return new OMQuotaRepairRequest(omRequest); default: throw new OMException("Unrecognized write command type request " + cmdType, OMException.ResultCodes.INVALID_REQUEST); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java index 29c7628e3cc..2a9cfa6baf0 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java @@ -19,10 +19,7 @@ package org.apache.hadoop.ozone.om.request.snapshot; -import org.apache.commons.lang3.tuple.Triple; -import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OMMetrics; -import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.ratis.server.protocol.TermIndex; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -44,15 +41,11 @@ import java.io.IOException; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Set; import java.util.UUID; -import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK; - /** * Handles OMSnapshotPurge Request. * This is an OM internal request. Does not need @RequireSnapshotFeatureState. @@ -92,62 +85,34 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn new HashMap<>(); // Each snapshot purge operation does three things: - // 1. Update the snapshot chain, - // 2. Update the deep clean flag for the next active snapshot (So that it can be + // 1. Update the deep clean flag for the next active snapshot (So that it can be // deep cleaned by the KeyDeletingService in the next run), + // 2. Update the snapshot chain, // 3. Finally, purge the snapshot. - // All of these steps have to be performed only when it acquires all the necessary - // locks (lock on the snapshot to be purged, lock on the next active snapshot, and - // lock on the next path and global previous snapshots). Ideally, there is no need - // for locks for snapshot purge and can rely on OMStateMachine because OMStateMachine - // is going to process each request sequentially. - // - // But there is a problem with that. After filtering unnecessary SST files for a snapshot, - // SstFilteringService updates that snapshot's SstFilter flag. SstFilteringService cannot - // use SetSnapshotProperty API because it runs on each OM independently and One OM does - // not know if the snapshot has been filtered on the other OM in HA environment. - // - // If locks are not taken snapshot purge and SstFilteringService will cause a race condition - // and override one's update with another. + // There is no need to take lock for snapshot purge as of now. We can simply rely on OMStateMachine + // because it executes transaction sequentially. for (String snapTableKey : snapshotDbKeys) { - // To acquire all the locks, a set is maintained which is keyed by snapshotTableKey. - // snapshotTableKey is nothing but /volumeName/bucketName/snapshotName. - // Once all the locks are acquired, it performs the three steps mentioned above and - // release all the locks after that. - Set> lockSet = new HashSet<>(4, 1); - try { - if (omMetadataManager.getSnapshotInfoTable().get(snapTableKey) == null) { - // Snapshot may have been purged in the previous iteration of SnapshotDeletingService. - LOG.warn("The snapshot {} is not longer in snapshot table, It maybe removed in the previous " + - "Snapshot purge request.", snapTableKey); - continue; - } - - acquireLock(lockSet, snapTableKey, omMetadataManager); - SnapshotInfo fromSnapshot = omMetadataManager.getSnapshotInfoTable().get(snapTableKey); - - SnapshotInfo nextSnapshot = - SnapshotUtils.getNextActiveSnapshot(fromSnapshot, snapshotChainManager, omSnapshotManager); - - if (nextSnapshot != null) { - acquireLock(lockSet, nextSnapshot.getTableKey(), omMetadataManager); - } - - // Update the chain first so that it has all the necessary locks before updating deep clean. - updateSnapshotChainAndCache(lockSet, omMetadataManager, fromSnapshot, trxnLogIndex, - updatedPathPreviousAndGlobalSnapshots); - updateSnapshotInfoAndCache(nextSnapshot, omMetadataManager, trxnLogIndex, updatedSnapInfos); - // Remove and close snapshot's RocksDB instance from SnapshotCache. - omSnapshotManager.invalidateCacheEntry(fromSnapshot.getSnapshotId()); - // Update SnapshotInfoTable cache. - omMetadataManager.getSnapshotInfoTable() - .addCacheEntry(new CacheKey<>(fromSnapshot.getTableKey()), CacheValue.get(trxnLogIndex)); - } finally { - for (Triple lockKey: lockSet) { - omMetadataManager.getLock() - .releaseWriteLock(SNAPSHOT_LOCK, lockKey.getLeft(), lockKey.getMiddle(), lockKey.getRight()); - } + SnapshotInfo fromSnapshot = omMetadataManager.getSnapshotInfoTable().get(snapTableKey); + if (fromSnapshot == null) { + // Snapshot may have been purged in the previous iteration of SnapshotDeletingService. + LOG.warn("The snapshot {} is not longer in snapshot table, It maybe removed in the previous " + + "Snapshot purge request.", snapTableKey); + continue; } + + SnapshotInfo nextSnapshot = + SnapshotUtils.getNextActiveSnapshot(fromSnapshot, snapshotChainManager, omSnapshotManager); + + // Step 1: Update the deep clean flag for the next active snapshot + updateSnapshotInfoAndCache(nextSnapshot, omMetadataManager, trxnLogIndex, updatedSnapInfos); + // Step 2: Update the snapshot chain. + updateSnapshotChainAndCache(omMetadataManager, fromSnapshot, trxnLogIndex, + updatedPathPreviousAndGlobalSnapshots); + // Remove and close snapshot's RocksDB instance from SnapshotCache. + omSnapshotManager.invalidateCacheEntry(fromSnapshot.getSnapshotId()); + // Step 3: Purge the snapshot from SnapshotInfoTable cache. + omMetadataManager.getSnapshotInfoTable() + .addCacheEntry(new CacheKey<>(fromSnapshot.getTableKey()), CacheValue.get(trxnLogIndex)); } omClientResponse = new OMSnapshotPurgeResponse(omResponse.build(), @@ -168,41 +133,19 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn return omClientResponse; } - private void acquireLock(Set> lockSet, String snapshotTableKey, - OMMetadataManager omMetadataManager) throws IOException { - SnapshotInfo snapshotInfo = omMetadataManager.getSnapshotInfoTable().get(snapshotTableKey); - - // It should not be the case that lock is required for non-existing snapshot. - if (snapshotInfo == null) { - LOG.error("Snapshot: '{}' doesn't not exist in snapshot table.", snapshotTableKey); - throw new OMException("Snapshot: '{" + snapshotTableKey + "}' doesn't not exist in snapshot table.", - OMException.ResultCodes.FILE_NOT_FOUND); - } - Triple lockKey = Triple.of(snapshotInfo.getVolumeName(), snapshotInfo.getBucketName(), - snapshotInfo.getName()); - if (!lockSet.contains(lockKey)) { - mergeOmLockDetails(omMetadataManager.getLock() - .acquireWriteLock(SNAPSHOT_LOCK, lockKey.getLeft(), lockKey.getMiddle(), lockKey.getRight())); - lockSet.add(lockKey); - } - } - private void updateSnapshotInfoAndCache(SnapshotInfo snapInfo, OmMetadataManagerImpl omMetadataManager, long trxnLogIndex, - Map updatedSnapInfos) throws IOException { + Map updatedSnapInfos) throws IOException { if (snapInfo != null) { - // Fetch the latest value again after acquiring lock. - SnapshotInfo updatedSnapshotInfo = omMetadataManager.getSnapshotInfoTable().get(snapInfo.getTableKey()); - // Setting next snapshot deep clean to false, Since the // current snapshot is deleted. We can potentially // reclaim more keys in the next snapshot. - updatedSnapshotInfo.setDeepClean(false); + snapInfo.setDeepClean(false); // Update table cache first - omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(updatedSnapshotInfo.getTableKey()), - CacheValue.get(trxnLogIndex, updatedSnapshotInfo)); - updatedSnapInfos.put(updatedSnapshotInfo.getTableKey(), updatedSnapshotInfo); + omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(snapInfo.getTableKey()), + CacheValue.get(trxnLogIndex, snapInfo)); + updatedSnapInfos.put(snapInfo.getTableKey(), snapInfo); } } @@ -213,7 +156,6 @@ private void updateSnapshotInfoAndCache(SnapshotInfo snapInfo, * update in DB. */ private void updateSnapshotChainAndCache( - Set> lockSet, OmMetadataManagerImpl metadataManager, SnapshotInfo snapInfo, long trxnLogIndex, @@ -247,18 +189,12 @@ private void updateSnapshotChainAndCache( snapInfo.getSnapshotPath(), snapInfo.getSnapshotId()); nextPathSnapshotKey = snapshotChainManager .getTableKey(nextPathSnapshotId); - - // Acquire lock from the snapshot - acquireLock(lockSet, nextPathSnapshotKey, metadataManager); } String nextGlobalSnapshotKey = null; if (hasNextGlobalSnapshot) { UUID nextGlobalSnapshotId = snapshotChainManager.nextGlobalSnapshot(snapInfo.getSnapshotId()); nextGlobalSnapshotKey = snapshotChainManager.getTableKey(nextGlobalSnapshotId); - - // Acquire lock from the snapshot - acquireLock(lockSet, nextGlobalSnapshotKey, metadataManager); } SnapshotInfo nextPathSnapInfo = diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java index c4ca3dc99e3..53047fd8026 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java @@ -38,7 +38,6 @@ import java.io.IOException; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND; -import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK; /** * Updates the exclusive size of the snapshot. @@ -55,7 +54,7 @@ public OMSnapshotSetPropertyRequest(OMRequest omRequest) { public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIndex termIndex) { OMMetrics omMetrics = ozoneManager.getMetrics(); - OMClientResponse omClientResponse = null; + OMClientResponse omClientResponse; OMMetadataManager metadataManager = ozoneManager.getMetadataManager(); OzoneManagerProtocolProtos.OMResponse.Builder omResponse = @@ -63,33 +62,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn OzoneManagerProtocolProtos.SetSnapshotPropertyRequest setSnapshotPropertyRequest = getOmRequest() .getSetSnapshotPropertyRequest(); - SnapshotInfo updatedSnapInfo = null; String snapshotKey = setSnapshotPropertyRequest.getSnapshotKey(); - boolean acquiredSnapshotLock = false; - String volumeName = null; - String bucketName = null; - String snapshotName = null; try { - SnapshotInfo snapshotInfo = metadataManager.getSnapshotInfoTable().get(snapshotKey); - if (snapshotInfo == null) { + SnapshotInfo updatedSnapInfo = metadataManager.getSnapshotInfoTable().get(snapshotKey); + if (updatedSnapInfo == null) { LOG.error("Snapshot: '{}' doesn't not exist in snapshot table.", snapshotKey); throw new OMException("Snapshot: '{" + snapshotKey + "}' doesn't not exist in snapshot table.", FILE_NOT_FOUND); } - volumeName = snapshotInfo.getVolumeName(); - bucketName = snapshotInfo.getBucketName(); - snapshotName = snapshotInfo.getName(); - - mergeOmLockDetails(metadataManager.getLock() - .acquireWriteLock(SNAPSHOT_LOCK, volumeName, bucketName, snapshotName)); - - acquiredSnapshotLock = getOmLockDetails().isLockAcquired(); - - updatedSnapInfo = metadataManager.getSnapshotInfoTable() - .get(snapshotKey); - if (setSnapshotPropertyRequest.hasDeepCleanedDeletedDir()) { updatedSnapInfo.setDeepCleanedDeletedDir(setSnapshotPropertyRequest @@ -126,14 +108,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn createErrorOMResponse(omResponse, ex)); omMetrics.incNumSnapshotSetPropertyFails(); LOG.error("Failed to execute snapshotSetPropertyRequest: {{}}.", setSnapshotPropertyRequest, ex); - } finally { - if (acquiredSnapshotLock) { - mergeOmLockDetails(metadataManager.getLock() - .releaseWriteLock(SNAPSHOT_LOCK, volumeName, bucketName, snapshotName)); - } - if (omClientResponse != null) { - omClientResponse.setOmLockDetails(getOmLockDetails()); - } } return omClientResponse; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMQuotaRepairRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMQuotaRepairRequest.java new file mode 100644 index 00000000000..e307a1f95fd --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMQuotaRepairRequest.java @@ -0,0 +1,191 @@ +/** + * 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.om.request.volume; + +import com.google.common.base.Preconditions; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.request.OMClientRequest; +import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.om.response.volume.OMQuotaRepairResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.ratis.server.protocol.TermIndex; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.ozone.OzoneConsts.OLD_QUOTA_DEFAULT; +import static org.apache.hadoop.ozone.OzoneConsts.QUOTA_RESET; +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK; + +/** + * Handle OMQuotaRepairRequest Request. + */ +public class OMQuotaRepairRequest extends OMClientRequest { + private static final Logger LOG = + LoggerFactory.getLogger(OMQuotaRepairRequest.class); + + public OMQuotaRepairRequest(OMRequest omRequest) { + super(omRequest); + } + + @Override + public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { + UserGroupInformation ugi = createUGIForApi(); + if (ozoneManager.getAclsEnabled() && !ozoneManager.isAdmin(ugi)) { + throw new OMException("Access denied for user " + ugi + ". Admin privilege is required for quota repair.", + OMException.ResultCodes.ACCESS_DENIED); + } + return super.preExecute(ozoneManager); + } + + @Override + @SuppressWarnings("methodlength") + public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIndex termIndex) { + final long transactionLogIndex = termIndex.getIndex(); + OzoneManagerProtocolProtos.QuotaRepairRequest quotaRepairRequest = + getOmRequest().getQuotaRepairRequest(); + Preconditions.checkNotNull(quotaRepairRequest); + + OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager(); + OzoneManagerProtocolProtos.OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(getOmRequest()); + Map, OmBucketInfo> bucketMap = new HashMap<>(); + OMClientResponse omClientResponse = null; + try { + for (int i = 0; i < quotaRepairRequest.getBucketCountCount(); ++i) { + OzoneManagerProtocolProtos.BucketQuotaCount bucketCountInfo = quotaRepairRequest.getBucketCount(i); + updateBucketInfo(omMetadataManager, bucketCountInfo, transactionLogIndex, bucketMap); + } + Map volUpdateMap; + if (quotaRepairRequest.getSupportVolumeOldQuota()) { + volUpdateMap = updateOldVolumeQuotaSupport(omMetadataManager, transactionLogIndex); + } else { + volUpdateMap = Collections.emptyMap(); + } + omResponse.setQuotaRepairResponse( + OzoneManagerProtocolProtos.QuotaRepairResponse.newBuilder().build()); + omClientResponse = new OMQuotaRepairResponse(omResponse.build(), volUpdateMap, bucketMap); + } catch (IOException ex) { + LOG.error("failed to update repair count", ex); + omClientResponse = new OMQuotaRepairResponse(createErrorOMResponse(omResponse, ex)); + } finally { + if (omClientResponse != null) { + omClientResponse.setOmLockDetails(getOmLockDetails()); + } + } + + return omClientResponse; + } + + private void updateBucketInfo( + OMMetadataManager omMetadataManager, OzoneManagerProtocolProtos.BucketQuotaCount bucketCountInfo, + long transactionLogIndex, Map, OmBucketInfo> bucketMap) throws IOException { + // acquire lock. + mergeOmLockDetails(omMetadataManager.getLock().acquireWriteLock( + BUCKET_LOCK, bucketCountInfo.getVolName(), bucketCountInfo.getBucketName())); + boolean acquiredBucketLock = getOmLockDetails().isLockAcquired(); + try { + String bucketKey = omMetadataManager.getBucketKey(bucketCountInfo.getVolName(), + bucketCountInfo.getBucketName()); + OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(bucketKey); + if (null == bucketInfo) { + // bucket might be deleted when running repair count parallel + return; + } + bucketInfo.incrUsedBytes(bucketCountInfo.getDiffUsedBytes()); + bucketInfo.incrUsedNamespace(bucketCountInfo.getDiffUsedNamespace()); + if (bucketCountInfo.getSupportOldQuota()) { + OmBucketInfo.Builder builder = bucketInfo.toBuilder(); + if (bucketInfo.getQuotaInBytes() == OLD_QUOTA_DEFAULT) { + builder.setQuotaInBytes(QUOTA_RESET); + } + if (bucketInfo.getQuotaInNamespace() == OLD_QUOTA_DEFAULT) { + builder.setQuotaInNamespace(QUOTA_RESET); + } + bucketInfo = builder.build(); + } + + omMetadataManager.getBucketTable().addCacheEntry( + new CacheKey<>(bucketKey), CacheValue.get(transactionLogIndex, bucketInfo)); + bucketMap.put(Pair.of(bucketCountInfo.getVolName(), bucketCountInfo.getBucketName()), bucketInfo); + } finally { + if (acquiredBucketLock) { + mergeOmLockDetails(omMetadataManager.getLock() + .releaseWriteLock(BUCKET_LOCK, bucketCountInfo.getVolName(), bucketCountInfo.getBucketName())); + } + } + } + + private Map updateOldVolumeQuotaSupport( + OMMetadataManager metadataManager, long transactionLogIndex) throws IOException { + LOG.info("Starting volume quota support update"); + Map volUpdateMap = new HashMap<>(); + try (TableIterator> + iterator = metadataManager.getVolumeTable().iterator()) { + while (iterator.hasNext()) { + Table.KeyValue entry = iterator.next(); + OmVolumeArgs omVolumeArgs = entry.getValue(); + if (!(omVolumeArgs.getQuotaInBytes() == OLD_QUOTA_DEFAULT + || omVolumeArgs.getQuotaInNamespace() == OLD_QUOTA_DEFAULT)) { + continue; + } + mergeOmLockDetails(metadataManager.getLock().acquireWriteLock( + VOLUME_LOCK, omVolumeArgs.getVolume())); + boolean acquiredVolumeLock = getOmLockDetails().isLockAcquired(); + try { + boolean isQuotaReset = false; + if (omVolumeArgs.getQuotaInBytes() == OLD_QUOTA_DEFAULT) { + omVolumeArgs.setQuotaInBytes(QUOTA_RESET); + isQuotaReset = true; + } + if (omVolumeArgs.getQuotaInNamespace() == OLD_QUOTA_DEFAULT) { + omVolumeArgs.setQuotaInNamespace(QUOTA_RESET); + isQuotaReset = true; + } + if (isQuotaReset) { + metadataManager.getVolumeTable().addCacheEntry( + new CacheKey<>(entry.getKey()), CacheValue.get(transactionLogIndex, omVolumeArgs)); + volUpdateMap.put(entry.getKey(), omVolumeArgs); + } + } finally { + if (acquiredVolumeLock) { + mergeOmLockDetails(metadataManager.getLock().releaseWriteLock(VOLUME_LOCK, omVolumeArgs.getVolume())); + } + } + } + } + LOG.info("Completed volume quota support update for volume count {}", volUpdateMap.size()); + return volUpdateMap; + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotPurgeResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotPurgeResponse.java index 45b0c5e0590..ea9e68cc9ad 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotPurgeResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotPurgeResponse.java @@ -81,9 +81,9 @@ protected void addToDBBatch(OMMetadataManager omMetadataManager, OmMetadataManagerImpl metadataManager = (OmMetadataManagerImpl) omMetadataManager; + updateSnapInfo(metadataManager, batchOperation, updatedSnapInfos); updateSnapInfo(metadataManager, batchOperation, updatedPreviousAndGlobalSnapInfos); - updateSnapInfo(metadataManager, batchOperation, updatedSnapInfos); for (String dbKey: snapshotDbKeys) { // Skip the cache here because snapshot is purged from cache in OMSnapshotPurgeRequest. SnapshotInfo snapshotInfo = omMetadataManager diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMQuotaRepairResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMQuotaRepairResponse.java new file mode 100644 index 00000000000..8fa028d7438 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMQuotaRepairResponse.java @@ -0,0 +1,72 @@ +/* + * 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.om.response.volume; + +import jakarta.annotation.Nonnull; +import java.io.IOException; +import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.request.volume.OMQuotaRepairRequest; +import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.BUCKET_TABLE; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.VOLUME_TABLE; + +/** + * Response for {@link OMQuotaRepairRequest} request. + */ +@CleanupTableInfo(cleanupTables = {VOLUME_TABLE, BUCKET_TABLE}) +public class OMQuotaRepairResponse extends OMClientResponse { + private Map volumeArgsMap; + private Map, OmBucketInfo> volBucketInfoMap; + + /** + * for quota failure response update. + */ + public OMQuotaRepairResponse(@Nonnull OMResponse omResponse) { + super(omResponse); + } + + public OMQuotaRepairResponse( + @Nonnull OMResponse omResponse, Map volumeArgsMap, + Map, OmBucketInfo> volBucketInfoMap) { + super(omResponse); + this.volBucketInfoMap = volBucketInfoMap; + this.volumeArgsMap = volumeArgsMap; + } + + @Override + public void addToDBBatch(OMMetadataManager metadataManager, + BatchOperation batchOp) throws IOException { + for (OmBucketInfo omBucketInfo : volBucketInfoMap.values()) { + metadataManager.getBucketTable().putWithBatch(batchOp, + metadataManager.getBucketKey(omBucketInfo.getVolumeName(), + omBucketInfo.getBucketName()), omBucketInfo); + } + for (OmVolumeArgs volArgs : volumeArgsMap.values()) { + metadataManager.getVolumeTable().putWithBatch(batchOp, volArgs.getVolume(), volArgs); + } + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/QuotaRepairTask.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/QuotaRepairTask.java index 8a8ebd06f4f..b3e64c98c5d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/QuotaRepairTask.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/QuotaRepairTask.java @@ -20,14 +20,18 @@ package org.apache.hadoop.ozone.om.service; import com.google.common.util.concurrent.UncheckedExecutionException; +import com.google.protobuf.ServiceException; +import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -36,25 +40,29 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.utils.db.DBCheckpoint; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; -import org.apache.hadoop.hdds.utils.db.cache.CacheKey; -import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OMRatisHelper; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; -import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.Message; +import org.apache.ratis.protocol.RaftClientRequest; +import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.ozone.OzoneConsts.OLD_QUOTA_DEFAULT; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; -import static org.apache.hadoop.ozone.OzoneConsts.QUOTA_RESET; -import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; -import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK; /** * Quota repair task. @@ -64,117 +72,189 @@ public class QuotaRepairTask { QuotaRepairTask.class); private static final int BATCH_SIZE = 5000; private static final int TASK_THREAD_CNT = 3; - public static final long EPOCH_DEFAULT = -1L; - private final OMMetadataManager metadataManager; - private final Map nameBucketInfoMap = new HashMap<>(); - private final Map idBucketInfoMap = new HashMap<>(); + private static final AtomicBoolean IN_PROGRESS = new AtomicBoolean(false); + private static final RepairStatus REPAIR_STATUS = new RepairStatus(); + private final OzoneManager om; + private final AtomicLong runCount = new AtomicLong(0); private ExecutorService executor; - private final Map keyCountMap = new ConcurrentHashMap<>(); - private final Map fileCountMap - = new ConcurrentHashMap<>(); - private final Map directoryCountMap - = new ConcurrentHashMap<>(); - private final Map oldVolumeKeyNameMap = new HashMap(); + public QuotaRepairTask(OzoneManager ozoneManager) { + this.om = ozoneManager; + } - public QuotaRepairTask(OMMetadataManager metadataManager) { - this.metadataManager = metadataManager; + public CompletableFuture repair() throws Exception { + // lock in progress operation and reject any other + if (!IN_PROGRESS.compareAndSet(false, true)) { + LOG.info("quota repair task already running"); + return CompletableFuture.supplyAsync(() -> false); + } + REPAIR_STATUS.reset(runCount.get() + 1); + return CompletableFuture.supplyAsync(() -> repairTask()); } - - public void repair() throws Exception { - LOG.info("Starting quota repair task"); - prepareAllVolumeBucketInfo(); - IOzoneManagerLock lock = metadataManager.getLock(); - // thread pool with 3 Table type * (1 task each + 3 thread each) - executor = Executors.newFixedThreadPool(12); + public static String getStatus() { + return REPAIR_STATUS.toString(); + } + private boolean repairTask() { + LOG.info("Starting quota repair task {}", REPAIR_STATUS); + OMMetadataManager activeMetaManager = null; try { - nameBucketInfoMap.values().stream().forEach(e -> lock.acquireReadLock( - BUCKET_LOCK, e.getVolumeName(), e.getBucketName())); - repairCount(); + // thread pool with 3 Table type * (1 task each + 3 thread for each task) + executor = Executors.newFixedThreadPool(3 * (1 + TASK_THREAD_CNT)); + OzoneManagerProtocolProtos.QuotaRepairRequest.Builder builder + = OzoneManagerProtocolProtos.QuotaRepairRequest.newBuilder(); + // repair active db + activeMetaManager = createActiveDBCheckpoint(om.getMetadataManager(), om.getConfiguration()); + repairActiveDb(activeMetaManager, builder); + + // TODO: repair snapshots for quota + + // submit request to update + ClientId clientId = ClientId.randomId(); + OzoneManagerProtocolProtos.OMRequest omRequest = OzoneManagerProtocolProtos.OMRequest.newBuilder() + .setCmdType(OzoneManagerProtocolProtos.Type.QuotaRepair) + .setQuotaRepairRequest(builder.build()) + .setClientId(clientId.toString()) + .build(); + OzoneManagerProtocolProtos.OMResponse response = submitRequest(omRequest, clientId); + if (response != null && !response.getSuccess()) { + LOG.error("update quota repair count response failed"); + REPAIR_STATUS.updateStatus("Response for update DB is failed"); + return false; + } else { + REPAIR_STATUS.updateStatus(builder, om.getMetadataManager()); + } + } catch (Exception exp) { + LOG.error("quota repair count failed", exp); + REPAIR_STATUS.updateStatus(exp.toString()); + return false; } finally { - nameBucketInfoMap.values().stream().forEach(e -> lock.releaseReadLock( - BUCKET_LOCK, e.getVolumeName(), e.getBucketName())); + LOG.info("Completed quota repair task {}", REPAIR_STATUS); executor.shutdown(); - LOG.info("Completed quota repair task"); + try { + if (null != activeMetaManager) { + activeMetaManager.stop(); + } + cleanTempCheckPointPath(om.getMetadataManager()); + } catch (Exception exp) { + LOG.error("failed to cleanup", exp); + } + IN_PROGRESS.set(false); } - updateOldVolumeQuotaSupport(); - - // cleanup epoch added to avoid extra epoch id in cache - ArrayList epochs = new ArrayList<>(); - epochs.add(EPOCH_DEFAULT); - metadataManager.getBucketTable().cleanupCache(epochs); - metadataManager.getVolumeTable().cleanupCache(epochs); + return true; } - - private void prepareAllVolumeBucketInfo() throws IOException { - try (TableIterator> - iterator = metadataManager.getVolumeTable().iterator()) { - OmVolumeArgs omVolumeArgs; - while (iterator.hasNext()) { - Table.KeyValue entry = - iterator.next(); - omVolumeArgs = entry.getValue(); - getAllBuckets(omVolumeArgs.getVolume(), omVolumeArgs.getObjectID()); - if (omVolumeArgs.getQuotaInBytes() == OLD_QUOTA_DEFAULT - || omVolumeArgs.getQuotaInNamespace() == OLD_QUOTA_DEFAULT) { - oldVolumeKeyNameMap.put(entry.getKey(), entry.getValue().getVolume()); - } + private void repairActiveDb( + OMMetadataManager metadataManager, + OzoneManagerProtocolProtos.QuotaRepairRequest.Builder builder) throws Exception { + Map nameBucketInfoMap = new HashMap<>(); + Map idBucketInfoMap = new HashMap<>(); + Map oriBucketInfoMap = new HashMap<>(); + prepareAllBucketInfo(nameBucketInfoMap, idBucketInfoMap, oriBucketInfoMap, metadataManager); + + repairCount(nameBucketInfoMap, idBucketInfoMap, metadataManager); + + // prepare and submit request to ratis + for (Map.Entry entry : nameBucketInfoMap.entrySet()) { + OmBucketInfo oriBucketInfo = oriBucketInfoMap.get(entry.getKey()); + OmBucketInfo updatedBuckedInfo = entry.getValue(); + boolean oldQuota = oriBucketInfo.getQuotaInBytes() == OLD_QUOTA_DEFAULT + || oriBucketInfo.getQuotaInNamespace() == OLD_QUOTA_DEFAULT; + if (!(oldQuota || isChange(oriBucketInfo, updatedBuckedInfo))) { + continue; } + OzoneManagerProtocolProtos.BucketQuotaCount.Builder bucketCountBuilder + = OzoneManagerProtocolProtos.BucketQuotaCount.newBuilder(); + bucketCountBuilder.setVolName(updatedBuckedInfo.getVolumeName()); + bucketCountBuilder.setBucketName(updatedBuckedInfo.getBucketName()); + bucketCountBuilder.setDiffUsedBytes(updatedBuckedInfo.getUsedBytes() - oriBucketInfo.getUsedBytes()); + bucketCountBuilder.setDiffUsedNamespace( + updatedBuckedInfo.getUsedNamespace() - oriBucketInfo.getUsedNamespace()); + bucketCountBuilder.setSupportOldQuota(oldQuota); + builder.addBucketCount(bucketCountBuilder.build()); } + + // update volume to support quota + builder.setSupportVolumeOldQuota(true); } - private void updateOldVolumeQuotaSupport() throws IOException { - LOG.info("Starting volume quota support update"); - IOzoneManagerLock lock = metadataManager.getLock(); - try (BatchOperation batchOperation = metadataManager.getStore() - .initBatchOperation()) { - for (Map.Entry volEntry - : oldVolumeKeyNameMap.entrySet()) { - lock.acquireReadLock(VOLUME_LOCK, volEntry.getValue()); - try { - OmVolumeArgs omVolumeArgs = metadataManager.getVolumeTable().get( - volEntry.getKey()); - boolean isQuotaReset = false; - if (omVolumeArgs.getQuotaInBytes() == OLD_QUOTA_DEFAULT) { - omVolumeArgs.setQuotaInBytes(QUOTA_RESET); - isQuotaReset = true; - } - if (omVolumeArgs.getQuotaInNamespace() == OLD_QUOTA_DEFAULT) { - omVolumeArgs.setQuotaInNamespace(QUOTA_RESET); - isQuotaReset = true; - } - if (isQuotaReset) { - metadataManager.getVolumeTable().addCacheEntry( - new CacheKey<>(volEntry.getKey()), - CacheValue.get(EPOCH_DEFAULT, omVolumeArgs)); - metadataManager.getVolumeTable().putWithBatch(batchOperation, - volEntry.getKey(), omVolumeArgs); - } - } finally { - lock.releaseReadLock(VOLUME_LOCK, volEntry.getValue()); - } + private OzoneManagerProtocolProtos.OMResponse submitRequest( + OzoneManagerProtocolProtos.OMRequest omRequest, ClientId clientId) { + try { + if (om.isRatisEnabled()) { + OzoneManagerRatisServer server = om.getOmRatisServer(); + RaftClientRequest raftClientRequest = RaftClientRequest.newBuilder() + .setClientId(clientId) + .setServerId(om.getOmRatisServer().getRaftPeerId()) + .setGroupId(om.getOmRatisServer().getRaftGroupId()) + .setCallId(runCount.getAndIncrement()) + .setMessage(Message.valueOf(OMRatisHelper.convertRequestToByteString(omRequest))) + .setType(RaftClientRequest.writeRequestType()) + .build(); + return server.submitRequest(omRequest, raftClientRequest); + } else { + return om.getOmServerProtocol().submitRequest( + null, omRequest); } - metadataManager.getStore().commitBatchOperation(batchOperation); + } catch (ServiceException e) { + LOG.error("repair quota count " + omRequest.getCmdType() + " request failed.", e); } - LOG.info("Completed volume quota support update"); + return null; } - private void getAllBuckets(String volumeName, long volumeId) - throws IOException { - List bucketList = metadataManager.listBuckets( - volumeName, null, null, Integer.MAX_VALUE, false); - for (OmBucketInfo bucketInfo : bucketList) { - bucketInfo.incrUsedNamespace(-bucketInfo.getUsedNamespace()); - bucketInfo.incrUsedBytes(-bucketInfo.getUsedBytes()); - nameBucketInfoMap.put(buildNamePath(volumeName, - bucketInfo.getBucketName()), bucketInfo); - idBucketInfoMap.put(buildIdPath(volumeId, bucketInfo.getObjectID()), - bucketInfo); + private OMMetadataManager createActiveDBCheckpoint( + OMMetadataManager omMetaManager, OzoneConfiguration conf) throws IOException { + // cleanup + String parentPath = cleanTempCheckPointPath(omMetaManager); + + // create snapshot + DBCheckpoint checkpoint = omMetaManager.getStore().getCheckpoint(parentPath, true); + return OmMetadataManagerImpl.createCheckpointMetadataManager(conf, checkpoint); + } + + private static String cleanTempCheckPointPath(OMMetadataManager omMetaManager) throws IOException { + File dbLocation = omMetaManager.getStore().getDbLocation(); + if (dbLocation == null) { + throw new NullPointerException("db location is null"); + } + String tempData = dbLocation.getParent(); + if (tempData == null) { + throw new NullPointerException("parent db dir is null"); + } + File repairTmpPath = Paths.get(tempData, "temp-repair-quota").toFile(); + FileUtils.deleteDirectory(repairTmpPath); + FileUtils.forceMkdir(repairTmpPath); + return repairTmpPath.toString(); + } + + private void prepareAllBucketInfo( + Map nameBucketInfoMap, Map idBucketInfoMap, + Map oriBucketInfoMap, OMMetadataManager metadataManager) throws IOException { + try (TableIterator> + iterator = metadataManager.getBucketTable().iterator()) { + while (iterator.hasNext()) { + Table.KeyValue entry = iterator.next(); + OmBucketInfo bucketInfo = entry.getValue(); + String bucketNameKey = buildNamePath(bucketInfo.getVolumeName(), + bucketInfo.getBucketName()); + oriBucketInfoMap.put(bucketNameKey, bucketInfo.copyObject()); + bucketInfo.incrUsedNamespace(-bucketInfo.getUsedNamespace()); + bucketInfo.incrUsedBytes(-bucketInfo.getUsedBytes()); + nameBucketInfoMap.put(bucketNameKey, bucketInfo); + idBucketInfoMap.put(buildIdPath(metadataManager.getVolumeId(bucketInfo.getVolumeName()), + bucketInfo.getObjectID()), bucketInfo); + } + } + } + + private boolean isChange(OmBucketInfo lBucketInfo, OmBucketInfo rBucketInfo) { + if (lBucketInfo.getUsedNamespace() != rBucketInfo.getUsedNamespace() + || lBucketInfo.getUsedBytes() != rBucketInfo.getUsedBytes()) { + return true; } + return false; } - private String buildNamePath(String volumeName, String bucketName) { + private static String buildNamePath(String volumeName, String bucketName) { final StringBuilder builder = new StringBuilder(); builder.append(OM_KEY_PREFIX) .append(volumeName) @@ -184,7 +264,7 @@ private String buildNamePath(String volumeName, String bucketName) { return builder.toString(); } - private String buildIdPath(long volumeId, long bucketId) { + private static String buildIdPath(long volumeId, long bucketId) { final StringBuilder builder = new StringBuilder(); builder.append(OM_KEY_PREFIX) .append(volumeId) @@ -194,8 +274,13 @@ private String buildIdPath(long volumeId, long bucketId) { return builder.toString(); } - private void repairCount() throws Exception { - LOG.info("Starting quota repair for all keys, files and directories"); + private void repairCount( + Map nameBucketInfoMap, Map idBucketInfoMap, + OMMetadataManager metadataManager) throws Exception { + LOG.info("Starting quota repair counting for all keys, files and directories"); + Map keyCountMap = new ConcurrentHashMap<>(); + Map fileCountMap = new ConcurrentHashMap<>(); + Map directoryCountMap = new ConcurrentHashMap<>(); try { nameBucketInfoMap.keySet().stream().forEach(e -> keyCountMap.put(e, new CountPair())); @@ -225,51 +310,11 @@ private void repairCount() throws Exception { throw new Exception(ex.getCause()); } - // persist bucket info + // update count to bucket info updateCountToBucketInfo(nameBucketInfoMap, keyCountMap); updateCountToBucketInfo(idBucketInfoMap, fileCountMap); updateCountToBucketInfo(idBucketInfoMap, directoryCountMap); - - // update quota enable flag for old volume and buckets - updateOldBucketQuotaSupport(); - - try (BatchOperation batchOperation = metadataManager.getStore() - .initBatchOperation()) { - for (Map.Entry entry - : nameBucketInfoMap.entrySet()) { - String bucketKey = metadataManager.getBucketKey( - entry.getValue().getVolumeName(), - entry.getValue().getBucketName()); - metadataManager.getBucketTable().putWithBatch(batchOperation, - bucketKey, entry.getValue()); - } - metadataManager.getStore().commitBatchOperation(batchOperation); - } - LOG.info("Completed quota repair for all keys, files and directories"); - } - - private void updateOldBucketQuotaSupport() { - for (Map.Entry entry : nameBucketInfoMap.entrySet()) { - if (entry.getValue().getQuotaInBytes() == OLD_QUOTA_DEFAULT - || entry.getValue().getQuotaInNamespace() == OLD_QUOTA_DEFAULT) { - OmBucketInfo.Builder builder = entry.getValue().toBuilder(); - if (entry.getValue().getQuotaInBytes() == OLD_QUOTA_DEFAULT) { - builder.setQuotaInBytes(QUOTA_RESET); - } - if (entry.getValue().getQuotaInNamespace() == OLD_QUOTA_DEFAULT) { - builder.setQuotaInNamespace(QUOTA_RESET); - } - OmBucketInfo bucketInfo = builder.build(); - entry.setValue(bucketInfo); - - // there is a new value to be updated in bucket cache - String bucketKey = metadataManager.getBucketKey( - bucketInfo.getVolumeName(), bucketInfo.getBucketName()); - metadataManager.getBucketTable().addCacheEntry( - new CacheKey<>(bucketKey), - CacheValue.get(EPOCH_DEFAULT, bucketInfo)); - } - } + LOG.info("Completed quota repair counting for all keys, files and directories"); } private void recalculateUsages( @@ -315,7 +360,7 @@ private void recalculateUsages( } } - private void captureCount( + private static void captureCount( Map prefixUsageMap, BlockingQueue>> q, AtomicBoolean isRunning, boolean haveValue) throws UncheckedIOException { @@ -334,7 +379,7 @@ private void captureCount( } } - private void extractCount( + private static void extractCount( Table.KeyValue kv, Map prefixUsageMap, boolean haveValue) { @@ -357,7 +402,7 @@ private void extractCount( } } - private synchronized void updateCountToBucketInfo( + private static synchronized void updateCountToBucketInfo( Map bucketInfoMap, Map prefixUsageMap) { for (Map.Entry entry : prefixUsageMap.entrySet()) { @@ -370,7 +415,7 @@ private synchronized void updateCountToBucketInfo( } } - private String getVolumeBucketPrefix(String key) { + private static String getVolumeBucketPrefix(String key) { // get bucket prefix with /// // -- as represents name in OBS and id in FSO String prefix = key; @@ -404,4 +449,66 @@ public long getNamespace() { return namespace.get(); } } + + /** + * Repair status for last run. + */ + public static class RepairStatus { + private boolean isTriggered = false; + private long taskId = 0; + private long lastRunStartTime = 0; + private long lastRunFinishedTime = 0; + private String errorMsg = null; + private Map> bucketCountDiffMap = new ConcurrentHashMap<>(); + + @Override + public String toString() { + if (!isTriggered) { + return "{}"; + } + Map status = new HashMap<>(); + status.put("taskId", taskId); + status.put("lastRunStartTime", lastRunStartTime); + status.put("lastRunFinishedTime", lastRunFinishedTime); + status.put("errorMsg", errorMsg); + status.put("bucketCountDiffMap", bucketCountDiffMap); + try { + return new ObjectMapper().writeValueAsString(status); + } catch (IOException e) { + LOG.error("error in generating status", e); + return "{}"; + } + } + + public void updateStatus(OzoneManagerProtocolProtos.QuotaRepairRequest.Builder builder, + OMMetadataManager metadataManager) { + isTriggered = true; + lastRunFinishedTime = System.currentTimeMillis(); + errorMsg = ""; + bucketCountDiffMap.clear(); + for (OzoneManagerProtocolProtos.BucketQuotaCount quotaCount : builder.getBucketCountList()) { + String bucketKey = metadataManager.getBucketKey(quotaCount.getVolName(), quotaCount.getBucketName()); + ConcurrentHashMap diffCountMap = new ConcurrentHashMap<>(); + diffCountMap.put("DiffUsedBytes", quotaCount.getDiffUsedBytes()); + diffCountMap.put("DiffUsedNamespace", quotaCount.getDiffUsedNamespace()); + bucketCountDiffMap.put(bucketKey, diffCountMap); + } + } + + public void updateStatus(String errMsg) { + isTriggered = true; + lastRunFinishedTime = System.currentTimeMillis(); + errorMsg = errMsg; + bucketCountDiffMap.clear(); + } + + public void reset(long tskId) { + isTriggered = true; + taskId = tskId; + lastRunStartTime = System.currentTimeMillis(); + lastRunFinishedTime = 0; + errorMsg = ""; + bucketCountDiffMap.clear(); + } + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java index 8a2e4f550e7..446c7382d50 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/QuotaRepairUpgradeAction.java @@ -21,26 +21,39 @@ import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException; +import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException; import org.apache.hadoop.ozone.om.service.QuotaRepairTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.QUOTA; -import static org.apache.hadoop.ozone.upgrade.LayoutFeature.UpgradeActionType.ON_FIRST_UPGRADE_START; +import static org.apache.hadoop.ozone.upgrade.LayoutFeature.UpgradeActionType.ON_FINALIZE; /** - * Quota repair for usages action to be triggered during first upgrade. + * Quota repair for usages action to be triggered after upgrade. */ -@UpgradeActionOm(type = ON_FIRST_UPGRADE_START, feature = - QUOTA) +@UpgradeActionOm(type = ON_FINALIZE, feature = QUOTA) public class QuotaRepairUpgradeAction implements OmUpgradeAction { + private static final Logger LOG = LoggerFactory.getLogger(QuotaRepairUpgradeAction.class); @Override public void execute(OzoneManager arg) throws Exception { boolean enabled = arg.getConfiguration().getBoolean( OMConfigKeys.OZONE_OM_UPGRADE_QUOTA_RECALCULATE_ENABLE, OMConfigKeys.OZONE_OM_UPGRADE_QUOTA_RECALCULATE_ENABLE_DEFAULT); if (enabled) { - QuotaRepairTask quotaRepairTask = new QuotaRepairTask( - arg.getMetadataManager()); - quotaRepairTask.repair(); + // just trigger quota repair and status can be checked via CLI + try { + if (arg.isRatisEnabled()) { + arg.checkLeaderStatus(); + } + QuotaRepairTask quotaRepairTask = new QuotaRepairTask(arg); + quotaRepairTask.repair(); + } catch (OMNotLeaderException | OMLeaderNotReadyException ex) { + // on leader node, repair will be triggered where finalize is called. For other nodes, it will be ignored. + // This can be triggered on need basis via CLI tool. + LOG.warn("Skip quota repair operation during upgrade on the node as this is not a leader node."); + } } } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java index eb9410501fa..a5e94689aee 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java @@ -133,6 +133,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetSafeModeRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetSafeModeResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetS3VolumeContextResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServerDefaultsResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotDiffRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotDiffResponse; @@ -383,6 +384,12 @@ public OMResponse handleReadRequest(OMRequest request) { getSnapshotInfo(request.getSnapshotInfoRequest()); responseBuilder.setSnapshotInfoResponse(snapshotInfoResponse); break; + case GetServerDefaults: + responseBuilder.setServerDefaultsResponse( + ServerDefaultsResponse.newBuilder() + .setServerDefaults(impl.getServerDefaults().getProtobuf()) + .build()); + break; default: responseBuilder.setSuccess(false); responseBuilder.setMessage("Unrecognized Command Type: " + cmdType); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OMCertificateClient.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OMCertificateClient.java index a6bfed699f7..0a28fc37f3c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OMCertificateClient.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/security/OMCertificateClient.java @@ -25,12 +25,12 @@ import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto; import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.security.SecurityConfig; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.client.DefaultCertificateClient; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; import org.apache.hadoop.hdds.security.x509.exception.CertificateException; import org.apache.hadoop.ozone.om.OMStorage; import org.apache.hadoop.security.UserGroupInformation; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +38,6 @@ import java.security.KeyPair; import java.util.function.Consumer; -import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest.getEncodedString; /** * Certificate client for OzoneManager. @@ -81,9 +80,9 @@ public OMCertificateClient( * @return CertificateSignRequest.Builder */ @Override - public CertificateSignRequest.Builder getCSRBuilder() - throws CertificateException { - CertificateSignRequest.Builder builder = super.getCSRBuilder(); + public CertificateSignRequest.Builder configureCSRBuilder() + throws SCMSecurityException { + CertificateSignRequest.Builder builder = super.configureCSRBuilder(); String hostname = omInfo.getHostName(); String subject; @@ -118,10 +117,8 @@ public CertificateSignRequest.Builder getCSRBuilder() } @Override - protected SCMGetCertResponseProto getCertificateSignResponse( - PKCS10CertificationRequest request) throws IOException { - return getScmSecureClient().getOMCertChain( - omInfo, getEncodedString(request)); + protected SCMGetCertResponseProto sign(CertificateSignRequest request) throws IOException { + return getScmSecureClient().getOMCertChain(omInfo, request.toEncodedFormat()); } @Override diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestScmClient.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestScmClient.java index 228f668d01a..08d9b9bc2cb 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestScmClient.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestScmClient.java @@ -43,7 +43,7 @@ import static com.google.common.collect.Sets.newHashSet; import static java.util.Arrays.asList; -import static org.apache.commons.lang.RandomStringUtils.randomAlphabetic; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.hadoop.hdds.client.ReplicationConfig.fromTypeAndFactor; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java index cf538f581c7..8163592cfc6 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java @@ -34,7 +34,7 @@ import java.util.concurrent.atomic.AtomicLong; import com.google.common.collect.ImmutableMap; -import org.apache.commons.lang.RandomStringUtils; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java index 1a0db118331..06b8beacb39 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestQuotaRepairTask.java @@ -22,8 +22,16 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; import static org.junit.jupiter.api.Assertions.assertEquals; - +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.om.helpers.BucketLayout; @@ -32,6 +40,11 @@ import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; import org.apache.hadoop.ozone.om.request.key.TestOMKeyRequest; +import org.apache.hadoop.ozone.om.request.volume.OMQuotaRepairRequest; +import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.om.response.volume.OMQuotaRepairResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB; import org.apache.hadoop.util.Time; import org.junit.jupiter.api.Test; @@ -44,6 +57,16 @@ public class TestQuotaRepairTask extends TestOMKeyRequest { @Test public void testQuotaRepair() throws Exception { + when(ozoneManager.isRatisEnabled()).thenReturn(false); + OzoneManagerProtocolProtos.OMResponse respMock = mock(OzoneManagerProtocolProtos.OMResponse.class); + when(respMock.getSuccess()).thenReturn(true); + OzoneManagerProtocolServerSideTranslatorPB serverMock = mock(OzoneManagerProtocolServerSideTranslatorPB.class); + AtomicReference ref = new AtomicReference<>(); + doAnswer(invocation -> { + ref.set(invocation.getArgument(1, OzoneManagerProtocolProtos.OMRequest.class)); + return respMock; + }).when(serverMock).submitRequest(any(), any()); + when(ozoneManager.getOmServerProtocol()).thenReturn(serverMock); OMRequestTestUtils.addVolumeAndBucketToDB(volumeName, bucketName, omMetadataManager, BucketLayout.OBJECT_STORE); @@ -88,9 +111,16 @@ public void testQuotaRepair() throws Exception { assertEquals(0, fsoBucketInfo.getUsedNamespace()); assertEquals(0, fsoBucketInfo.getUsedBytes()); - QuotaRepairTask quotaRepairTask = new QuotaRepairTask(omMetadataManager); - quotaRepairTask.repair(); - + QuotaRepairTask quotaRepairTask = new QuotaRepairTask(ozoneManager); + CompletableFuture repair = quotaRepairTask.repair(); + Boolean repairStatus = repair.get(); + assertTrue(repairStatus); + + OMQuotaRepairRequest omQuotaRepairRequest = new OMQuotaRepairRequest(ref.get()); + OMClientResponse omClientResponse = omQuotaRepairRequest.validateAndUpdateCache(ozoneManager, 1); + BatchOperation batchOperation = omMetadataManager.getStore().initBatchOperation(); + ((OMQuotaRepairResponse)omClientResponse).addToDBBatch(omMetadataManager, batchOperation); + omMetadataManager.getStore().commitBatchOperation(batchOperation); // 10 files of each type, obs have replication of three and // fso have replication of one OmBucketInfo obsUpdateBucketInfo = omMetadataManager.getBucketTable().get( @@ -105,6 +135,16 @@ public void testQuotaRepair() throws Exception { @Test public void testQuotaRepairForOldVersionVolumeBucket() throws Exception { + when(ozoneManager.isRatisEnabled()).thenReturn(false); + OzoneManagerProtocolProtos.OMResponse respMock = mock(OzoneManagerProtocolProtos.OMResponse.class); + when(respMock.getSuccess()).thenReturn(true); + OzoneManagerProtocolServerSideTranslatorPB serverMock = mock(OzoneManagerProtocolServerSideTranslatorPB.class); + AtomicReference ref = new AtomicReference<>(); + doAnswer(invocation -> { + ref.set(invocation.getArgument(1, OzoneManagerProtocolProtos.OMRequest.class)); + return respMock; + }).when(serverMock).submitRequest(any(), any()); + when(ozoneManager.getOmServerProtocol()).thenReturn(serverMock); // add volume with -2 value OmVolumeArgs omVolumeArgs = OmVolumeArgs.newBuilder().setCreationTime(Time.now()) @@ -117,13 +157,14 @@ public void testQuotaRepairForOldVersionVolumeBucket() throws Exception { new CacheKey<>(omMetadataManager.getVolumeKey(volumeName)), CacheValue.get(1L, omVolumeArgs)); - // add bucket with -2 value + // add bucket with -2 value and add to db OMRequestTestUtils.addBucketToDB(volumeName, bucketName, omMetadataManager, -2); + String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName); + omMetadataManager.getBucketTable().put(bucketKey, omMetadataManager.getBucketTable().get(bucketKey)); // pre check for quota flag - OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get( - omMetadataManager.getBucketKey(volumeName, bucketName)); + OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(bucketKey); assertEquals(-2, bucketInfo.getQuotaInBytes()); omVolumeArgs = omMetadataManager.getVolumeTable().get( @@ -131,11 +172,18 @@ public void testQuotaRepairForOldVersionVolumeBucket() throws Exception { assertEquals(-2, omVolumeArgs.getQuotaInBytes()); assertEquals(-2, omVolumeArgs.getQuotaInNamespace()); - QuotaRepairTask quotaRepairTask = new QuotaRepairTask(omMetadataManager); - quotaRepairTask.repair(); + QuotaRepairTask quotaRepairTask = new QuotaRepairTask(ozoneManager); + CompletableFuture repair = quotaRepairTask.repair(); + Boolean repairStatus = repair.get(); + assertTrue(repairStatus); + OMQuotaRepairRequest omQuotaRepairRequest = new OMQuotaRepairRequest(ref.get()); + OMClientResponse omClientResponse = omQuotaRepairRequest.validateAndUpdateCache(ozoneManager, 1); + BatchOperation batchOperation = omMetadataManager.getStore().initBatchOperation(); + ((OMQuotaRepairResponse)omClientResponse).addToDBBatch(omMetadataManager, batchOperation); + omMetadataManager.getStore().commitBatchOperation(batchOperation); bucketInfo = omMetadataManager.getBucketTable().get( - omMetadataManager.getBucketKey(volumeName, bucketName)); + bucketKey); assertEquals(-1, bucketInfo.getQuotaInBytes()); OmVolumeArgs volArgsVerify = omMetadataManager.getVolumeTable() .get(omMetadataManager.getVolumeKey(volumeName)); diff --git a/hadoop-ozone/ozonefs-common/pom.xml b/hadoop-ozone/ozonefs-common/pom.xml index 99ab7ba21bf..6132f9bc125 100644 --- a/hadoop-ozone/ozonefs-common/pom.xml +++ b/hadoop-ozone/ozonefs-common/pom.xml @@ -43,10 +43,22 @@ + + org.apache.ozone + hdds-client + + + org.apache.ozone + hdds-config + org.apache.ozone hdds-hadoop-dependency-client + + org.apache.ozone + hdds-interface-client + org.apache.ozone ozone-client @@ -60,6 +72,45 @@ ozone-common + + commons-collections + commons-collections + + + org.apache.commons + commons-lang3 + + + org.apache.httpcomponents + httpclient + + + org.apache.ratis + ratis-common + + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + + + io.opentracing + opentracing-api + + + io.opentracing + opentracing-util + + + jakarta.annotation + jakarta.annotation-api + + org.apache.ozone diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java index 658685779e9..df8ece03486 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java @@ -55,6 +55,7 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; @@ -463,6 +464,11 @@ public Token getDelegationToken(String renewer) } + @Override + public OzoneFsServerDefaults getServerDefaults() throws IOException { + return objectStore.getServerDefaults(); + } + @Override public KeyProvider getKeyProvider() throws IOException { return objectStore.getKeyProvider(); diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java index aabf17d086e..f25d9011475 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.client.io.SelectorOutputStream; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils; @@ -865,6 +866,11 @@ public short getDefaultReplication() { return adapter.getDefaultReplication(); } + @Override + public OzoneFsServerDefaults getServerDefaults() throws IOException { + return adapter.getServerDefaults(); + } + @Override public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst) throws IOException { diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java index bb2725063e3..14c297d9f47 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.ozone.OFSPath; import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; @@ -963,6 +964,11 @@ public ObjectStore getObjectStore() { return objectStore; } + @Override + public OzoneFsServerDefaults getServerDefaults() throws IOException { + return objectStore.getServerDefaults(); + } + @Override public KeyProvider getKeyProvider() throws IOException { return objectStore.getKeyProvider(); diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java index 224d94baa5d..eb346b5edc5 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.ozone.OFSPath; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.io.SelectorOutputStream; @@ -722,7 +723,7 @@ private boolean deleteInSpan(Path f, boolean recursive) throws IOException { throw new IOException("Recursive volume delete using " + "ofs is not supported. " + "Instead use 'ozone sh volume delete -r " + - "-id ' command"); + "o3:///' command"); } return deleteVolume(f, ofsPath); } @@ -1107,6 +1108,11 @@ public short getDefaultReplication() { return adapter.getDefaultReplication(); } + @Override + public OzoneFsServerDefaults getServerDefaults() throws IOException { + return adapter.getServerDefaults(); + } + @Override public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst) throws IOException { diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java index 7e78d6650ee..e468ac498c4 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.SafeModeAction; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; @@ -74,6 +75,8 @@ List listStatus(String keyName, boolean recursive, Token getDelegationToken(String renewer) throws IOException; + + OzoneFsServerDefaults getServerDefaults() throws IOException; KeyProvider getKeyProvider() throws IOException; diff --git a/hadoop-ozone/ozonefs/pom.xml b/hadoop-ozone/ozonefs/pom.xml index 33bbb893cfd..176f21b9860 100644 --- a/hadoop-ozone/ozonefs/pom.xml +++ b/hadoop-ozone/ozonefs/pom.xml @@ -73,15 +73,49 @@ + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-common + org.apache.ozone ozone-client + + org.apache.ozone + ozone-common + org.apache.ozone ozone-filesystem-common + + org.apache.ratis + ratis-common + + + + com.google.guava + guava + + + io.opentracing + opentracing-api + + + io.opentracing + opentracing-util + + + org.slf4j + slf4j-api + + org.apache.ozone diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml index 059db6b9513..e262895664f 100644 --- a/hadoop-ozone/pom.xml +++ b/hadoop-ozone/pom.xml @@ -138,6 +138,11 @@ ozone-filesystem-hadoop2 ${ozone.version} + + org.apache.ozone + hdds-annotation-processing + ${hdds.version} + org.apache.ozone hdds-config @@ -148,11 +153,26 @@ hdds-erasurecode ${hdds.version} + + org.apache.ozone + hdds-interface-admin + ${hdds.version} + org.apache.ozone hdds-interface-client ${hdds.version} + + org.apache.ozone + hdds-interface-server + ${hdds.version} + + + org.apache.ozone + hdds-managed-rocksdb + ${hdds.version} + org.apache.ozone ozone-s3-secret-store diff --git a/hadoop-ozone/recon-codegen/pom.xml b/hadoop-ozone/recon-codegen/pom.xml index 26a70c45974..bb7756a9de3 100644 --- a/hadoop-ozone/recon-codegen/pom.xml +++ b/hadoop-ozone/recon-codegen/pom.xml @@ -28,18 +28,36 @@ + + org.apache.ozone + hdds-config + org.apache.ozone ozone-common + + + commons-io + commons-io + org.apache.derby derby + + + org.slf4j + slf4j-api + org.springframework spring-jdbc + + org.springframework + spring-tx + org.jooq jooq-codegen diff --git a/hadoop-ozone/recon/pom.xml b/hadoop-ozone/recon/pom.xml index 0e75e0850b0..a24252c1ed6 100644 --- a/hadoop-ozone/recon/pom.xml +++ b/hadoop-ozone/recon/pom.xml @@ -238,10 +238,46 @@ + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-container-service + + + org.apache.ozone + hdds-interface-client + + + org.apache.ozone + hdds-interface-server + + + org.apache.ozone + hdds-managed-rocksdb + + + org.apache.ozone + hdds-server-framework + org.apache.ozone ozone-common + + org.apache.ozone + ozone-interface-client + + + org.apache.ozone + ozone-interface-storage + org.apache.ozone ozone-reconcodegen @@ -272,6 +308,48 @@ org.apache.ozone hdds-server-scm + + + commons-collections + commons-collections + + + org.apache.commons + commons-compress + + + commons-io + commons-io + + + org.apache.commons + commons-lang3 + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-proto + + + + aopalliance + aopalliance + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.google.guava + guava + com.google.inject guice @@ -284,10 +362,18 @@ com.google.inject.extensions guice-servlet + + com.google.protobuf + protobuf-java + org.glassfish.jersey.containers jersey-container-servlet + + info.picocli + picocli + org.glassfish.jersey.containers jersey-container-servlet-core @@ -296,6 +382,14 @@ org.glassfish.hk2 guice-bridge + + org.glassfish.hk2.external + jakarta.inject + + + org.glassfish.jersey.core + jersey-common + org.glassfish.jersey.core jersey-server @@ -332,18 +426,74 @@ org.apache.derby derby + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-util + + + org.glassfish.hk2 + hk2-api + + + org.reflections + reflections + + + org.rocksdb + rocksdbjni + org.xerial sqlite-jdbc + + org.slf4j + slf4j-api + + + org.springframework + spring-core + org.springframework spring-jdbc + + org.springframework + spring-tx + jakarta.activation jakarta.activation-api + + jakarta.annotation + jakarta.annotation-api + + + jakarta.validation + jakarta.validation-api + + + jakarta.ws.rs + jakarta.ws.rs-api + + + jakarta.xml.bind + jakarta.xml.bind-api + + + javax.inject + javax.inject + + + javax.servlet + javax.servlet-api + org.javassist javassist diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java index 15fea3574aa..91cb61369fc 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java @@ -169,8 +169,16 @@ public List listVolumes(String startKey, // Unlike in {@link OmMetadataManagerImpl}, the volumes are queried directly // from the volume table (not through cache) since Recon does not use // Table cache. + Table volumeTable = getVolumeTable(); + + // If the table is not yet initialized, i.e. it is null + // Return empty list as response + if (volumeTable == null) { + return result; + } + try (TableIterator> - iterator = getVolumeTable().iterator()) { + iterator = volumeTable.iterator()) { while (iterator.hasNext() && result.size() < maxKeys) { Table.KeyValue kv = iterator.next(); @@ -296,6 +304,11 @@ private List listAllBuckets(final int maxNumberOfBuckets) int currentCount = 0; Table bucketTable = getBucketTable(); + // If the table is not yet initialized, i.e. it is null + // Return empty list as response + if (bucketTable == null) { + return result; + } try (TableIterator> iterator = bucketTable.iterator()) { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/security/ReconCertificateClient.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/security/ReconCertificateClient.java index 92b540ecd13..95fdfabbb88 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/security/ReconCertificateClient.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/security/ReconCertificateClient.java @@ -21,12 +21,12 @@ import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto; import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.security.SecurityConfig; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.client.DefaultCertificateClient; import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest; import org.apache.hadoop.hdds.security.x509.exception.CertificateException; import org.apache.hadoop.ozone.recon.scm.ReconStorageConfig; import org.apache.hadoop.security.UserGroupInformation; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,7 +35,6 @@ import java.security.KeyPair; import java.util.function.Consumer; -import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest.getEncodedString; import static org.apache.hadoop.hdds.security.x509.exception.CertificateException.ErrorCode.CSR_ERROR; /** @@ -62,11 +61,11 @@ public ReconCertificateClient( } @Override - public CertificateSignRequest.Builder getCSRBuilder() - throws CertificateException { + public CertificateSignRequest.Builder configureCSRBuilder() + throws SCMSecurityException { LOG.info("Creating CSR for Recon."); try { - CertificateSignRequest.Builder builder = super.getCSRBuilder(); + CertificateSignRequest.Builder builder = super.configureCSRBuilder(); String hostname = InetAddress.getLocalHost().getCanonicalHostName(); String subject = UserGroupInformation.getCurrentUser() .getShortUserName() + "@" + hostname; @@ -85,8 +84,7 @@ public CertificateSignRequest.Builder getCSRBuilder() } @Override - protected SCMGetCertResponseProto getCertificateSignResponse( - PKCS10CertificationRequest request) throws IOException { + protected SCMGetCertResponseProto sign(CertificateSignRequest request) throws IOException { SCMGetCertResponseProto response; HddsProtos.NodeDetailsProto.Builder reconDetailsProtoBuilder = HddsProtos.NodeDetailsProto.newBuilder() @@ -95,8 +93,7 @@ protected SCMGetCertResponseProto getCertificateSignResponse( .setUuid(reconID) .setNodeType(HddsProtos.NodeType.RECON); // TODO: For SCM CA we should fetch certificate from multiple SCMs. - response = getScmSecureClient().getCertificateChain( - reconDetailsProtoBuilder.build(), getEncodedString(request)); + response = getScmSecureClient().getCertificateChain(reconDetailsProtoBuilder.build(), request.toEncodedFormat()); return response; } diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json index 5416ca7f00c..8cfb23ad685 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json @@ -5,8 +5,9 @@ "healthyDatanodes": 24, "storageReport": { "capacity": 202114732032, - "used": 16384, - "remaining": 182447632384 + "used": 4667099648, + "remaining": 182447632384, + "committed": 12000222315 }, "containers": 3230, "missingContainers": 1002, diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json index 71bdf85ad1c..d931a0ed79b 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/package.json @@ -56,6 +56,8 @@ ] }, "devDependencies": { + "@testing-library/jest-dom": "^6.4.8", + "@testing-library/react": "^12.1.5", "@types/react": "16.8.15", "@types/react-dom": "16.8.4", "@types/react-router-dom": "^4.3.5", @@ -66,7 +68,9 @@ "eslint": "^7.28.0", "eslint-config-prettier": "^8.10.0", "eslint-plugin-prettier": "^3.4.1", + "jsdom": "^24.1.1", "json-server": "^0.15.1", + "msw": "1.3.3", "npm-run-all": "^4.1.5", "prettier": "^2.8.4", "vite": "4.5.3", diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml index ccfdcc01862..d1b8844ac62 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/pnpm-lock.yaml @@ -61,6 +61,12 @@ dependencies: version: 4.9.5 devDependencies: + '@testing-library/jest-dom': + specifier: ^6.4.8 + version: 6.4.8 + '@testing-library/react': + specifier: ^12.1.5 + version: 12.1.5(react-dom@16.14.0)(react@16.14.0) '@types/react': specifier: 16.8.15 version: 16.8.15 @@ -91,9 +97,15 @@ devDependencies: eslint-plugin-prettier: specifier: ^3.4.1 version: 3.4.1(eslint-config-prettier@8.10.0)(eslint@7.32.0)(prettier@2.8.8) + jsdom: + specifier: ^24.1.1 + version: 24.1.1 json-server: specifier: ^0.15.1 version: 0.15.1 + msw: + specifier: 1.3.3 + version: 1.3.3(typescript@4.9.5) npm-run-all: specifier: ^4.1.5 version: 4.1.5 @@ -111,10 +123,14 @@ devDependencies: version: 3.6.0(vite@4.5.3) vitest: specifier: ^1.6.0 - version: 1.6.0(less@3.13.1) + version: 1.6.0(jsdom@24.1.1)(less@3.13.1) packages: + /@adobe/css-tools@4.4.0: + resolution: {integrity: sha512-Ff9+ksdQQB3rMncgqDK78uLznstjyfIf2Arnh22pW8kBpLs6rpKDwgnZT46hin5Hl1WzazzK64DOrhSwYpS7bQ==} + dev: true + /@ampproject/remapping@2.3.0: resolution: {integrity: sha512-30iZtAPgz+LTIYoeivqYo853f02jBYSd5uGnGpkFV0M3xOt9aN73erkgYAmZU43x4VfqcnLxW9Kpg3R5LC4YYw==} engines: {node: '>=6.0.0'} @@ -975,6 +991,30 @@ packages: '@jridgewell/resolve-uri': 3.1.2 '@jridgewell/sourcemap-codec': 1.4.15 + /@mswjs/cookies@0.2.2: + resolution: {integrity: sha512-mlN83YSrcFgk7Dm1Mys40DLssI1KdJji2CMKN8eOlBqsTADYzj2+jWzsANsUTFbxDMWPD5e9bfA1RGqBpS3O1g==} + engines: {node: '>=14'} + dependencies: + '@types/set-cookie-parser': 2.4.10 + set-cookie-parser: 2.6.0 + dev: true + + /@mswjs/interceptors@0.17.10: + resolution: {integrity: sha512-N8x7eSLGcmUFNWZRxT1vsHvypzIRgQYdG0rJey/rZCy6zT/30qDt8Joj7FxzGNLSwXbeZqJOMqDurp7ra4hgbw==} + engines: {node: '>=14'} + dependencies: + '@open-draft/until': 1.0.3 + '@types/debug': 4.1.12 + '@xmldom/xmldom': 0.8.10 + debug: 4.3.5 + headers-polyfill: 3.2.5 + outvariant: 1.4.3 + strict-event-emitter: 0.2.8 + web-encoding: 1.1.5 + transitivePeerDependencies: + - supports-color + dev: true + /@nodelib/fs.scandir@2.1.5: resolution: {integrity: sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g==} engines: {node: '>= 8'} @@ -996,6 +1036,10 @@ packages: fastq: 1.17.1 dev: true + /@open-draft/until@1.0.3: + resolution: {integrity: sha512-Aq58f5HiWdyDlFffbbSjAlv596h/cOnt2DO1w3DOC7OJ5EHs0hd/nycJfiu9RJbT6Yk6F1knnRRXNSpxoIVZ9Q==} + dev: true + /@pkgjs/parseargs@0.11.0: resolution: {integrity: sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg==} engines: {node: '>=14'} @@ -1287,6 +1331,52 @@ packages: defer-to-connect: 1.1.3 dev: true + /@testing-library/dom@8.20.1: + resolution: {integrity: sha512-/DiOQ5xBxgdYRC8LNk7U+RWat0S3qRLeIw3ZIkMQ9kkVlRmwD/Eg8k8CqIpD6GW7u20JIUOfMKbxtiLutpjQ4g==} + engines: {node: '>=12'} + dependencies: + '@babel/code-frame': 7.24.7 + '@babel/runtime': 7.24.7 + '@types/aria-query': 5.0.4 + aria-query: 5.1.3 + chalk: 4.1.2 + dom-accessibility-api: 0.5.16 + lz-string: 1.5.0 + pretty-format: 27.5.1 + dev: true + + /@testing-library/jest-dom@6.4.8: + resolution: {integrity: sha512-JD0G+Zc38f5MBHA4NgxQMR5XtO5Jx9g86jqturNTt2WUfRmLDIY7iKkWHDCCTiDuFMre6nxAD5wHw9W5kI4rGw==} + engines: {node: '>=14', npm: '>=6', yarn: '>=1'} + dependencies: + '@adobe/css-tools': 4.4.0 + '@babel/runtime': 7.24.7 + aria-query: 5.3.0 + chalk: 3.0.0 + css.escape: 1.5.1 + dom-accessibility-api: 0.6.3 + lodash: 4.17.21 + redent: 3.0.0 + dev: true + + /@testing-library/react@12.1.5(react-dom@16.14.0)(react@16.14.0): + resolution: {integrity: sha512-OfTXCJUFgjd/digLUuPxa0+/3ZxsQmE7ub9kcbW/wi96Bh3o/p5vrETcBGfP17NWPGqeYYl5LTRpwyGoMC4ysg==} + engines: {node: '>=12'} + peerDependencies: + react: <18.0.0 + react-dom: <18.0.0 + dependencies: + '@babel/runtime': 7.24.7 + '@testing-library/dom': 8.20.1 + '@types/react-dom': 16.8.4 + react: 16.14.0 + react-dom: 16.14.0(react@16.14.0) + dev: true + + /@types/aria-query@5.0.4: + resolution: {integrity: sha512-rfT93uj5s0PRL7EzccGMs3brplhcrghnDoV26NqKhCAS1hVo+WdNsPvE/yb6ilfr5hi2MEk6d5EWJTKdxg8jVw==} + dev: true + /@types/babel__core@7.20.5: resolution: {integrity: sha512-qoQprZvz5wQFJwMDqeseRXWv3rqMvhgpbXFfVyWhbx9X47POIA6i/+dXefEmZKoAgOaTdaIgNSMqMIU61yRyzA==} dependencies: @@ -1316,6 +1406,16 @@ packages: '@babel/types': 7.24.7 dev: true + /@types/cookie@0.4.1: + resolution: {integrity: sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q==} + dev: true + + /@types/debug@4.1.12: + resolution: {integrity: sha512-vIChWdVG3LG1SMxEvI/AK+FWJthlrqlTu7fbrlywTkkaONwk/UAGaULXRlf8vkzFBLVm0zkMdCquhL5aOjhXPQ==} + dependencies: + '@types/ms': 0.7.34 + dev: true + /@types/estree@1.0.5: resolution: {integrity: sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==} dev: true @@ -1331,6 +1431,10 @@ packages: history: 5.3.0 dev: true + /@types/js-levenshtein@1.1.3: + resolution: {integrity: sha512-jd+Q+sD20Qfu9e2aEXogiO3vpOC1PYJOUdyN9gvs4Qrvkg4wF43L5OhqrPeokdv8TL0/mXoYfpkcoGZMNN2pkQ==} + dev: true + /@types/json-schema@7.0.15: resolution: {integrity: sha512-5+fP8P8MFNC+AyZCDxrB2pkZFPGzqQWUzpSeuuVLvm8VMcorNYavBqoFcxK8bQz4Qsbn4oUEEem4wDLfcysGHA==} dev: true @@ -1341,6 +1445,10 @@ packages: '@types/node': 20.14.8 dev: true + /@types/ms@0.7.34: + resolution: {integrity: sha512-nG96G3Wp6acyAgJqGasjODb+acrI7KltPiRxzHPXnP3NgI28bpQDRv53olbqGXbfcgF5aiiHmO3xpwEpS5Ld9g==} + dev: true + /@types/node@20.14.8: resolution: {integrity: sha512-DO+2/jZinXfROG7j7WKFn/3C6nFwxy2lLpgLjEXJz+0XKphZlTLJ14mo8Vfg8X5BWN6XjyESXq+LcYdT7tR3bA==} dependencies: @@ -1407,6 +1515,12 @@ packages: resolution: {integrity: sha512-I8EUhyrgfLrcTkzV3TSsGyl1tSuPrEDzr0yd5m90UgNxQkyDXULk3b6MlQqTCpZpNtWe1K0hzclnZkTcLBe2UQ==} dev: true + /@types/set-cookie-parser@2.4.10: + resolution: {integrity: sha512-GGmQVGpQWUe5qglJozEjZV/5dyxbOOZ0LHe/lqyWssB88Y4svNfst0uqBVscdDeIKl5Jy5+aPSvy7mI9tYRguw==} + dependencies: + '@types/node': 20.14.8 + dev: true + /@typescript-eslint/eslint-plugin@5.62.0(@typescript-eslint/parser@5.62.0)(eslint@7.32.0)(typescript@4.9.5): resolution: {integrity: sha512-TiZzBSJja/LbhNPvk6yc0JrX9XqhQ0hdh6M2svYfsHGejaKFIAGd9MQ+ERIMzLGlN/kZoYIgdxFV0PuljTKXag==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} @@ -1592,6 +1706,17 @@ packages: pretty-format: 29.7.0 dev: true + /@xmldom/xmldom@0.8.10: + resolution: {integrity: sha512-2WALfTl4xo2SkGCYRt6rDTFfk9R1czmBvUQy12gK2KuRKIpWEhcbbzy8EZXtz/jkRqHX8bFEc6FC1HjX4TUWYw==} + engines: {node: '>=10.0.0'} + dev: true + + /@zxing/text-encoding@0.9.0: + resolution: {integrity: sha512-U/4aVJ2mxI0aDNI8Uq0wEhMgY+u4CNtEb0om3+y3+niDAsoTCOB33UF0sxpzqzdqXLqmvc+vZyAt4O8pPdfkwA==} + requiresBuild: true + dev: true + optional: true + /accepts@1.3.8: resolution: {integrity: sha512-PYAthTa2m2VKxuvSD3DPC/Gy+U+sOA1LAuT8mkmRuvw+NACSaeXEQ+NHcVF7rONl6qcaxV3Uuemwawk+7+SJLw==} engines: {node: '>= 0.6'} @@ -1644,6 +1769,15 @@ packages: react-dom: 16.14.0(react@16.14.0) dev: false + /agent-base@7.1.1: + resolution: {integrity: sha512-H0TSyFNDMomMNJQBn8wFV5YC/2eJ+VXECwOadZJT554xP6cODZHPX3H9QMQECxvrgiSOP1pHjy1sMWQVYJOUOA==} + engines: {node: '>= 14'} + dependencies: + debug: 4.3.5 + transitivePeerDependencies: + - supports-color + dev: true + /ajv@6.12.6: resolution: {integrity: sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==} dependencies: @@ -1673,6 +1807,13 @@ packages: engines: {node: '>=6'} dev: true + /ansi-escapes@4.3.2: + resolution: {integrity: sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ==} + engines: {node: '>=8'} + dependencies: + type-fest: 0.21.3 + dev: true + /ansi-regex@3.0.1: resolution: {integrity: sha512-+O9Jct8wf++lXxxFc4hc8LsjaSq0HFzzL7cVsw8pRDIPdjKD2mT4ytDZlLuSBZ4cLKZFXIrMGO7DbQCtMJJMKw==} engines: {node: '>=4'} @@ -1771,6 +1912,14 @@ packages: resolution: {integrity: sha512-7UvmKalWRt1wgjL1RrGxoSJW/0QZFIegpeGvZG9kjp8vrRu55XTHbwnqq2GpXm9uLbcuhxm3IqX9OB4MZR1b2A==} dev: true + /anymatch@3.1.3: + resolution: {integrity: sha512-KMReFUr0B4t+D+OBkjR3KYqvocp2XaSzO55UcB6mgQMd3KbcE+mWTyvVV7D/zsdEbNnV6acZUutkiHQXvTr1Rw==} + engines: {node: '>= 8'} + dependencies: + normalize-path: 3.0.0 + picomatch: 2.3.1 + dev: true + /argparse@1.0.10: resolution: {integrity: sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==} dependencies: @@ -1781,6 +1930,18 @@ packages: resolution: {integrity: sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==} dev: true + /aria-query@5.1.3: + resolution: {integrity: sha512-R5iJ5lkuHybztUfuOAznmboyjWq8O6sqNqtK7CLOqdydi54VNbORp49mb14KbWgG1QD3JFO9hJdZ+y4KutfdOQ==} + dependencies: + deep-equal: 2.2.3 + dev: true + + /aria-query@5.3.0: + resolution: {integrity: sha512-b0P0sZPKtyu8HkeRAfCq0IfURZK+SuwMjY1UXGBU27wpAiTwQAIlq56IbIO+ytk/JjS1fMR14ee5WBBfKi5J6A==} + dependencies: + dequal: 2.0.3 + dev: true + /array-buffer-byte-length@1.0.1: resolution: {integrity: sha512-ahC5W1xgou+KTXix4sAO8Ki12Q+jf4i0+tmk3sC+zgcynshkHxzpXdImBehiUYKKKDwvfFiJl1tZt6ewscS1Mg==} engines: {node: '>= 0.4'} @@ -1901,6 +2062,10 @@ packages: resolution: {integrity: sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==} dev: true + /base64-js@1.5.1: + resolution: {integrity: sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==} + dev: true + /basic-auth@2.0.1: resolution: {integrity: sha512-NF+epuEdnUYVlGuhaxbbq+dvJttwLnGY+YixlXlME5KpQ5W3CnXA5cVTneY3SPbPDRkcjMbifrwmFYcClgOZeg==} engines: {node: '>= 0.8'} @@ -1914,6 +2079,19 @@ packages: tweetnacl: 0.14.5 dev: true + /binary-extensions@2.3.0: + resolution: {integrity: sha512-Ceh+7ox5qe7LJuLHoY0feh3pHuUDHAcRUeyL2VYghZwfpkNIy/+8Ocg0a3UuSoYzavmylwuLWQOf3hl0jjMMIw==} + engines: {node: '>=8'} + dev: true + + /bl@4.1.0: + resolution: {integrity: sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w==} + dependencies: + buffer: 5.7.1 + inherits: 2.0.4 + readable-stream: 3.6.2 + dev: true + /body-parser@1.20.2: resolution: {integrity: sha512-ml9pReCu3M61kGlqoTm2umSXTlRTuGTx0bfYj+uIUKKYycG5NtSbeetV3faSU6R7ajOPw0g/J1PvK4qNy7s5bA==} engines: {node: '>= 0.8', npm: 1.2.8000 || >= 1.4.16} @@ -1979,6 +2157,13 @@ packages: update-browserslist-db: 1.0.16(browserslist@4.23.1) dev: true + /buffer@5.7.1: + resolution: {integrity: sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ==} + dependencies: + base64-js: 1.5.1 + ieee754: 1.2.1 + dev: true + /bytes@3.0.0: resolution: {integrity: sha512-pMhOfFDPiv9t5jjIXkHosWmkSyQbvsgEVNkz0ERHbuLh2T/7j4Mqqpz523Fe8MVY89KC6Sh/QfS2sM+SjgFDcw==} engines: {node: '>= 0.8'} @@ -2061,6 +2246,14 @@ packages: escape-string-regexp: 1.0.5 supports-color: 5.5.0 + /chalk@3.0.0: + resolution: {integrity: sha512-4D3B6Wf41KOYRFdszmDqMCGq5VV/uMAB273JILmO+3jAlh8X4qDtdtgCR3fxtbLEMzSx22QdhnDcJvu2u1fVwg==} + engines: {node: '>=8'} + dependencies: + ansi-styles: 4.3.0 + supports-color: 7.2.0 + dev: true + /chalk@4.1.2: resolution: {integrity: sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==} engines: {node: '>=10'} @@ -2069,12 +2262,31 @@ packages: supports-color: 7.2.0 dev: true + /chardet@0.7.0: + resolution: {integrity: sha512-mT8iDcrh03qDGRRmoA2hmBJnxpllMR+0/0qlzjqZES6NdiWDcZkCNAk4rPFZ9Q85r27unkiNNg8ZOiwZXBHwcA==} + dev: true + /check-error@1.0.3: resolution: {integrity: sha512-iKEoDYaRmd1mxM90a2OEfWhjsjPpYPuQ+lMYsoxB126+t8fw7ySEO48nmDg5COTjxDI65/Y2OWpeEHk3ZOe8zg==} dependencies: get-func-name: 2.0.2 dev: true + /chokidar@3.6.0: + resolution: {integrity: sha512-7VT13fmjotKpGipCW9JEQAusEPE+Ei8nl6/g4FBAmIm0GOOLMua9NDDo/DWp0ZAxCr3cPq5ZpBqmPAQgDda2Pw==} + engines: {node: '>= 8.10.0'} + dependencies: + anymatch: 3.1.3 + braces: 3.0.3 + glob-parent: 5.1.2 + is-binary-path: 2.1.0 + is-glob: 4.0.3 + normalize-path: 3.0.0 + readdirp: 3.6.0 + optionalDependencies: + fsevents: 2.3.3 + dev: true + /ci-info@2.0.0: resolution: {integrity: sha512-5tK7EtrZ0N+OLFMthtqOj4fI2Jeb88C4CAZPu25LDVUgXJ0A3Js4PMGqrn0JU1W0Mh1/Z8wZzYPxqUrXeBboCQ==} dev: true @@ -2088,6 +2300,23 @@ packages: engines: {node: '>=6'} dev: true + /cli-cursor@3.1.0: + resolution: {integrity: sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw==} + engines: {node: '>=8'} + dependencies: + restore-cursor: 3.1.0 + dev: true + + /cli-spinners@2.9.2: + resolution: {integrity: sha512-ywqV+5MmyL4E7ybXgKys4DugZbX0FC6LnwrhjuykIjnK9k8OQacQ7axGKnjDXWNhns0xot3bZI5h55H8yo9cJg==} + engines: {node: '>=6'} + dev: true + + /cli-width@3.0.0: + resolution: {integrity: sha512-FxqpkPPwu1HjuN93Omfm4h8uIanXofW0RxVEW3k5RKx+mJJYSthzNhp32Kzxxy3YAEZ/Dc/EWN1vZRY0+kOhbw==} + engines: {node: '>= 10'} + dev: true + /cliui@5.0.0: resolution: {integrity: sha512-PYeGSEmmHM6zvoef2w8TPzlrnNpXIjTipYK780YswmIP9vjxmd6Y2a3CB2Ks6/AU8NHjZugXvo8w3oWM2qnwXA==} dependencies: @@ -2096,12 +2325,26 @@ packages: wrap-ansi: 5.1.0 dev: true + /cliui@8.0.1: + resolution: {integrity: sha512-BSeNnyus75C4//NQ9gQt1/csTXyo/8Sb+afLAkzAptFuMsod9HFokGNudZpi/oQV73hnVK+sR+5PVRMd+Dr7YQ==} + engines: {node: '>=12'} + dependencies: + string-width: 4.2.3 + strip-ansi: 6.0.1 + wrap-ansi: 7.0.0 + dev: true + /clone-response@1.0.3: resolution: {integrity: sha512-ROoL94jJH2dUVML2Y/5PEDNaSHgeOdSDicUyS7izcF63G6sTc/FTjLub4b8Il9S8S0beOfYt0TaA5qvFK+w0wA==} dependencies: mimic-response: 1.0.1 dev: true + /clone@1.0.4: + resolution: {integrity: sha512-JQHZ2QMW6l3aH/j6xCqQThY/9OH4D/9ls34cgkUBiEeocRTU04tHfKPBsUK1PqZCUQM7GiA0IIXJSuXHI64Kbg==} + engines: {node: '>=0.8'} + dev: true + /color-convert@1.9.3: resolution: {integrity: sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg==} dependencies: @@ -2206,6 +2449,11 @@ packages: resolution: {integrity: sha512-QADzlaHc8icV8I7vbaJXJwod9HWYp8uCqf1xa4OfNu1T7JVxQIrUgOWtHdNDtPiywmFbiS12VjotIXLrKM3orQ==} dev: true + /cookie@0.4.2: + resolution: {integrity: sha512-aSWTXFzaKWkvHO1Ny/s+ePFpvKsPnjc551iI41v3ny/ow6tBG5Vd+FuqGNhh1LxOmVzOlGUriIlOaokOvhaStA==} + engines: {node: '>= 0.6'} + dev: true + /cookie@0.6.0: resolution: {integrity: sha512-U71cyTamuh1CRNCfpGY6to28lxvNwPG4Guz/EVjgf3Jmzv0vlDp1atT9eS5dDjMYHucpHbWns6Lwf3BKz6svdw==} engines: {node: '>= 0.6'} @@ -2294,6 +2542,17 @@ packages: engines: {node: '>=4'} dev: true + /css.escape@1.5.1: + resolution: {integrity: sha512-YUifsXXuknHlUsmlgyY0PKzgPOr7/FjCePfHNt0jxm83wHZi44VDMQ7/fGNkjY3/jV1MC+1CmZbaHzugyeRtpg==} + dev: true + + /cssstyle@4.0.1: + resolution: {integrity: sha512-8ZYiJ3A/3OkDd093CBT/0UKDWry7ak4BdPTFP2+QEP7cmhouyq/Up709ASSj2cK02BbZiMgk7kYjZNS4QP5qrQ==} + engines: {node: '>=18'} + dependencies: + rrweb-cssom: 0.6.0 + dev: true + /csstype@2.6.21: resolution: {integrity: sha512-Z1PhmomIfypOpoMjRQB70jfvy/wxT50qW08YXO5lMIJkrdq4yOTR+AW7FqutScmB9NkLwxo+jU+kZLbofZZq/w==} @@ -2308,6 +2567,14 @@ packages: assert-plus: 1.0.0 dev: true + /data-urls@5.0.0: + resolution: {integrity: sha512-ZYP5VBHshaDAiVZxjbRVcFJpc+4xGgT0bK3vzy1HLN8jTO975HEbuYzZJcHoQEY5K1a0z8YayJkyVETa08eNTg==} + engines: {node: '>=18'} + dependencies: + whatwg-mimetype: 4.0.0 + whatwg-url: 14.0.0 + dev: true + /data-view-buffer@1.0.1: resolution: {integrity: sha512-0lht7OugA5x3iJLOWFhWK/5ehONdprk0ISXqVFn/NFrDu+cuc8iADFrGQz5BnRK7LLU3JmkbXSxaqX+/mXYtUA==} engines: {node: '>= 0.4'} @@ -2384,6 +2651,10 @@ packages: engines: {node: '>=0.10.0'} dev: true + /decimal.js@10.4.3: + resolution: {integrity: sha512-VBBaLc1MgL5XpzgIP7ny5Z6Nx3UrRkIViUkPUdtl9aya5amy3De1gsUUSB1g3+3sExYNjCAsAznmukyxCb1GRA==} + dev: true + /decompress-response@3.3.0: resolution: {integrity: sha512-BzRPQuY1ip+qDonAOz42gRm/pg9F768C+npV/4JOsxRC2sq+Rlk+Q4ZCAsOhnIaMrgarILY+RMUIvMmmX1qAEA==} engines: {node: '>=4'} @@ -2398,6 +2669,30 @@ packages: type-detect: 4.0.8 dev: true + /deep-equal@2.2.3: + resolution: {integrity: sha512-ZIwpnevOurS8bpT4192sqAowWM76JDKSHYzMLty3BZGSswgq6pBaH3DhCSW5xVAZICZyKdOBPjwww5wfgT/6PA==} + engines: {node: '>= 0.4'} + dependencies: + array-buffer-byte-length: 1.0.1 + call-bind: 1.0.7 + es-get-iterator: 1.1.3 + get-intrinsic: 1.2.4 + is-arguments: 1.1.1 + is-array-buffer: 3.0.4 + is-date-object: 1.0.5 + is-regex: 1.1.4 + is-shared-array-buffer: 1.0.3 + isarray: 2.0.5 + object-is: 1.1.6 + object-keys: 1.1.1 + object.assign: 4.1.5 + regexp.prototype.flags: 1.5.2 + side-channel: 1.0.6 + which-boxed-primitive: 1.0.2 + which-collection: 1.0.2 + which-typed-array: 1.1.15 + dev: true + /deep-extend@0.6.0: resolution: {integrity: sha512-LOHxIOaPYdHlJRtCQfDIVZtfw/ufM8+rVj649RIHzcm/vGwQRXFt6OPqIFWsm2XEMrNIEtWR64sY1LEKD2vAOA==} engines: {node: '>=4.0.0'} @@ -2407,6 +2702,12 @@ packages: resolution: {integrity: sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ==} dev: true + /defaults@1.0.4: + resolution: {integrity: sha512-eFuaLoy/Rxalv2kr+lqMlUnrDWV+3j4pljOIJgLIhI058IQfWJ7vXhyEIHu+HtC738klGALYxOKDO0bQP3tg8A==} + dependencies: + clone: 1.0.4 + dev: true + /defer-to-connect@1.1.3: resolution: {integrity: sha512-0ISdNousHvZT2EiFlZeZAHBUvSxmKswVCEf8hW7KWgG4a8MVEu/3Vb6uWYozkjylyCxe0JBIiRB1jV45S70WVQ==} dev: true @@ -2438,6 +2739,11 @@ packages: engines: {node: '>= 0.8'} dev: true + /dequal@2.0.3: + resolution: {integrity: sha512-0je+qPKHEMohvfRTCEo3CrPG6cAzAYgmzKyxRiYSSDkS6eGJdyVJm7WaYA5ECaAD9wLB2T4EEeymA5aFVcYXCA==} + engines: {node: '>=6'} + dev: true + /destroy@1.2.0: resolution: {integrity: sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg==} engines: {node: '>= 0.8', npm: 1.2.8000 || >= 1.4.16} @@ -2462,6 +2768,14 @@ packages: esutils: 2.0.3 dev: true + /dom-accessibility-api@0.5.16: + resolution: {integrity: sha512-X7BJ2yElsnOJ30pZF4uIIDfBEVgF4XEBxL9Bxhy6dnrm5hkzqmsWHGTiHqRiITNhMyFLyAiWndIJP7Z1NTteDg==} + dev: true + + /dom-accessibility-api@0.6.3: + resolution: {integrity: sha512-7ZgogeTnjuHbo+ct10G9Ffp0mif17idi0IyWNVA/wcwcm7NPOD/WEHVP3n7n3MhXqxoIYm8d6MuZohYWIZ4T3w==} + dev: true + /dom-align@1.12.4: resolution: {integrity: sha512-R8LUSEay/68zE5c8/3BDxiTEvgb4xZTF0RKmAHfiEVN3klfIpXfi2/QCoiWPccVQ0J/ZGdz9OjzL4uJEP/MRAw==} dev: false @@ -2638,6 +2952,20 @@ packages: engines: {node: '>= 0.4'} dev: true + /es-get-iterator@1.1.3: + resolution: {integrity: sha512-sPZmqHBe6JIiTfN5q2pEi//TwxmAFHwj/XEuYjTuse78i8KxaqMTTzxPoFKuzRpDpTJ+0NAbpfenkmH2rePtuw==} + dependencies: + call-bind: 1.0.7 + get-intrinsic: 1.2.4 + has-symbols: 1.0.3 + is-arguments: 1.1.1 + is-map: 2.0.3 + is-set: 2.0.3 + is-string: 1.0.7 + isarray: 2.0.5 + stop-iteration-iterator: 1.0.0 + dev: true + /es-object-atoms@1.0.0: resolution: {integrity: sha512-MZ4iQ6JwHOBQjahnjwaC1ZtIBH+2ohjamzAO3oaHcXYup7qxjF2fixyH+Q71voWHeOkI2q/TnJao/KfXYIZWbw==} engines: {node: '>= 0.4'} @@ -2906,6 +3234,11 @@ packages: engines: {node: '>= 0.6'} dev: true + /events@3.3.0: + resolution: {integrity: sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q==} + engines: {node: '>=0.8.x'} + dev: true + /execa@0.7.0: resolution: {integrity: sha512-RztN09XglpYI7aBBrJCPW95jEH7YF1UEPOoX9yDhUTPdp7mK+CQvnLTuD10BNXZ3byLTu2uehZ8EcKT/4CGiFw==} engines: {node: '>=4'} @@ -2986,6 +3319,15 @@ packages: resolution: {integrity: sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==} dev: true + /external-editor@3.1.0: + resolution: {integrity: sha512-hMQ4CX1p1izmuLYyZqLMO/qGNw10wSv9QDCPfzXfyFrOaCSSoRfqE1Kf1s5an66J5JZC62NewG+mK49jOCtQew==} + engines: {node: '>=4'} + dependencies: + chardet: 0.7.0 + iconv-lite: 0.4.24 + tmp: 0.0.33 + dev: true + /extsprintf@1.3.0: resolution: {integrity: sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==} engines: {'0': node >=0.6.0} @@ -3024,6 +3366,13 @@ packages: reusify: 1.0.4 dev: true + /figures@3.2.0: + resolution: {integrity: sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg==} + engines: {node: '>=8'} + dependencies: + escape-string-regexp: 1.0.5 + dev: true + /file-entry-cache@6.0.1: resolution: {integrity: sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg==} engines: {node: ^10.12.0 || >=12.0.0} @@ -3126,7 +3475,6 @@ packages: asynckit: 0.4.0 combined-stream: 1.0.8 mime-types: 2.1.35 - dev: false /forwarded@0.2.0: resolution: {integrity: sha512-buRG0fpBtRHSTCOASe6hD258tEubFoRLb4ZNA6NxMVHNw2gOcwHo9wyablzMzOA5z9xA9L1KNjk/Nt6MT9aYow==} @@ -3345,6 +3693,11 @@ packages: resolution: {integrity: sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag==} dev: true + /graphql@16.9.0: + resolution: {integrity: sha512-GGTKBX4SD7Wdb8mqeDLni2oaRGYQWjWHGKPQ24ZMnUtKfcsVoiv4uX8+LJr1K6U5VW2Lu1BwJnj7uiori0YtRw==} + engines: {node: ^12.22.0 || ^14.16.0 || ^16.0.0 || >=17.0.0} + dev: true + /har-schema@2.0.0: resolution: {integrity: sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q==} engines: {node: '>=4'} @@ -3406,6 +3759,10 @@ packages: dependencies: function-bind: 1.1.2 + /headers-polyfill@3.2.5: + resolution: {integrity: sha512-tUCGvt191vNSQgttSyJoibR+VO+I6+iCHIUdhzEMJKE+EAL8BwCN7fUOZlY4ofOelNHsK+gEjxB/B+9N3EWtdA==} + dev: true + /history@4.10.1: resolution: {integrity: sha512-36nwAD620w12kuzPAsyINPWJqlNbij+hpK1k9XRloDtym8mxzGYl2c17LnV6IAGB2Dmg4tEa7G7DlawS0+qjew==} dependencies: @@ -3433,6 +3790,13 @@ packages: resolution: {integrity: sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw==} dev: true + /html-encoding-sniffer@4.0.0: + resolution: {integrity: sha512-Y22oTqIU4uuPgEemfz7NDJz6OeKf12Lsu+QC+s3BVpda64lTiMYCyGwg5ki4vFxkMwQdeZDl2adZoqUgdFuTgQ==} + engines: {node: '>=18'} + dependencies: + whatwg-encoding: 3.1.1 + dev: true + /http-cache-semantics@4.1.1: resolution: {integrity: sha512-er295DKPVsV82j5kw1Gjt+ADA/XYHsajl82cGNQG2eyoPkvgUhX+nDIyelzhIWbbsXP39EHcI6l5tYs2FYqYXQ==} dev: true @@ -3448,6 +3812,16 @@ packages: toidentifier: 1.0.1 dev: true + /http-proxy-agent@7.0.2: + resolution: {integrity: sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig==} + engines: {node: '>= 14'} + dependencies: + agent-base: 7.1.1 + debug: 4.3.5 + transitivePeerDependencies: + - supports-color + dev: true + /http-signature@1.2.0: resolution: {integrity: sha512-CAbnr6Rz4CYQkLYUtSNXxQPUH2gK8f3iWexVlsnMeD+GjlsQ0Xsy1cOX+mN3dtxYomRy21CiOzU8Uhw6OwncEQ==} engines: {node: '>=0.8', npm: '>=1.3.7'} @@ -3457,6 +3831,16 @@ packages: sshpk: 1.18.0 dev: true + /https-proxy-agent@7.0.5: + resolution: {integrity: sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==} + engines: {node: '>= 14'} + dependencies: + agent-base: 7.1.1 + debug: 4.3.5 + transitivePeerDependencies: + - supports-color + dev: true + /human-signals@5.0.0: resolution: {integrity: sha512-AXcZb6vzzrFAUE61HnN4mpLqd/cSIwNQjtNWR0euPm6y0iqx3G4gOXaIDdtdDwZmhwe82LA6+zinmW4UBWVePQ==} engines: {node: '>=16.17.0'} @@ -3469,6 +3853,17 @@ packages: safer-buffer: 2.1.2 dev: true + /iconv-lite@0.6.3: + resolution: {integrity: sha512-4fCk79wshMdzMp2rH06qWrJE4iolqLhCUH+OiuIgU++RB0+94NlDL81atO7GX55uUKueo0txHNtvEyI6D7WdMw==} + engines: {node: '>=0.10.0'} + dependencies: + safer-buffer: 2.1.2 + dev: true + + /ieee754@1.2.1: + resolution: {integrity: sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA==} + dev: true + /ignore@4.0.6: resolution: {integrity: sha512-cyFDKrqc/YdcWFniJhzI42+AzS+gNwmUzOSFcRCQYwySuBBBy/KjuxWLZ/FHEH6Moq1NizMOBWyTcv8O4OZIMg==} engines: {node: '>= 4'} @@ -3503,6 +3898,11 @@ packages: engines: {node: '>=0.8.19'} dev: true + /indent-string@4.0.0: + resolution: {integrity: sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg==} + engines: {node: '>=8'} + dev: true + /inflight@1.0.6: resolution: {integrity: sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==} deprecated: This module is not supported, and leaks memory. Do not use it. Check out lru-cache if you want a good and tested way to coalesce async requests by a key value, which is much more comprehensive and powerful. @@ -3519,6 +3919,27 @@ packages: resolution: {integrity: sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew==} dev: true + /inquirer@8.2.6: + resolution: {integrity: sha512-M1WuAmb7pn9zdFRtQYk26ZBoY043Sse0wVDdk4Bppr+JOXyQYybdtvK+l9wUibhtjdjvtoiNy8tk+EgsYIUqKg==} + engines: {node: '>=12.0.0'} + dependencies: + ansi-escapes: 4.3.2 + chalk: 4.1.2 + cli-cursor: 3.1.0 + cli-width: 3.0.0 + external-editor: 3.1.0 + figures: 3.2.0 + lodash: 4.17.21 + mute-stream: 0.0.8 + ora: 5.4.1 + run-async: 2.4.1 + rxjs: 7.8.1 + string-width: 4.2.3 + strip-ansi: 6.0.1 + through: 2.3.8 + wrap-ansi: 6.2.0 + dev: true + /internal-slot@1.0.7: resolution: {integrity: sha512-NGnrKwXzSms2qUUih/ILZ5JBqNTSa1+ZmP6flaIp6KmSElgE9qdndzS3cqjrDovwFdmwsGsLdeFgB6suw+1e9g==} engines: {node: '>= 0.4'} @@ -3533,6 +3954,14 @@ packages: engines: {node: '>= 0.10'} dev: true + /is-arguments@1.1.1: + resolution: {integrity: sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.7 + has-tostringtag: 1.0.2 + dev: true + /is-array-buffer@3.0.4: resolution: {integrity: sha512-wcjaerHw0ydZwfhiKbXJWLDY8A7yV7KhjQOpb83hGgGfId/aQa4TOvwyzn2PuswW2gPCYEL/nEAiSVpdOj1lXw==} engines: {node: '>= 0.4'} @@ -3550,6 +3979,13 @@ packages: has-bigints: 1.0.2 dev: true + /is-binary-path@2.1.0: + resolution: {integrity: sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==} + engines: {node: '>=8'} + dependencies: + binary-extensions: 2.3.0 + dev: true + /is-boolean-object@1.1.2: resolution: {integrity: sha512-gDYaKHJmnj4aWxyj6YHyXVpdQawtVLHU5cb+eztPGczf6cjuTdwve5ZIEfgXqH4e57An1D1AKf8CZ3kYrQRqYA==} engines: {node: '>= 0.4'} @@ -3605,6 +4041,13 @@ packages: engines: {node: '>=8'} dev: true + /is-generator-function@1.0.10: + resolution: {integrity: sha512-jsEjy9l3yiXEQ+PsXdmBwEPcOxaXWLspKdplFUVI9vq1iZgIekeC0L167qeu86czQaxed3q/Uzuw0swL0irL8A==} + engines: {node: '>= 0.4'} + dependencies: + has-tostringtag: 1.0.2 + dev: true + /is-glob@4.0.3: resolution: {integrity: sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==} engines: {node: '>=0.10.0'} @@ -3620,11 +4063,25 @@ packages: is-path-inside: 1.0.1 dev: true + /is-interactive@1.0.0: + resolution: {integrity: sha512-2HvIEKRoqS62guEC+qBjpvRubdX910WCMuJTZ+I9yvqKU2/12eSL549HMwtabb4oupdj2sMP50k+XJfB/8JE6w==} + engines: {node: '>=8'} + dev: true + + /is-map@2.0.3: + resolution: {integrity: sha512-1Qed0/Hr2m+YqxnM09CjA2d/i6YZNfF6R2oRAOj36eUdS6qIV/huPJNSEpKbupewFs+ZsJlxsjjPbc0/afW6Lw==} + engines: {node: '>= 0.4'} + dev: true + /is-negative-zero@2.0.3: resolution: {integrity: sha512-5KoIu2Ngpyek75jXodFvnafB6DJgr3u8uuK0LEZJjrU19DrMD3EVERaR8sjz8CCGgpZvxPl9SuE1GMVPFHx1mw==} engines: {node: '>= 0.4'} dev: true + /is-node-process@1.2.0: + resolution: {integrity: sha512-Vg4o6/fqPxIjtxgUH5QLJhwZ7gW5diGCVlXpuUfELC62CuxM1iHcRe51f2W1FDy04Ai4KJkagKjx3XaqyfRKXw==} + dev: true + /is-npm@3.0.0: resolution: {integrity: sha512-wsigDr1Kkschp2opC4G3yA6r9EgVA6NjRpWzIi9axXqeIaAATPRJc4uLujXe3Nd9uO8KoDyA4MD6aZSeXTADhA==} engines: {node: '>=8'} @@ -3654,6 +4111,10 @@ packages: path-is-inside: 1.0.2 dev: true + /is-potential-custom-element-name@1.0.1: + resolution: {integrity: sha512-bCYeRA2rVibKZd+s2625gGnGF/t7DSqDs4dP7CrLA1m7jKWz6pps0LpYLJN8Q64HtmPKJ1hrN3nzPNKFEKOUiQ==} + dev: true + /is-promise@2.2.2: resolution: {integrity: sha512-+lP4/6lKUBfQjZ2pdxThZvLUAafmZb8OAxFb8XXtiQmS35INgr85hdOGoEs124ez1FCnZJt6jau/T+alh58QFQ==} dev: true @@ -3666,6 +4127,11 @@ packages: has-tostringtag: 1.0.2 dev: true + /is-set@2.0.3: + resolution: {integrity: sha512-iPAjerrse27/ygGLxw+EBR9agv9Y6uLeYVJMu+QNCoouJ1/1ri0mGrcWpfCqFZuzzx3WjtwxG098X+n4OuRkPg==} + engines: {node: '>= 0.4'} + dev: true + /is-shared-array-buffer@1.0.3: resolution: {integrity: sha512-nA2hv5XIhLR3uVzDDfCIknerhx8XUKnstuOERPNNIinXG7v9u+ohXF67vxm4TPTEPU6lm61ZkwP3c9PCB97rhg==} engines: {node: '>= 0.4'} @@ -3708,12 +4174,30 @@ packages: resolution: {integrity: sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA==} dev: true - /is-weakref@1.0.2: - resolution: {integrity: sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ==} + /is-unicode-supported@0.1.0: + resolution: {integrity: sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==} + engines: {node: '>=10'} + dev: true + + /is-weakmap@2.0.2: + resolution: {integrity: sha512-K5pXYOm9wqY1RgjpL3YTkF39tni1XajUIkawTLUo9EZEVUFga5gSQJF8nNS7ZwJQ02y+1YCNYcMh+HIf1ZqE+w==} + engines: {node: '>= 0.4'} + dev: true + + /is-weakref@1.0.2: + resolution: {integrity: sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ==} dependencies: call-bind: 1.0.7 dev: true + /is-weakset@2.0.3: + resolution: {integrity: sha512-LvIm3/KWzS9oRFHugab7d+M/GcBXuXX5xZkzPmN+NxihdQlZUQ4dWuSV1xR/sq6upL1TJEDrfBgRepHFdBtSNQ==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.7 + get-intrinsic: 1.2.4 + dev: true + /is-what@3.14.1: resolution: {integrity: sha512-sNxgpk9793nzSs7bA6JQJGeIuRBQhAaNGG77kzYQgMkrID+lS6SlK07K5LaptscDlSaIgH+GPFzf+d75FVxozA==} @@ -3749,6 +4233,11 @@ packages: resolution: {integrity: sha512-8wb9Yw966OSxApiCt0K3yNJL8pnNeIv+OEq2YMidz4FKP6nonSRoOXc80iXY4JaN2FC11B9qsNmDsm+ZOfMROA==} dev: true + /js-levenshtein@1.1.6: + resolution: {integrity: sha512-X2BB11YZtrRqY4EnQcLX5Rh373zbK4alC1FW7D7MBhL2gtcC17cTnr6DmfHZeS0s2rTHjUTMMHfG7gO8SSdw+g==} + engines: {node: '>=0.10.0'} + dev: true + /js-tokens@4.0.0: resolution: {integrity: sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==} @@ -3775,6 +4264,42 @@ packages: resolution: {integrity: sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==} dev: true + /jsdom@24.1.1: + resolution: {integrity: sha512-5O1wWV99Jhq4DV7rCLIoZ/UIhyQeDR7wHVyZAHAshbrvZsLs+Xzz7gtwnlJTJDjleiTKh54F4dXrX70vJQTyJQ==} + engines: {node: '>=18'} + peerDependencies: + canvas: ^2.11.2 + peerDependenciesMeta: + canvas: + optional: true + dependencies: + cssstyle: 4.0.1 + data-urls: 5.0.0 + decimal.js: 10.4.3 + form-data: 4.0.0 + html-encoding-sniffer: 4.0.0 + http-proxy-agent: 7.0.2 + https-proxy-agent: 7.0.5 + is-potential-custom-element-name: 1.0.1 + nwsapi: 2.2.12 + parse5: 7.1.2 + rrweb-cssom: 0.7.1 + saxes: 6.0.0 + symbol-tree: 3.2.4 + tough-cookie: 4.1.4 + w3c-xmlserializer: 5.0.0 + webidl-conversions: 7.0.0 + whatwg-encoding: 3.1.1 + whatwg-mimetype: 4.0.0 + whatwg-url: 14.0.0 + ws: 8.18.0 + xml-name-validator: 5.0.0 + transitivePeerDependencies: + - bufferutil + - supports-color + - utf-8-validate + dev: true + /jsesc@2.5.2: resolution: {integrity: sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==} engines: {node: '>=4'} @@ -3962,12 +4487,19 @@ packages: /lodash@4.17.21: resolution: {integrity: sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==} + /log-symbols@4.1.0: + resolution: {integrity: sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg==} + engines: {node: '>=10'} + dependencies: + chalk: 4.1.2 + is-unicode-supported: 0.1.0 + dev: true + /loose-envify@1.4.0: resolution: {integrity: sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q==} hasBin: true dependencies: js-tokens: 4.0.0 - dev: false /loupe@2.3.7: resolution: {integrity: sha512-zSMINGVYkdpYSOBmLi0D1Uo7JU9nVdQKrHxC8eYlV+9YKK9WePqAlL7lSlorG/U2Fw1w0hTBmaa/jrQ3UbPHtA==} @@ -4020,6 +4552,11 @@ packages: yallist: 3.1.1 dev: true + /lz-string@1.5.0: + resolution: {integrity: sha512-h5bgJWpxJNswbU7qCrV0tIKQCaS3blPDrqKWx+QxzuzL1zGUzij9XCWLrSLsJPu5t+eWA/ycetzYAO5IOMcWAQ==} + hasBin: true + dev: true + /magic-string@0.30.10: resolution: {integrity: sha512-iIRwTIf0QKV3UAnYK4PU8uiEc4SRh5jX0mwpIwETPpHdhVM4f53RSwS/vXvN1JhGX+Cs7B8qIq3d6AH49O5fAQ==} dependencies: @@ -4109,6 +4646,11 @@ packages: engines: {node: '>=4'} hasBin: true + /mimic-fn@2.1.0: + resolution: {integrity: sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg==} + engines: {node: '>=6'} + dev: true + /mimic-fn@4.0.0: resolution: {integrity: sha512-vqiC06CuhBTUdZH+RYl8sFrL096vA45Ok5ISO6sE/Mr1jRbGH4Csnhi8f3wKVl7x8mO4Au7Ir9D3Oyv1VYMFJw==} engines: {node: '>=12'} @@ -4119,6 +4661,11 @@ packages: engines: {node: '>=4'} dev: true + /min-indent@1.0.1: + resolution: {integrity: sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg==} + engines: {node: '>=4'} + dev: true + /mini-store@3.0.6(react-dom@16.14.0)(react@16.14.0): resolution: {integrity: sha512-YzffKHbYsMQGUWQRKdsearR79QsMzzJcDDmZKlJBqt5JNkqpyJHYlK6gP61O36X+sLf76sO9G6mhKBe83gIZIQ==} peerDependencies: @@ -4190,6 +4737,46 @@ packages: resolution: {integrity: sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==} dev: true + /msw@1.3.3(typescript@4.9.5): + resolution: {integrity: sha512-CiPyRFiYJCXYyH/vwxT7m+sa4VZHuUH6cGwRBj0kaTjBGpsk4EnL47YzhoA859htVCF2vzqZuOsomIUlFqg9GQ==} + engines: {node: '>=14'} + hasBin: true + requiresBuild: true + peerDependencies: + typescript: '>= 4.4.x' + peerDependenciesMeta: + typescript: + optional: true + dependencies: + '@mswjs/cookies': 0.2.2 + '@mswjs/interceptors': 0.17.10 + '@open-draft/until': 1.0.3 + '@types/cookie': 0.4.1 + '@types/js-levenshtein': 1.1.3 + chalk: 4.1.2 + chokidar: 3.6.0 + cookie: 0.4.2 + graphql: 16.9.0 + headers-polyfill: 3.2.5 + inquirer: 8.2.6 + is-node-process: 1.2.0 + js-levenshtein: 1.1.6 + node-fetch: 2.7.0 + outvariant: 1.4.3 + path-to-regexp: 6.2.2 + strict-event-emitter: 0.4.6 + type-fest: 2.19.0 + typescript: 4.9.5 + yargs: 17.7.2 + transitivePeerDependencies: + - encoding + - supports-color + dev: true + + /mute-stream@0.0.8: + resolution: {integrity: sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA==} + dev: true + /mz@2.7.0: resolution: {integrity: sha512-z81GNO7nnYMEhrGh9LeymoE4+Yr0Wn5McHIZMK5cfQCl+NDX08sCZgUc9/6MHni9IWuFLm1Z3HTCXu2z9fN62Q==} dependencies: @@ -4237,6 +4824,18 @@ packages: tslib: 2.6.3 dev: true + /node-fetch@2.7.0: + resolution: {integrity: sha512-c4FRfUm/dbcWZ7U+1Wq0AwCyFL+3nt2bEw05wfxSz+DWpWsitgmSgYmy2dQdWyKC1694ELPqMs/YzUSNozLt8A==} + engines: {node: 4.x || >=6.0.0} + peerDependencies: + encoding: ^0.1.0 + peerDependenciesMeta: + encoding: + optional: true + dependencies: + whatwg-url: 5.0.0 + dev: true + /node-releases@2.0.14: resolution: {integrity: sha512-y10wOWt8yZpqXmOgRo77WaHEmhYQYGNA6y421PKsKYWEK8aW+cqAphborZDhqfyKrbZEN92CN1X2KbafY2s7Yw==} dev: true @@ -4250,6 +4849,11 @@ packages: validate-npm-package-license: 3.0.4 dev: true + /normalize-path@3.0.0: + resolution: {integrity: sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==} + engines: {node: '>=0.10.0'} + dev: true + /normalize-url@4.5.1: resolution: {integrity: sha512-9UZCFRHQdNrfTpGg8+1INIg93B6zE0aXMVFkw1WFwvO4SlZywU6aLg5Of0Ap/PgcbSw4LNxvMWXMeugwMCX0AA==} engines: {node: '>=8'} @@ -4285,6 +4889,10 @@ packages: path-key: 4.0.0 dev: true + /nwsapi@2.2.12: + resolution: {integrity: sha512-qXDmcVlZV4XRtKFzddidpfVP4oMSGhga+xdMc25mv8kaLUHtgzCDhUxkrN8exkGdTlLNaXj7CV3GtON7zuGZ+w==} + dev: true + /oauth-sign@0.9.0: resolution: {integrity: sha512-fexhUFFPTGV8ybAtSIGbV6gOkSv8UtRbDBnAyLQw4QPKkgNlsH2ByPGtMUqdWkos6YCRmAqViwgZrJc/mRDzZQ==} dev: true @@ -4298,6 +4906,14 @@ packages: engines: {node: '>= 0.4'} dev: true + /object-is@1.1.6: + resolution: {integrity: sha512-F8cZ+KfGlSGi09lJT7/Nd6KJZ9ygtvYC0/UYYLI9nmQKLMnydpB9yvbv9K1uSkEu7FU9vYPmVwLg328tX+ot3Q==} + engines: {node: '>= 0.4'} + dependencies: + call-bind: 1.0.7 + define-properties: 1.2.1 + dev: true + /object-keys@1.1.1: resolution: {integrity: sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==} engines: {node: '>= 0.4'} @@ -4338,6 +4954,13 @@ packages: wrappy: 1.0.2 dev: true + /onetime@5.1.2: + resolution: {integrity: sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg==} + engines: {node: '>=6'} + dependencies: + mimic-fn: 2.1.0 + dev: true + /onetime@6.0.0: resolution: {integrity: sha512-1FlR+gjXK7X+AsAHso35MnyN5KqGwJRi/31ft6x0M194ht7S+rWAvd7PHss9xSKMzE0asv1pyIHaJYq+BbacAQ==} engines: {node: '>=12'} @@ -4357,6 +4980,30 @@ packages: word-wrap: 1.2.5 dev: true + /ora@5.4.1: + resolution: {integrity: sha512-5b6Y85tPxZZ7QytO+BQzysW31HJku27cRIlkbAXaNx+BdcVi+LlRFmVXzeF6a7JCwJpyw5c4b+YSVImQIrBpuQ==} + engines: {node: '>=10'} + dependencies: + bl: 4.1.0 + chalk: 4.1.2 + cli-cursor: 3.1.0 + cli-spinners: 2.9.2 + is-interactive: 1.0.0 + is-unicode-supported: 0.1.0 + log-symbols: 4.1.0 + strip-ansi: 6.0.1 + wcwidth: 1.0.1 + dev: true + + /os-tmpdir@1.0.2: + resolution: {integrity: sha512-D2FR03Vir7FIu45XBY20mTb+/ZSWB00sjU9jdQXt83gDrI4Ztz5Fs7/yy74g2N5SVQY4xY1qDr4rNddwYRVX0g==} + engines: {node: '>=0.10.0'} + dev: true + + /outvariant@1.4.3: + resolution: {integrity: sha512-+Sl2UErvtsoajRDKCE5/dBz4DIvHXQQnAxtQTF04OJxY0+DyZXSo5P5Bb7XYWOh81syohlYL24hbDwxedPUJCA==} + dev: true + /p-cancelable@1.1.0: resolution: {integrity: sha512-s73XxOZ4zpt1edZYZzvhqFa6uvQc1vwUa0K0BdtIZgQMAJj9IbebH+JkgKZc9h+B05PKHLOTl4ajG1BmNrVZlw==} engines: {node: '>=6'} @@ -4435,6 +5082,12 @@ packages: engines: {node: '>=6'} dev: false + /parse5@7.1.2: + resolution: {integrity: sha512-Czj1WaSVpaoj0wbhMzLmWD69anp2WH7FXMB9n1Sy8/ZFF9jolSQVMu1Ij5WIyGmcBmhk7EOndpO4mIpihVqAXw==} + dependencies: + entities: 4.5.0 + dev: true + /parseurl@1.3.3: resolution: {integrity: sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ==} engines: {node: '>= 0.8'} @@ -4489,6 +5142,10 @@ packages: dependencies: isarray: 0.0.1 + /path-to-regexp@6.2.2: + resolution: {integrity: sha512-GQX3SSMokngb36+whdpRXE+3f9V8UzyAorlYvOGx87ufGHehNTn5lCxrKtLyZ4Yl/wEKnNnr98ZzOwwDZV5ogw==} + dev: true + /path-type@3.0.0: resolution: {integrity: sha512-T2ZUsdZFHgA3u4e5PfPbjd7HDDpxPnQb5jN0SrDsjNSuVXHJqtwTnWqG0B1jZrgmJ/7lj1EmVIByWt1gxGkWvg==} engines: {node: '>=4'} @@ -4598,6 +5255,15 @@ packages: hasBin: true dev: true + /pretty-format@27.5.1: + resolution: {integrity: sha512-Qb1gy5OrP5+zDf2Bvnzdl3jsTf1qXVMazbvCoKhtKqVs4/YK4ozX4gKQJJVyNe+cajNPn0KoC0MC3FUmaHWEmQ==} + engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0} + dependencies: + ansi-regex: 5.0.1 + ansi-styles: 5.2.0 + react-is: 17.0.2 + dev: true + /pretty-format@29.7.0: resolution: {integrity: sha512-Pdlw/oPxN+aXdmM9R00JVC9WVFoCLTKJvDVLgmJ+qAffBMxsV85l/Lu7sNx4zSzPyoL2euImuEwHhOXdEgNFZQ==} engines: {node: ^14.15.0 || ^16.10.0 || >=18.0.0} @@ -4625,7 +5291,6 @@ packages: loose-envify: 1.4.0 object-assign: 4.1.1 react-is: 16.13.1 - dev: false /proxy-addr@2.0.7: resolution: {integrity: sha512-llQsMLSUDUPT44jdrU/O37qlnifitDP+ZwrmmZcoSKyLKvtZxpyV0n2/bD/N4tBAAZ/gJEdZU7KMraoK1+XYAg==} @@ -4676,6 +5341,10 @@ packages: engines: {node: '>=0.6'} dev: true + /querystringify@2.2.0: + resolution: {integrity: sha512-FIqgj2EUvTa7R50u0rGsyTftzjYmv/a3hO345bZNrqabNqjtgiDMgmo4mkUjd+nzU5oF3dClKqFIPUKybUyqoQ==} + dev: true + /queue-microtask@1.2.3: resolution: {integrity: sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A==} dev: true @@ -5208,7 +5877,6 @@ packages: prop-types: 15.8.1 react: 16.14.0 scheduler: 0.19.1 - dev: false /react-input-autosize@3.0.0(react@16.14.0): resolution: {integrity: sha512-nL9uS7jEs/zu8sqwFE5MAPx6pPkNAriACQ2rGLlqmKr2sPGtN7TXTyDdQt4lbNXVx7Uzadb40x8qotIuru6Rhg==} @@ -5221,7 +5889,10 @@ packages: /react-is@16.13.1: resolution: {integrity: sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ==} - dev: false + + /react-is@17.0.2: + resolution: {integrity: sha512-w2GsyukL62IJnlaff/nRegPQR94C/XXamvMWmSHRJ4y7Ts/4ocGRmTHvOs8PSE6pB3dWOrD/nueuU5sduBsQ4w==} + dev: true /react-is@18.3.1: resolution: {integrity: sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg==} @@ -5304,7 +5975,6 @@ packages: loose-envify: 1.4.0 object-assign: 4.1.1 prop-types: 15.8.1 - dev: false /read-pkg@3.0.0: resolution: {integrity: sha512-BLq/cCO9two+lBgiTYNqD6GdtK8s4NpaWrl6/rCO9w0TUS8oJl7cmToOZfRYllKTISY6nt1U7jQ53brmKqY6BA==} @@ -5315,6 +5985,22 @@ packages: path-type: 3.0.0 dev: true + /readable-stream@3.6.2: + resolution: {integrity: sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA==} + engines: {node: '>= 6'} + dependencies: + inherits: 2.0.4 + string_decoder: 1.3.0 + util-deprecate: 1.0.2 + dev: true + + /readdirp@3.6.0: + resolution: {integrity: sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==} + engines: {node: '>=8.10.0'} + dependencies: + picomatch: 2.3.1 + dev: true + /recrawl-sync@2.2.3: resolution: {integrity: sha512-vSaTR9t+cpxlskkdUFrsEpnf67kSmPk66yAGT1fZPrDudxQjoMzPgQhSMImQ0pAw5k0NPirefQfhopSjhdUtpQ==} dependencies: @@ -5325,6 +6011,14 @@ packages: tslib: 1.14.1 dev: true + /redent@3.0.0: + resolution: {integrity: sha512-6tDA8g98We0zd0GvVeMT9arEOnTw9qM03L9cJXaCjrip1OO764RDBLBfrB4cwzNGDj5OA5ioymC9GkizgWJDUg==} + engines: {node: '>=8'} + dependencies: + indent-string: 4.0.0 + strip-indent: 3.0.0 + dev: true + /regenerator-runtime@0.14.1: resolution: {integrity: sha512-dYnhHh0nJoMfnkZs6GmmhFknAGRrLznOu5nc9ML+EJxGvrx6H7teuevqVqCuPcPK//3eDrrjQhehXVx9cnkGdw==} @@ -5398,6 +6092,10 @@ packages: resolution: {integrity: sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg==} dev: true + /requires-port@1.0.0: + resolution: {integrity: sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ==} + dev: true + /resize-observer-polyfill@1.5.1: resolution: {integrity: sha512-LwZrotdHOo12nQuZlHEmtuXdqGoOD0OhaxopaNFxWzInpEgaLWoVuAMbTzixuosCx2nEG58ngzW3vxdWoxIgdg==} dev: false @@ -5424,6 +6122,14 @@ packages: lowercase-keys: 1.0.1 dev: true + /restore-cursor@3.1.0: + resolution: {integrity: sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA==} + engines: {node: '>=8'} + dependencies: + onetime: 5.1.2 + signal-exit: 3.0.7 + dev: true + /reusify@1.0.4: resolution: {integrity: sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw==} engines: {iojs: '>=1.0.0', node: '>=0.10.0'} @@ -5471,12 +6177,31 @@ packages: fsevents: 2.3.3 dev: true + /rrweb-cssom@0.6.0: + resolution: {integrity: sha512-APM0Gt1KoXBz0iIkkdB/kfvGOwC4UuJFeG/c+yV7wSc7q96cG/kJ0HiYCnzivD9SB53cLV1MlHFNfOuPaadYSw==} + dev: true + + /rrweb-cssom@0.7.1: + resolution: {integrity: sha512-TrEMa7JGdVm0UThDJSx7ddw5nVm3UJS9o9CCIZ72B1vSyEZoziDqBYP3XIoi/12lKrJR8rE3jeFHMok2F/Mnsg==} + dev: true + + /run-async@2.4.1: + resolution: {integrity: sha512-tvVnVv01b8c1RrA6Ep7JkStj85Guv/YrMcwqYQnwjsAS2cTmmPGBBjAjpCW7RrSodNSoE2/qg9O4bceNvUuDgQ==} + engines: {node: '>=0.12.0'} + dev: true + /run-parallel@1.2.0: resolution: {integrity: sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA==} dependencies: queue-microtask: 1.2.3 dev: true + /rxjs@7.8.1: + resolution: {integrity: sha512-AA3TVj+0A2iuIoQkWEK/tqFjBq2j+6PO6Y0zJcvzLAFhEFIO3HL0vls9hWLncZbAAbK0mar7oZ4V079I/qPMxg==} + dependencies: + tslib: 2.6.3 + dev: true + /safe-array-concat@1.1.2: resolution: {integrity: sha512-vj6RsCsWBCf19jIeHEfkRMw8DPiBb+DMXklQ/1SGDHOMlHdPUkZXFQ2YdplS23zESTijAcurb1aSgJA3AgMu1Q==} engines: {node: '>=0.4'} @@ -5508,12 +6233,18 @@ packages: resolution: {integrity: sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==} dev: true + /saxes@6.0.0: + resolution: {integrity: sha512-xAg7SOnEhrm5zI3puOOKyy1OMcMlIJZYNJY7xLBwSze0UjhPLnWfj2GF2EpT0jmzaJKIWKHLsaSSajf35bcYnA==} + engines: {node: '>=v12.22.7'} + dependencies: + xmlchars: 2.2.0 + dev: true + /scheduler@0.19.1: resolution: {integrity: sha512-n/zwRWRYSUj0/3g/otKDRPMh6qv2SYMWNq85IEa8iZyAv8od9zDYpGSnpBEjNgcMNq6Scbu5KfIPxNF72R/2EA==} dependencies: loose-envify: 1.4.0 object-assign: 4.1.1 - dev: false /scroll-into-view-if-needed@2.2.31: resolution: {integrity: sha512-dGCXy99wZQivjmjIqihaBQNjryrz5rueJY7eHfTdyWEiR4ttYpsajb14rn9s5d4DY4EcY6+4+U/maARBXJedkA==} @@ -5588,6 +6319,10 @@ packages: resolution: {integrity: sha512-KiKBS8AnWGEyLzofFfmvKwpdPzqiy16LvQfK3yv/fVH7Bj13/wl3JSR1J+rfgRE9q7xUJK4qvgS8raSOeLUehw==} dev: true + /set-cookie-parser@2.6.0: + resolution: {integrity: sha512-RVnVQxTXuerk653XfuliOxBP81Sf0+qfQE73LIYKcyMYHG94AuH0kgrQpRDuTZnSmjpysHmzxJXKNfa6PjFhyQ==} + dev: true + /set-function-length@1.2.2: resolution: {integrity: sha512-pgRc4hJ4/sNjWCSS9AmnS40x3bNMDTknHgL5UaMBTMyJnU90EgWh1Rz+MC9eFu4BuN/UwZjKQuY/1v3rM7HMfg==} engines: {node: '>= 0.4'} @@ -5767,6 +6502,23 @@ packages: graceful-fs: 4.2.11 dev: true + /stop-iteration-iterator@1.0.0: + resolution: {integrity: sha512-iCGQj+0l0HOdZ2AEeBADlsRC+vsnDsZsbdSiH1yNSjcfKM7fdpCMfqAL/dwF5BLiw/XhRft/Wax6zQbhq2BcjQ==} + engines: {node: '>= 0.4'} + dependencies: + internal-slot: 1.0.7 + dev: true + + /strict-event-emitter@0.2.8: + resolution: {integrity: sha512-KDf/ujU8Zud3YaLtMCcTI4xkZlZVIYxTLr+XIULexP+77EEVWixeXroLUXQXiVtH4XH2W7jr/3PT1v3zBuvc3A==} + dependencies: + events: 3.3.0 + dev: true + + /strict-event-emitter@0.4.6: + resolution: {integrity: sha512-12KWeb+wixJohmnwNFerbyiBrAlq5qJLwIt38etRtKtmmHyDSoGlIqFE9wx+4IwG0aDjI7GV8tc8ZccjWZZtTg==} + dev: true + /string-convert@0.2.1: resolution: {integrity: sha512-u/1tdPl4yQnPBjnVrmdLo9gtuLvELKsAoRapekWggdiQNvvvum+jYF329d84NAa660KQw7pB2n36KrIKVoXa3A==} dev: false @@ -5843,6 +6595,12 @@ packages: es-object-atoms: 1.0.0 dev: true + /string_decoder@1.3.0: + resolution: {integrity: sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA==} + dependencies: + safe-buffer: 5.2.1 + dev: true + /strip-ansi@4.0.0: resolution: {integrity: sha512-4XaJ2zQdCzROZDivEVIDPkcQn8LMFSa8kj8Gxb/Lnwzv9A8VctNZ+lfivC/sV3ivW8ElJTERXZoPBRrZKkNKow==} engines: {node: '>=4'} @@ -5886,6 +6644,13 @@ packages: engines: {node: '>=12'} dev: true + /strip-indent@3.0.0: + resolution: {integrity: sha512-laJTa3Jb+VQpaC6DseHhF7dXVqHTfJPCRDaEbid/drOhgitgYku/letMUqOXFoWV0zIIUbjpdH2t+tYj4bQMRQ==} + engines: {node: '>=8'} + dependencies: + min-indent: 1.0.1 + dev: true + /strip-json-comments@2.0.1: resolution: {integrity: sha512-4gB8na07fecVVkOI6Rs4e7T6NOTki5EmL7TUduTs6bu3EdnSycntVJ4re8kgZA+wx9IueI2Y11bfbgwtzuE0KQ==} engines: {node: '>=0.10.0'} @@ -5937,6 +6702,10 @@ packages: resolution: {integrity: sha512-e4hG1hRwoOdRb37cIMSgzNsxyzKfayW6VOflrwvR+/bzrkyxY/31WkbgnQpgtrNp1SdpJvpUAGTa/ZoiPNDuRQ==} dev: true + /symbol-tree@3.2.4: + resolution: {integrity: sha512-9QNk5KwDF+Bvz+PyObkmSYjI5ksVUYtjW7AU22r2NKcfLJcXp96hkDWU3+XndOsUb+AQ9QhfzfCT2O+CNWT5Tw==} + dev: true + /table@6.8.2: resolution: {integrity: sha512-w2sfv80nrAh2VCbqR5AK27wswXhqcck2AhfnNW76beQXskGZ1V12GwS//yYVa3d3fcvAip2OUnbDAjW2k3v9fA==} engines: {node: '>=10.0.0'} @@ -5972,6 +6741,10 @@ packages: any-promise: 1.3.0 dev: true + /through@2.3.8: + resolution: {integrity: sha512-w89qg7PI8wAdvX60bMDP+bFoD5Dvhm9oLheFp5O4a2QF0cSBGsBX4qZmadPMvVqlLJBBci+WqGGOAPvcDeNSVg==} + dev: true + /tiny-invariant@1.3.3: resolution: {integrity: sha512-+FbBPE1o9QAYvviau/qC5SE3caw21q3xkvWKBtja5vgqOWIHHJ3ioaq1VPfn/Szqctz2bU/oYeKd9/z5BL+PVg==} dev: false @@ -5994,6 +6767,13 @@ packages: engines: {node: '>=14.0.0'} dev: true + /tmp@0.0.33: + resolution: {integrity: sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==} + engines: {node: '>=0.6.0'} + dependencies: + os-tmpdir: 1.0.2 + dev: true + /to-fast-properties@2.0.0: resolution: {integrity: sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog==} engines: {node: '>=4'} @@ -6027,6 +6807,27 @@ packages: punycode: 2.3.1 dev: true + /tough-cookie@4.1.4: + resolution: {integrity: sha512-Loo5UUvLD9ScZ6jh8beX1T6sO1w2/MpCRpEP7V280GKMVUQ0Jzar2U3UJPsrdbziLEMMhu3Ujnq//rhiFuIeag==} + engines: {node: '>=6'} + dependencies: + psl: 1.9.0 + punycode: 2.3.1 + universalify: 0.2.0 + url-parse: 1.5.10 + dev: true + + /tr46@0.0.3: + resolution: {integrity: sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw==} + dev: true + + /tr46@5.0.0: + resolution: {integrity: sha512-tk2G5R2KRwBd+ZN0zaEXpmzdKyOYksXwywulIX95MBODjSzMIuQnQ3m8JxgbhnL1LeVo7lqQKsYa1O3Htl7K5g==} + engines: {node: '>=18'} + dependencies: + punycode: 2.3.1 + dev: true + /ts-interface-checker@0.1.13: resolution: {integrity: sha512-Y/arvbn+rrz3JCKl9C4kVNfTfSm2/mEp5FSz5EsZSANGPSlQrpRI5M4PKF+mJnE52jOO90PnPSc3Ur3bTQw0gA==} dev: true @@ -6088,11 +6889,21 @@ packages: engines: {node: '>=10'} dev: true + /type-fest@0.21.3: + resolution: {integrity: sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==} + engines: {node: '>=10'} + dev: true + /type-fest@0.3.1: resolution: {integrity: sha512-cUGJnCdr4STbePCgqNFbpVNCepa+kAVohJs1sLhxzdH+gnEoOd8VhbYa7pD3zZYGiURWM2xzEII3fQcRizDkYQ==} engines: {node: '>=6'} dev: true + /type-fest@2.19.0: + resolution: {integrity: sha512-RAH822pAdBgcNMAfWnCBU3CFZcfZ/i1eZjwFU/dsLKumyuuP3niueg2UAukXYF0E2AAoc82ZSSf9J0WQBinzHA==} + engines: {node: '>=12.20'} + dev: true + /type-is@1.6.18: resolution: {integrity: sha512-TkRKr9sUTxEH8MdfuCSP7VizJyzRNMjj2J2do2Jr3Kym598JVdEksuzPQCnlFPW4ky9Q+iA+ma9BGm06XQBy8g==} engines: {node: '>= 0.6'} @@ -6174,6 +6985,11 @@ packages: crypto-random-string: 1.0.0 dev: true + /universalify@0.2.0: + resolution: {integrity: sha512-CJ1QgKmNg3CwvAv/kOFmtnEN05f0D/cn9QntgNOQlQF9dgvVTHj3t+8JPdjqawCHk7V/KA+fbUqzZ9XWhcqPUg==} + engines: {node: '>= 4.0.0'} + dev: true + /unpipe@1.0.0: resolution: {integrity: sha512-pjy2bYhSsufwWlKwPc+l3cN7+wuJlK6uz0YdJEOlQDbl6jo/YlPi4mb8agUkVC8BF7V8NuzeyPNqRksA3hztKQ==} engines: {node: '>= 0.8'} @@ -6221,6 +7037,27 @@ packages: prepend-http: 2.0.0 dev: true + /url-parse@1.5.10: + resolution: {integrity: sha512-WypcfiRhfeUP9vvF0j6rw0J3hrWrw6iZv3+22h6iRMJ/8z1Tj6XfLP4DsUix5MhMPnXpiHDoKyoZ/bdCkwBCiQ==} + dependencies: + querystringify: 2.2.0 + requires-port: 1.0.0 + dev: true + + /util-deprecate@1.0.2: + resolution: {integrity: sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==} + dev: true + + /util@0.12.5: + resolution: {integrity: sha512-kZf/K6hEIrWHI6XqOFUiiMa+79wE/D8Q+NCNAWclkyg3b4d2k7s0QGepNjiABc+aR3N1PAyHL7p6UcLY6LmrnA==} + dependencies: + inherits: 2.0.4 + is-arguments: 1.1.1 + is-generator-function: 1.0.10 + is-typed-array: 1.1.13 + which-typed-array: 1.1.15 + dev: true + /utils-merge@1.0.1: resolution: {integrity: sha512-pMZTvIkT1d+TFGvDOqodOclx0QWkkgi6Tdoa8gC8ffGAAqz9pzPTZWAybbsHHoED/ztMtkv/VoYTYyShUn81hA==} engines: {node: '>= 0.4.0'} @@ -6383,7 +7220,7 @@ packages: fsevents: 2.3.3 dev: true - /vitest@1.6.0(less@3.13.1): + /vitest@1.6.0(jsdom@24.1.1)(less@3.13.1): resolution: {integrity: sha512-H5r/dN06swuFnzNFhq/dnz37bPXnq8xB2xB5JOVk8K09rUtoeNN+LHWkoQ0A/i3hvbUKKcCei9KpbxqHMLhLLA==} engines: {node: ^18.0.0 || >=20.0.0} hasBin: true @@ -6417,6 +7254,7 @@ packages: chai: 4.4.1 debug: 4.3.5 execa: 8.0.1 + jsdom: 24.1.1 local-pkg: 0.5.0 magic-string: 0.30.10 pathe: 1.1.2 @@ -6438,12 +7276,69 @@ packages: - terser dev: true + /w3c-xmlserializer@5.0.0: + resolution: {integrity: sha512-o8qghlI8NZHU1lLPrpi2+Uq7abh4GGPpYANlalzWxyWteJOCsr/P+oPBA49TOLu5FTZO4d3F9MnWJfiMo4BkmA==} + engines: {node: '>=18'} + dependencies: + xml-name-validator: 5.0.0 + dev: true + /warning@4.0.3: resolution: {integrity: sha512-rpJyN222KWIvHJ/F53XSZv0Zl/accqHR8et1kpaMTD/fLCRxtV8iX8czMzY7sVZupTI3zcUTg8eycS2kNF9l6w==} dependencies: loose-envify: 1.4.0 dev: false + /wcwidth@1.0.1: + resolution: {integrity: sha512-XHPEwS0q6TaxcvG85+8EYkbiCux2XtWG2mkc47Ng2A77BQu9+DqIOJldST4HgPkuea7dvKSj5VgX3P1d4rW8Tg==} + dependencies: + defaults: 1.0.4 + dev: true + + /web-encoding@1.1.5: + resolution: {integrity: sha512-HYLeVCdJ0+lBYV2FvNZmv3HJ2Nt0QYXqZojk3d9FJOLkwnuhzM9tmamh8d7HPM8QqjKH8DeHkFTx+CFlWpZZDA==} + dependencies: + util: 0.12.5 + optionalDependencies: + '@zxing/text-encoding': 0.9.0 + dev: true + + /webidl-conversions@3.0.1: + resolution: {integrity: sha512-2JAn3z8AR6rjK8Sm8orRC0h/bcl/DqL7tRPdGZ4I1CjdF+EaMLmYxBHyXuKL849eucPFhvBoxMsflfOb8kxaeQ==} + dev: true + + /webidl-conversions@7.0.0: + resolution: {integrity: sha512-VwddBukDzu71offAQR975unBIGqfKZpM+8ZX6ySk8nYhVoo5CYaZyzt3YBvYtRtO+aoGlqxPg/B87NGVZ/fu6g==} + engines: {node: '>=12'} + dev: true + + /whatwg-encoding@3.1.1: + resolution: {integrity: sha512-6qN4hJdMwfYBtE3YBTTHhoeuUrDBPZmbQaxWAqSALV/MeEnR5z1xd8UKud2RAkFoPkmB+hli1TZSnyi84xz1vQ==} + engines: {node: '>=18'} + dependencies: + iconv-lite: 0.6.3 + dev: true + + /whatwg-mimetype@4.0.0: + resolution: {integrity: sha512-QaKxh0eNIi2mE9p2vEdzfagOKHCcj1pJ56EEHGQOVxp8r9/iszLUUV7v89x9O1p/T+NlTM5W7jW6+cz4Fq1YVg==} + engines: {node: '>=18'} + dev: true + + /whatwg-url@14.0.0: + resolution: {integrity: sha512-1lfMEm2IEr7RIV+f4lUNPOqfFL+pO+Xw3fJSqmjX9AbXcXcYOkCe1P6+9VBZB6n94af16NfZf+sSk0JCBZC9aw==} + engines: {node: '>=18'} + dependencies: + tr46: 5.0.0 + webidl-conversions: 7.0.0 + dev: true + + /whatwg-url@5.0.0: + resolution: {integrity: sha512-saE57nupxk6v3HY35+jzBwYa0rKSy0XR8JSxZPwgLr7ys0IBzhGviA1/TUGJLmSVqs8pb9AnvICXEuOHLprYTw==} + dependencies: + tr46: 0.0.3 + webidl-conversions: 3.0.1 + dev: true + /which-boxed-primitive@1.0.2: resolution: {integrity: sha512-bwZdv0AKLpplFY2KZRX6TvyuN7ojjr7lwkg6ml0roIy9YeuSr7JS372qlNW18UQYzgYK9ziGcerWqZOmEn9VNg==} dependencies: @@ -6454,6 +7349,16 @@ packages: is-symbol: 1.0.4 dev: true + /which-collection@1.0.2: + resolution: {integrity: sha512-K4jVyjnBdgvc86Y6BkaLZEN933SwYOuBFkdmBu9ZfkcAbdVbpITnDmjvZ/aQjRXQrv5EPkTnD1s39GiiqbngCw==} + engines: {node: '>= 0.4'} + dependencies: + is-map: 2.0.3 + is-set: 2.0.3 + is-weakmap: 2.0.2 + is-weakset: 2.0.3 + dev: true + /which-module@2.0.1: resolution: {integrity: sha512-iBdZ57RDvnOR9AGBhML2vFZf7h8vmBjhoaZqODJBFWHVtKkDmKuHai3cx5PgVMrX5YDNp27AofYbAwctSS+vhQ==} dev: true @@ -6514,6 +7419,15 @@ packages: strip-ansi: 5.2.0 dev: true + /wrap-ansi@6.2.0: + resolution: {integrity: sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA==} + engines: {node: '>=8'} + dependencies: + ansi-styles: 4.3.0 + string-width: 4.2.3 + strip-ansi: 6.0.1 + dev: true + /wrap-ansi@7.0.0: resolution: {integrity: sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==} engines: {node: '>=10'} @@ -6544,15 +7458,42 @@ packages: signal-exit: 3.0.7 dev: true + /ws@8.18.0: + resolution: {integrity: sha512-8VbfWfHLbbwu3+N6OKsOMpBdT4kXPDDB9cJk2bJ6mh9ucxdlnNvH1e+roYkKmN9Nxw2yjz7VzeO9oOz2zJ04Pw==} + engines: {node: '>=10.0.0'} + peerDependencies: + bufferutil: ^4.0.1 + utf-8-validate: '>=5.0.2' + peerDependenciesMeta: + bufferutil: + optional: true + utf-8-validate: + optional: true + dev: true + /xdg-basedir@3.0.0: resolution: {integrity: sha512-1Dly4xqlulvPD3fZUQJLY+FUIeqN3N2MM3uqe4rCJftAvOjFa3jFGfctOgluGx4ahPbUCsZkmJILiP0Vi4T6lQ==} engines: {node: '>=4'} dev: true + /xml-name-validator@5.0.0: + resolution: {integrity: sha512-EvGK8EJ3DhaHfbRlETOWAS5pO9MZITeauHKJyb8wyajUfQUenkIg2MvLDTZ4T/TgIcm3HU0TFBgWWboAZ30UHg==} + engines: {node: '>=18'} + dev: true + + /xmlchars@2.2.0: + resolution: {integrity: sha512-JZnDKK8B0RCDw84FNdDAIpZK+JuJw+s7Lz8nksI7SIuU3UXJJslUthsi+uWBUYOwPFwW7W7PRLRfUKpxjtjFCw==} + dev: true + /y18n@4.0.3: resolution: {integrity: sha512-JKhqTOwSrqNA1NY5lSztJ1GrBiUodLMmIZuLiDaMRJ+itFd+ABVE8XBjOvIWL+rSqNDC74LCSFmlb/U4UZ4hJQ==} dev: true + /y18n@5.0.8: + resolution: {integrity: sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA==} + engines: {node: '>=10'} + dev: true + /yallist@2.1.2: resolution: {integrity: sha512-ncTzHV7NvsQZkYe1DW7cbDLm0YpzHmZF5r/iyP3ZnQtMiJ+pjzisCiMNI+Sj+xQF5pXhSHxSB3uDbsBTzY/c2A==} dev: true @@ -6573,6 +7514,11 @@ packages: decamelize: 1.2.0 dev: true + /yargs-parser@21.1.1: + resolution: {integrity: sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw==} + engines: {node: '>=12'} + dev: true + /yargs@14.2.3: resolution: {integrity: sha512-ZbotRWhF+lkjijC/VhmOT9wSgyBQ7+zr13+YLkhfsSiTriYsMzkTUFP18pFhWwBeMa5gUc1MzbhrO6/VB7c9Xg==} dependencies: @@ -6589,6 +7535,19 @@ packages: yargs-parser: 15.0.3 dev: true + /yargs@17.7.2: + resolution: {integrity: sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w==} + engines: {node: '>=12'} + dependencies: + cliui: 8.0.1 + escalade: 3.1.2 + get-caller-file: 2.0.5 + require-directory: 2.1.1 + string-width: 4.2.3 + y18n: 5.0.8 + yargs-parser: 21.1.1 + dev: true + /yocto-queue@1.0.0: resolution: {integrity: sha512-9bnSc/HEW2uRy67wc+T8UwauLuPJVn28jb+GtJY16iiKWyvmYJRXVT4UamsAEGQfPohgr2q4Tq0sQbQlxTfi1g==} engines: {node: '>=12.20'} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/Overview.test.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/Overview.test.tsx new file mode 100644 index 00000000000..8af860e3241 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/Overview.test.tsx @@ -0,0 +1,214 @@ +/* + * 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 React from 'react'; +import { BrowserRouter } from 'react-router-dom'; + +/** + * The dont-cleanup-after-each is imported to prevent autoCleanup of rendered + * component after each test. + * Since we are needing to set a timeout everytime the component is rendered + * and we would be verifying whether the UI is correct, we can skip the cleanup + * and leave it for after all the tests run - saving test time + */ +import '@testing-library/react/dont-cleanup-after-each'; +import { cleanup, render, screen } from '@testing-library/react'; + +import { overviewLocators } from '@tests/locators/locators'; +import { faultyOverviewServer, overviewServer } from '@tests/mocks/overviewMocks/overviewServer'; +import Overview from '@/v2/pages/overview/overview'; + +const WrappedOverviewComponent = () => { + return ( + + + + ) +} + +/** + * We need to mock the EChart component as in the virtual DOM + * it cannot access the DOM height and width and will throw errors + * Hence we intercept and mock the import to return an empty + * React fragment + */ +vi.mock('@/v2/components/eChart/eChart', () => ({ + default: () => (<>) +})) + +describe.each([ + true, + false +])('Overview Tests - Data is present = %s', (scenario) => { + beforeAll(async () => { + (scenario) ? overviewServer.listen() : faultyOverviewServer.listen(); + render( + + ); + //Setting a timeout of 100ms to allow requests to be resolved and states to be set + await new Promise((r) => { setTimeout(r, 100) }) + }); + + afterAll(() => { + (scenario) ? overviewServer.close() : faultyOverviewServer.close(); + vi.clearAllMocks(); + /** + * Need to cleanup the DOM after one suite has completely run + * Otherwise we will get duplicate elements + */ + cleanup(); + }) + + // Tests begin here + // All the data is being mocked by MSW, so we have a fixed data that we can verify + // the content against + it('Datanode row has the correct count of Datanodes', () => { + const datanodeRow = screen.getByTestId(overviewLocators.datanodeRow); + expect(datanodeRow).toBeVisible(); + expect(datanodeRow).toHaveTextContent((scenario) ? '3/5' : 'N/A'); + }); + + it('Containers row has the correct count of containers', () => { + const containerRow = screen.getByTestId(overviewLocators.containersRow); + expect(containerRow).toBeVisible(); + expect(containerRow).toHaveTextContent((scenario) ? '20' : 'N/A'); + }); + + it('Capacity card has the correct capacity data', () => { + const capacityOzoneUsed = screen.getByTestId(overviewLocators.capacityOzoneUsed); + const capacityNonOzoneUsed = screen.getByTestId(overviewLocators.capacityNonOzoneUsed); + const capacityRemaining = screen.getByTestId(overviewLocators.capacityRemaining); + const capacityPreAllocated = screen.getByTestId(overviewLocators.capacityPreAllocated); + + expect(capacityOzoneUsed).toBeVisible(); + expect(capacityNonOzoneUsed).toBeVisible(); + expect(capacityRemaining).toBeVisible(); + expect(capacityPreAllocated).toBeVisible(); + + expect(capacityOzoneUsed).toHaveTextContent( + (scenario) + ? /Ozone Used\s*784.7 MB/ + : /Ozone Used\s*0 B/ + ); + expect(capacityNonOzoneUsed).toHaveTextContent( + (scenario) + ? /Non Ozone Used\s*263.1 GB/ + : /Non Ozone Used\s*0 B/ + ); + expect(capacityRemaining).toHaveTextContent( + (scenario) + ? /Remaining\s*995.4 GB/ + : /Remaining\s*0 B/ + ); + expect(capacityPreAllocated).toHaveTextContent( + (scenario) + ? /Container Pre-allocated\s*11.2 GB/ + : /Container Pre-allocated\s*0 B/ + ); + }); + + it('Volumes card has the correct number of volumes', () => { + const volumeCard = screen.getByTestId(overviewLocators.volumesCard); + expect(volumeCard).toBeVisible(); + expect(volumeCard).toHaveTextContent((scenario) ? '2' : 'N/A'); + }); + + it('Buckets card has the correct number of buckets', () => { + const bucketsCard = screen.getByTestId(overviewLocators.bucketsCard); + expect(bucketsCard).toBeVisible(); + expect(bucketsCard).toHaveTextContent((scenario) ? '24' : 'N/A'); + }); + + it('Keys card has the correct number of keys', () => { + const keysCard = screen.getByTestId(overviewLocators.keysCard); + expect(keysCard).toBeVisible(); + expect(keysCard).toHaveTextContent((scenario) ? '1424' : 'N/A'); + }); + + it('Pipelines card has the correct count of Pipelines', () => { + const pipelinesCard = screen.getByTestId(overviewLocators.pipelinesCard); + expect(pipelinesCard).toBeVisible(); + expect(pipelinesCard).toHaveTextContent((scenario) ? '7' : 'N/A'); + }); + + it('Deleted Containers card has the correct count of deleted containers', () => { + const deletedContainersCard = screen.getByTestId(overviewLocators.deletedContainersCard); + expect(deletedContainersCard).toBeVisible(); + expect(deletedContainersCard).toHaveTextContent((scenario) ? '10' : 'N/A') + }) + + it('Delete Pending Summary has the correct data', () => { + const deletePendingReplicatedData = screen.getByTestId( + overviewLocators.deletePendingTotalReplicatedData + ); + const deletePendingUnreplicatedData = screen.getByTestId( + overviewLocators.deletePendingTotalUnreplicatedData + ); + const deletePendingKeys = screen.getByTestId(overviewLocators.deletePendingKeys); + + expect(deletePendingReplicatedData).toBeVisible(); + expect(deletePendingUnreplicatedData).toBeVisible(); + expect(deletePendingKeys).toBeVisible(); + + expect(deletePendingReplicatedData).toHaveTextContent( + (scenario) + ? /Total Replicated Data\s*1 KB/ + : /Total Replicated Data\s*N\/A/ + ); + expect(deletePendingUnreplicatedData).toHaveTextContent( + (scenario) + ? /Total Unreplicated Data\s*4 KB/ + : /Total Unreplicated Data\s*N\/A/ + ); + expect(deletePendingKeys).toHaveTextContent( + (scenario) + ? /Delete Pending Keys\s*3/ + : /Delete Pending Keys\s*N\/A/ + ); + }); + + it('Open Keys summary has the correct data', () => { + const openKeysReplicatedData = screen.getByTestId( + overviewLocators.openTotalReplicatedData + ); + const openKeysUnreplicatedData = screen.getByTestId( + overviewLocators.openTotalUnreplicatedData + ); + const openKeys = screen.getByTestId(overviewLocators.openKeys); + + expect(openKeysReplicatedData).toBeVisible(); + expect(openKeysUnreplicatedData).toBeVisible(); + expect(openKeys).toBeVisible(); + + expect(openKeysReplicatedData).toHaveTextContent( + (scenario) + ? /Total Replicated Data\s*1 KB/ + : /Total Replicated Data\s*N\/A/ + ); + expect(openKeysUnreplicatedData).toHaveTextContent( + (scenario) + ? /Total Unreplicated Data\s*4 KB/ + : /Total Unreplicated Data\s*N\/A/ + ); + expect(openKeys).toHaveTextContent( + (scenario) + ? /Open Keys\s*10/ + : /Open Keys\s*N\/A/ + ); + }); +}) diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/locators/locators.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/locators/locators.ts new file mode 100644 index 00000000000..23fbc768703 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/locators/locators.ts @@ -0,0 +1,48 @@ +/* + * 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. + */ + +export const overviewLocators = { + 'datanodeRow': 'overview-Health-Datanodes', + 'containersRow': 'overview-Health-Containers', + 'capacityOzoneUsed': 'capacity-ozone-used', + 'capacityNonOzoneUsed': 'capacity-non-ozone-used', + 'capacityRemaining': 'capacity-remaining', + 'capacityPreAllocated': 'capacity-pre-allocated', + 'volumesCard': 'overview-Volumes', + 'bucketsCard': 'overview-Buckets', + 'keysCard': 'overview-Keys', + 'pipelinesCard': 'overview-Pipelines', + 'deletedContainersCard': 'overview-Deleted Containers', + 'openTotalReplicatedData': 'overview-Open Keys Summary-Total Replicated Data', + 'openTotalUnreplicatedData': 'overview-Open Keys Summary-Total Unreplicated Data', + 'openKeys': 'overview-Open Keys Summary-Open Keys', + 'deletePendingTotalReplicatedData': 'overview-Delete Pending Keys Summary-Total Replicated Data', + 'deletePendingTotalUnreplicatedData': 'overview-Delete Pending Keys Summary-Total Unreplicated Data', + 'deletePendingKeys': 'overview-Delete Pending Keys Summary-Delete Pending Keys' +} + +export const datanodeLocators = { + 'datanodeContainer': 'datanodes-container', + 'datanodeMultiSelect': 'datanodes-multiselect' +} + +export const autoReloadPanelLocators = { + 'autoreloadPanel': 'autoreload-panel', + 'refreshButton': 'autoreload-panel-refresh', + 'toggleSwitch': 'autoreload-panel-switch' +} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/mocks/overviewMocks/overviewResponseMocks.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/mocks/overviewMocks/overviewResponseMocks.ts new file mode 100644 index 00000000000..2a0bbb687fd --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/mocks/overviewMocks/overviewResponseMocks.ts @@ -0,0 +1,50 @@ +/* + * 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. + */ + +export const ClusterState = { + "deletedDirs": 0, + "pipelines": 7, + "totalDatanodes": 5, + "healthyDatanodes": 3, + "storageReport": { + "capacity": 1352149585920, + "used": 822805801, + "remaining": 1068824879104, + "committed": 12000222315 + }, + "containers": 20, + "missingContainers": 2, + "openContainers": 8, + "deletedContainers": 10, + "volumes": 2, + "buckets": 24, + "keys": 1424, + "keysPendingDeletion": 2 +} + +export const OpenKeys = { + "totalUnreplicatedDataSize": 4096, + "totalReplicatedDataSize": 1024, + "totalOpenKeys": 10 +} + +export const DeletePendingSummary = { + "totalUnreplicatedDataSize": 4096, + "totalReplicatedDataSize": 1024, + "totalDeletedKeys": 3 +} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/mocks/overviewMocks/overviewServer.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/mocks/overviewMocks/overviewServer.ts new file mode 100644 index 00000000000..748f8e4ed53 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/mocks/overviewMocks/overviewServer.ts @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { setupServer } from "msw/node"; +import { rest } from "msw"; + +import * as mockResponses from "./overviewResponseMocks"; + +const handlers = [ + rest.get("api/v1/clusterState", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(mockResponses.ClusterState) + ); + }), + rest.get("api/v1/task/status", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(mockResponses.TaskStatus) + ); + }), + rest.get("api/v1/keys/open/summary", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(mockResponses.OpenKeys) + ); + }), + rest.get("api/v1/keys/deletePending/summary", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(mockResponses.DeletePendingSummary) + ); + }) +] + +const faultyHandlers = [ + rest.get("api/v1/clusterState", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(null) + ); + }), + rest.get("api/v1/task/status", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(null) + ); + }), + rest.get("api/v1/keys/open/summary", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(null) + ); + }), + rest.get("api/v1/keys/deletePending/summary", (req, res, ctx) => { + return res( + ctx.status(200), + ctx.json(null) + ); + }) +] +//This will configure a request mocking server using MSW +export const overviewServer = setupServer(...handlers); +export const faultyOverviewServer = setupServer(...faultyHandlers); diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/vitest.setup.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/vitest.setup.ts new file mode 100644 index 00000000000..54dc2d5e52b --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/__tests__/vitest.setup.ts @@ -0,0 +1,61 @@ +/* + * 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 '@testing-library/jest-dom/vitest'; + + +/** jsdom currently doesn't implement a local storage + * Hence we need to implement a minimal storage functionality + * for the virtualDOM, to be able to use localstorage in virtual components +*/ +const localStorageMock = (function () { + let store: { [key: string]: any } = {} + + return { + getItem: function (key: string): any | null { + return store[key] || null; + }, + setItem: function (key: string, value: any): void { + store[key] = value.toString(); + }, + removeItem: function (key: string): void { + delete store[key] + }, + clear: function (): void { + store = {} + } + } +}); + +Object.defineProperty(window, 'localStorage', { + value: localStorageMock +}); + +Object.defineProperty(window, 'matchMedia', { + writable: true, + value: vi.fn().mockImplementation(query => ({ + matches: false, + media: query, + onchange: null, + addListener: vi.fn(), // deprecated + removeListener: vi.fn(), // deprecated + addEventListener: vi.fn(), + removeEventListener: vi.fn(), + dispatchEvent: vi.fn(), + })), +}) \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.less b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.less index 3d2cbc814e4..1895cabc184 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.less +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.less @@ -44,6 +44,13 @@ font-weight: 500; } +.page-header-v2 { + padding: 10px 20px; + font-size: 20px; + font-weight: 500; + background-color: #FFFFFF; +} + .content-div { padding: 24px; background-color: #FFF; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.tsx index 3fec211e7ef..0ad6aa3f174 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.tsx @@ -16,16 +16,19 @@ * limitations under the License. */ -import React from 'react'; +import React, { Suspense } from 'react'; import { Switch as AntDSwitch, Layout } from 'antd'; import NavBar from './components/navBar/navBar'; import Breadcrumbs from './components/breadcrumbs/breadcrumbs'; import { HashRouter as Router, Switch, Route, Redirect } from 'react-router-dom'; import { routes } from '@/routes'; +import { routesV2 } from '@/v2/routes-v2'; import { MakeRouteWithSubRoutes } from '@/makeRouteWithSubRoutes'; import classNames from 'classnames'; +import Loader from '@/v2/components/loader/loader'; + import './app.less'; const { @@ -51,7 +54,7 @@ class App extends React.Component, IAppState> { }; render() { - const { collapsed } = this.state; + const { collapsed, enableNewUI } = this.state; const layoutClass = classNames('content-layout', { 'sidebar-collapsed': collapsed }); @@ -73,13 +76,18 @@ class App extends React.Component, IAppState> { }} />

- + - { - routes.map( + {(enableNewUI) + ? }> + {routesV2.map( + (route, index) => + )} + + : routes.map( (route, index) => ) } diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/autoReloadPanel/autoReloadPanel.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/autoReloadPanel/autoReloadPanel.tsx index 6510b3b3b66..0230d4dd61d 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/autoReloadPanel/autoReloadPanel.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/autoReloadPanel/autoReloadPanel.tsx @@ -84,11 +84,24 @@ class AutoReloadPanel extends React.Component { ); return ( -
+
Auto Refresh -   +   +   | Refreshed at {lastRefreshedText} -  
); diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/overviewCard/overviewCard.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/overviewCard/overviewCard.tsx index c8e19db14c8..977eddb9811 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/overviewCard/overviewCard.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/components/overviewCard/overviewCard.tsx @@ -142,10 +142,10 @@ class OverviewCard extends React.Component { render() { let { icon, data, title, loading, hoverable, storageReport, linkToUrl, error } = this.props; - let meta = ; + let meta = ; let errorClass = error ? 'card-error' : ''; - if (typeof data === 'string' && data === 'N/A'){ + if (typeof data === 'string' && data === 'N/A') { errorClass = 'card-error'; } diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/utils/common.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/utils/common.tsx index 6886fd189f6..f641b8797d9 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/utils/common.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/utils/common.tsx @@ -44,9 +44,8 @@ const showInfoNotification = (title: string, description: string) => { export const showDataFetchError = (error: string) => { let title = 'Error while fetching data'; - if (error.includes('CanceledError')) { - error = 'Previous request cancelled because context changed' - } + + if (error.includes('CanceledError')) return; if (error.includes('metadata')) { title = 'Metadata Initialization:'; showInfoNotification(title, error); diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/aclDrawer/aclDrawer.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/aclDrawer/aclDrawer.tsx new file mode 100644 index 00000000000..af0931c17fa --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/aclDrawer/aclDrawer.tsx @@ -0,0 +1,119 @@ +/* + * 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 React, { useEffect, useState } from 'react'; +import { Table, Drawer, Tag } from 'antd'; + +import { AclRightsColorMap, AclIdColorMap } from '@/v2/constants/acl.constants'; +import { Acl, ACLIdentity, ACLIdentityTypeList } from '@/v2/types/acl.types'; +import { ColumnType } from 'antd/es/table'; + +// ------------- Types -------------- // +type AclDrawerProps = { + visible: boolean; + acls: Acl[] | undefined; + entityName: string; + entityType: string; + onClose: () => void; +} + + +// ------------- Component -------------- // +const AclPanel: React.FC = ({ + visible, + acls, + entityType, + entityName, + onClose +}) => { + const [isVisible, setIsVisible] = useState(false); + + useEffect(() => { + setIsVisible(visible); + }, [visible]); + + const renderAclList = (_: string, acl: Acl) => { + return acl.aclList.map(aclRight => ( + + {aclRight} + + )) + } + + const renderAclIdentityType = (acl: string) => { + return ( + + {acl} + + ) + } + + const COLUMNS: ColumnType[] = [ + { + title: 'Name', + dataIndex: 'name', + key: 'name', + sorter: (a: Acl, b: Acl) => a.name.localeCompare(b.name), + }, + { + title: 'ACL Type', + dataIndex: 'type', + key: 'type', + filterMultiple: true, + filters: ACLIdentityTypeList.map(state => ({ text: state, value: state })), + onFilter: (value: ACLIdentity, record: Acl) => (record.type === value), + sorter: (a: Acl, b: Acl) => a.type.localeCompare(b.type), + render: renderAclIdentityType + }, + { + title: 'ACL Scope', + dataIndex: 'scope', + key: 'scope', + }, + { + title: 'ACLs', + dataIndex: 'aclList', + key: 'acls', + render: renderAclList + } + ]; + + return ( +
+ + +
+
+
+ ); +}; + +export default AclPanel; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/eChart/eChart.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/eChart/eChart.tsx new file mode 100644 index 00000000000..79fa0760338 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/eChart/eChart.tsx @@ -0,0 +1,89 @@ +/* + * 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 React, { useRef, useEffect } from "react"; +import { init, getInstanceByDom } from 'echarts'; +import type { CSSProperties } from "react"; +import type { EChartsOption, ECharts, SetOptionOpts } from 'echarts'; + +export interface EChartProps { + option: EChartsOption; + style?: CSSProperties; + settings?: SetOptionOpts; + loading?: boolean; + theme?: 'light'; + onClick?: () => any | void; +} + +const EChart = ({ + option, + style, + settings, + loading, + theme, + onClick +}: EChartProps): JSX.Element => { + const chartRef = useRef(null); + useEffect(() => { + // Initialize chart + let chart: ECharts | undefined; + if (chartRef.current !== null) { + chart = init(chartRef.current, theme); + if (onClick) { + chart.on('click', onClick); + } + } + + // Add chart resize listener + // ResizeObserver is leading to a bit janky UX + function resizeChart() { + chart?.resize(); + } + window.addEventListener("resize", resizeChart); + + // Return cleanup function + return () => { + chart?.dispose(); + window.removeEventListener("resize", resizeChart); + }; + }, [theme]); + + useEffect(() => { + // Update chart + if (chartRef.current !== null) { + const chart = getInstanceByDom(chartRef.current); + chart!.setOption(option, settings); + if (onClick) { + chart!.on('click', onClick); + } + } + }, [option, settings, theme]); // Whenever theme changes we need to add option and setting due to it being deleted in cleanup function + + useEffect(() => { + // Update chart + if (chartRef.current !== null) { + const chart = getInstanceByDom(chartRef.current); + // eslint-disable-next-line @typescript-eslint/no-unused-expressions + loading === true ? chart!.showLoading() : chart!.hideLoading(); + } + }, [loading, theme]); // If we switch theme we should put chart in loading mode, and also if loading changes i.e completes then hide loader + + return
; +} + +export default EChart; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/errorBoundary/errorBoundary.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/errorBoundary/errorBoundary.tsx new file mode 100644 index 00000000000..a7f7c9f45a8 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/errorBoundary/errorBoundary.tsx @@ -0,0 +1,52 @@ +/* + * 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 React from "react"; + +type ErrorProps = { + fallback: string | React.ReactNode; + children: React.ReactNode; +} + +type ErrorState = { + hasError: boolean; +} + +class ErrorBoundary extends React.Component{ + constructor(props: ErrorProps) { + super(props); + this.state = { hasError: false } + } + + static getDerivedStateFromError(error: Error) { + return { hasError: true } + } + + componentDidCatch(error: Error, errorInfo: React.ErrorInfo): void { + console.error(error, errorInfo) + } + + render(): React.ReactNode { + if (this.state.hasError) { + return this.props.fallback; + } + return this.props.children; + } +} + +export default ErrorBoundary; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/loader/loader.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/loader/loader.tsx new file mode 100644 index 00000000000..b05eaa5f0a5 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/loader/loader.tsx @@ -0,0 +1,40 @@ +/* + * 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 React from "react" +import { Spin } from "antd" +import { LoadingOutlined } from "@ant-design/icons" + +// ------------- Constants -------------- // +const loaderStyle: React.CSSProperties = { + height: '100%', + width: '100%', + textAlign: 'center', + paddingTop: '25%' +} + +// ------------- Component -------------- // +const Loader: React.FC = () => { + return ( +
+ }/> +
+ ) +} + +export default Loader; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewCardWrapper.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewCardWrapper.tsx new file mode 100644 index 00000000000..5e0998e6f64 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewCardWrapper.tsx @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import React from 'react'; +import { Link } from 'react-router-dom'; + +// ------------- Types -------------- // +type OverviewCardWrapperProps = { + linkToUrl: string; + title: string; + children: React.ReactElement; +} + +// ------------- Component -------------- // +const OverviewCardWrapper: React.FC = ({ + linkToUrl = '/', + title = '', + children = <> +}) => { + + const setCurrentActiveTab = (title: string): { active: string } => { + if (title === 'Open Keys Summary') { + return { + active: '2' + } + } + else if (title === 'Pending Deleted Keys Summary') { + return { + active: '3' + } + } + else if (title === 'OM Service') { + return { + active: '4' + } + } + return { + active: '1' + } + }; + + if (linkToUrl === '/Om') { + return ( + + {children} + + ); + } + else if (linkToUrl) { + return ( + + {children} + + ); + } + else { + return children; + } +} + +export default OverviewCardWrapper; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewSimpleCard.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewSimpleCard.tsx new file mode 100644 index 00000000000..a685e3a2bcd --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewSimpleCard.tsx @@ -0,0 +1,148 @@ +/* + * 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 React from 'react'; +import { Card, Col, Row } from 'antd'; +import { Link } from 'react-router-dom'; +import { + ClusterOutlined, + ContainerOutlined, + DatabaseOutlined, + DeleteOutlined, + DeploymentUnitOutlined, + FileTextOutlined, + FolderOpenOutlined, + InboxOutlined, + QuestionCircleOutlined +} from '@ant-design/icons'; + + +// ------------- Types -------------- // +type IconOptions = { + [key: string]: React.ReactElement +} + +type OverviewCardProps = { + icon: string; + data: number | React.ReactElement; + title: string; + hoverable?: boolean; + loading?: boolean; + linkToUrl?: string; +} + +// ------------- Styles -------------- // +const defaultIconStyle: React.CSSProperties = { + fontSize: '50px', + float: 'right' +}; +const iconStyle: React.CSSProperties = { + fontSize: '20px', + paddingRight: '4px', + float: 'inline-start' +}; +const cardHeadStyle: React.CSSProperties = { fontSize: '14px' }; +const cardBodyStyle: React.CSSProperties = { + padding: '16px', + justifyTracks: 'space-between' +}; +const dataColStyle: React.CSSProperties = { fontSize: '24px' }; +const titleLinkStyle: React.CSSProperties = { fontWeight: 400 } + +// Since AntD no longer supports string icon component +// we are using a utility function to map the strings to +// the appropriate Icon to render +const IconSelector = ({ + iconType, style +}: { + iconType: string; + style: React.CSSProperties +}) => { + const Icons: IconOptions = { + 'cluster': , + 'deployment-unit': , + 'database': , + 'container': , + 'inbox': , + 'folder-open': , + 'file-text': , + 'delete': + }; + + const selectIcon = (iconType: string): React.ReactElement => { + // Setting the default Icon as a question mark in case no match found + let ico = + + const found = Object.entries(Icons).find( + ([k]) => k.toLowerCase() === iconType.toLowerCase() + ); + + if (found) { + [, ico] = found; + } + return ico; + } + return selectIcon(iconType); +} + + +// ------------- Component -------------- // +const OverviewSimpleCard: React.FC = ({ + icon = '', + data = 0, + title = '', + hoverable = false, + loading = false, + linkToUrl = '' +}) => { + + const titleElement = (linkToUrl) + ? ( +
+ {title} + + View More + +
) + : title + + return ( + + + + + + + {data} + + + + ); +} + +export default OverviewSimpleCard; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewStorageCard.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewStorageCard.tsx new file mode 100644 index 00000000000..d6e29a2f968 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewStorageCard.tsx @@ -0,0 +1,244 @@ +/* + * 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 React, { HTMLAttributes, useMemo } from 'react'; +import filesize from 'filesize'; +import { Card, Row, Col, Table, Tag } from 'antd'; + +import EChart from '@/v2/components/eChart/eChart'; +import OverviewCardWrapper from '@/v2/components/overviewCard/overviewCardWrapper'; + +import { StorageReport } from '@/v2/types/overview.types'; + +// ------------- Types -------------- // +type OverviewStorageCardProps = { + loading?: boolean; + storageReport: StorageReport; +} + +const size = filesize.partial({ round: 1 }); + +function getUsagePercentages( + { used, remaining, capacity, committed }: StorageReport): ({ + ozoneUsedPercentage: number, + nonOzoneUsedPercentage: number, + committedPercentage: number, + usagePercentage: number + }) { + return { + ozoneUsedPercentage: Math.floor(used / capacity * 100), + nonOzoneUsedPercentage: Math.floor((capacity - remaining - used) / capacity * 100), + committedPercentage: Math.floor(committed / capacity * 100), + usagePercentage: Math.floor((capacity - remaining) / capacity * 100) + } +} + +// ------------- Styles -------------- // +const cardHeadStyle: React.CSSProperties = { fontSize: '14px' }; +const cardBodyStyle: React.CSSProperties = { padding: '16px' }; +const cardStyle: React.CSSProperties = { + boxSizing: 'border-box', + height: '100%' +} +const eChartStyle: React.CSSProperties = { + width: '280px', + height: '200px' +} + + +// ------------- Component -------------- // +const OverviewStorageCard: React.FC = ({ + loading = false, + storageReport = { + capacity: 0, + used: 0, + remaining: 0, + committed: 0 + } +}) => { + + const { + ozoneUsedPercentage, + nonOzoneUsedPercentage, + committedPercentage, + usagePercentage + } = useMemo(() => + getUsagePercentages(storageReport), + [ + storageReport.capacity, + storageReport.committed, + storageReport.remaining, + storageReport.used, + ] + ) + + let capacityData = [{ + value: ozoneUsedPercentage, + itemStyle: { + color: '#52C41A' + } + }, { + value: nonOzoneUsedPercentage, + itemStyle: { + color: '#1890FF' + } + }, { + value: committedPercentage, + itemStyle: { + color: '#FF595E' + } + }] + // Remove all zero values + // because guage chart shows a dot if value is zero + capacityData = capacityData.filter((val) => val.value > 0) + + const eChartOptions = { + title: { + left: 'center', + bottom: 'bottom', + text: `${size(storageReport.capacity - storageReport.remaining)} / ${size(storageReport.capacity)}`, + textStyle: { + fontWeight: 'normal', + fontFamily: 'Roboto' + } + }, + series: [ + { + type: 'gauge', + startAngle: 90, + endAngle: -270, + radius: '70%', + center: ['50%', '45%'], + bottom: '50%', + pointer: { + show: false + }, + progress: { + show: true, + overlap: true, + roundCap: true, + clip: true + }, + splitLine: { + show: false + }, + axisTick: { + show: false + }, + axisLabel: { + show: false, + distance: 50 + }, + detail: { + rich: { + value: { + fontSize: 24, + fontWeight: 400, + fontFamily: 'Roboto', + color: '#1B232A' + }, + percent: { + fontSize: 20, + fontWeight: 400, + color: '#1B232A' + } + }, + formatter: `{value|${usagePercentage}}{percent|%}`, + offsetCenter: [0, 0] + }, + data: capacityData + } + ] + } + + const cardChildren = ( + + + + + + + Ozone Used, + size: size(storageReport.used) + }, + { + key: 'non-ozone-used', + usage: Non Ozone Used, + size: size(storageReport.capacity - storageReport.remaining - storageReport.used) + }, + { + key: 'remaining', + usage: + Remaining + , + size: size(storageReport.remaining) + }, + { + key: 'pre-allocated', + usage: Container Pre-allocated, + size: size(storageReport.committed) + } + ]} + onRow={(record) => ({ + 'data-testid': `capacity-${record.key}` + }) as HTMLAttributes} /> + + + + ) + + return ( + + ) +} + +export default OverviewStorageCard; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewSummaryCard.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewSummaryCard.tsx new file mode 100644 index 00000000000..e383512f20e --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/overviewCard/overviewSummaryCard.tsx @@ -0,0 +1,111 @@ +/* + * 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 React, { HTMLAttributes } from 'react'; +import { Card, Row, Table } from 'antd'; + +import { ColumnType } from 'antd/es/table'; +import { Link } from 'react-router-dom'; + +// ------------- Types -------------- // +type TableData = { + key: React.Key; + name: string; + value: string; + action?: React.ReactElement | string; +} + +type OverviewTableCardProps = { + title: string; + columns: ColumnType[]; + tableData: TableData[]; + hoverable?: boolean; + loading?: boolean; + data?: string | React.ReactElement; + linkToUrl?: string; + showHeader?: boolean; +} + +// ------------- Styles -------------- // +const cardStyle: React.CSSProperties = { + height: '100%' +} +const cardHeadStyle: React.CSSProperties = { + fontSize: '14px' +} +const cardBodyStyle: React.CSSProperties = { + padding: '16px', + justifyTracks: 'space-between' +} + + +// ------------- Component -------------- // +const OverviewSummaryCard: React.FC = ({ + data = '', + title = '', + hoverable = false, + loading = false, + columns = [], + tableData = [], + linkToUrl = '', + showHeader = false +}) => { + + const titleElement = (linkToUrl) + ? ( +
+ {title} + View Insights +
) + : title + + return ( + + { + (data) && + + {data} + + } +
({ + 'data-testid': `overview-${title}-${record.name}` + } as HTMLAttributes)}/> + + ) +} + +export default OverviewSummaryCard; \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/search/search.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/search/search.tsx new file mode 100644 index 00000000000..21d4341787e --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/search/search.tsx @@ -0,0 +1,70 @@ +/* + * 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 React from 'react'; +import { Input, Select } from 'antd'; + +import { Option } from '@/v2/components/select/singleSelect'; + +// ------------- Types -------------- // +type SearchProps = { + searchColumn?: string; + searchInput: string; + searchOptions?: Option[]; + onSearchChange: ( + arg0: React.ChangeEvent + ) => void; + onChange: ( + value: string, + //OptionType, OptionGroupData and OptionData are not + //currently exported by AntD hence set to any + option: any + ) => void; +} + +// ------------- Component -------------- // +const Search: React.FC = ({ + searchColumn, + searchInput = '', + searchOptions = [], + onSearchChange = () => {}, + onChange = () => {} // Assign default value as a void function +}) => { + + const selectFilter = searchColumn + ? ( + ) +} + +export default Search; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/columnTag.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/columnTag.tsx new file mode 100644 index 00000000000..f367504286f --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/columnTag.tsx @@ -0,0 +1,67 @@ +/* + * 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 React from "react"; +import { Tag } from "antd"; +import { createPortal } from "react-dom"; + + +// ------------- Types -------------- // +/** + * Due to design decisions we are currently not using the Tags + * Until we reach a concensus on a better way to display the filter + * Keeping the code in case we require it in the future + */ +export type TagProps = { + label: string; + closable: boolean; + tagRef: React.RefObject; + onClose: (arg0: string) => void; +} + +// ------------- Component -------------- // +const ColumnTag: React.FC = ({ + label = '', + closable = true, + tagRef = null, + onClose = () => {} // Assign default value as void funciton +}) => { + const onPreventMouseDown = (event: React.MouseEvent) => { + // By default when clickin on the tags the text will get selected + // which might interfere with user experience as people would want to close tags + // but accidentally select tag text. Hence we prevent this behaviour. + event.preventDefault(); + event.stopPropagation(); + }; + + if (!tagRef?.current) return null; + + return createPortal( + (onClose(label))} + style={{marginRight: 3}}> + {label} + , + tagRef.current + ); +} + +export default ColumnTag; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/multiSelect.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/multiSelect.tsx new file mode 100644 index 00000000000..7a6b494aaeb --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/multiSelect.tsx @@ -0,0 +1,104 @@ +/* + * 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 React from "react"; +import { + default as ReactSelect, + Props as ReactSelectProps, + components, + OptionProps, + ValueType +} from 'react-select'; + +import { selectStyles } from "@/v2/constants/select.constants"; + + +// ------------- Types -------------- // +export type Option = { + label: string; + value: string; +} + +interface MultiSelectProps extends ReactSelectProps { + options: Option[]; + selected: Option[]; + placeholder: string; + fixedColumn: string; + columnLength: number; + onChange: (arg0: ValueType) => void; + onTagClose: (arg0: string) => void; +} + +// ------------- Component -------------- // +const MultiSelect: React.FC = ({ + options = [], + selected = [], + maxSelected = 5, + placeholder = 'Columns', + fixedColumn, + columnLength, + tagRef, + onTagClose = () => { }, // Assign default value as a void function + onChange = () => { }, // Assign default value as a void function + ...props +}) => { + + const Option: React.FC> = (props) => { + return ( +
+ + null} /> + + +
+ ) + } + + return ( + ) => { + if (selected?.length === options.length) return onChange!(options); + return onChange!(selected); + }} + styles={selectStyles} /> + ) +} + +export default MultiSelect; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/singleSelect.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/singleSelect.tsx new file mode 100644 index 00000000000..41ab03f5982 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/components/select/singleSelect.tsx @@ -0,0 +1,87 @@ +/* + * 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 React from "react"; +import Select, { + Props as ReactSelectProps, + components, + ValueType, + ValueContainerProps, + StylesConfig +} from 'react-select'; + +import { selectStyles } from "@/v2/constants/select.constants"; + + +// ------------- Types -------------- // +export type Option = { + label: string; + value: string; +} + +interface SingleSelectProps extends ReactSelectProps { + options: Option[]; + placeholder: string; + onChange: (arg0: ValueType) => void; +} + +// ------------- Component -------------- // +const SingleSelect: React.FC = ({ + options = [], + placeholder = 'Limit', + onChange = () => { }, // Assign default value as a void function + ...props // Desctructure other select props +}) => { + + + const ValueContainer = ({ children, ...props }: ValueContainerProps) => { + const selectedLimit = props.getValue() as Option[]; + return ( + + {React.Children.map(children, (child) => ( + ((child as React.ReactElement> + | React.ReactPortal)?.type as React.JSXElementConstructor)).name === "DummyInput" + ? child + : null + )} + Limit: {selectedLimit[0]?.label ?? ''} + + ); + }; + + return ( +
+ + + setShowPanel(false)}/> + + + ); +} + +export default Volumes; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.test.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/routes-v2.tsx similarity index 72% rename from hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.test.tsx rename to hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/routes-v2.tsx index ad787462109..5d710246161 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/app.test.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/routes-v2.tsx @@ -15,13 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +import { lazy } from 'react'; -import React from 'react'; -import ReactDOM from 'react-dom'; -import App from './app'; +const Overview = lazy(() => import('@/v2/pages/overview/overview')); +const Volumes = lazy(() => import('@/v2/pages/volumes/volumes')) -it('renders without crashing', () => { - const div = document.createElement('div'); - ReactDOM.render(, div); - ReactDOM.unmountComponentAtNode(div); -}); +export const routesV2 = [ + { + path: '/Overview', + component: Overview + }, + { + path: '/Volumes', + component: Volumes + } +]; diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/acl.types.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/acl.types.ts new file mode 100644 index 00000000000..33cd047d18e --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/acl.types.ts @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +export const ACLIdentityTypeList = [ + 'USER', + 'GROUP', + 'WORLD', + 'ANONYMOUS', + 'CLIENT_IP' +] as const; +export type ACLIdentity = typeof ACLIdentityTypeList[number]; + +export const ACLRightList = [ + 'READ', + 'WRITE', + 'CREATE', + 'LIST', + 'DELETE', + 'READ_ACL', + 'WRITE_ACL', + 'ALL', + 'NONE' +] as const; +export type ACLRight = typeof ACLRightList[number]; + + +export type Acl = { + type: string; + name: string; + scope: string; + aclList: string[]; +} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/bucket.types.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/bucket.types.ts new file mode 100644 index 00000000000..8b2fd0c694c --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/bucket.types.ts @@ -0,0 +1,55 @@ +/* + * 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 { Acl } from "@/v2/types/acl.types"; + +// Corresponds to OzoneManagerProtocolProtos.StorageTypeProto +export const BucketStorageTypeList = [ + 'RAM_DISK', + 'SSD', + 'DISK', + 'ARCHIVE' +] as const; +export type BucketStorage = typeof BucketStorageTypeList[number]; + +// Corresponds to OzoneManagerProtocolProtos.BucketLayoutProto +export const BucketLayoutTypeList = [ + 'FILE_SYSTEM_OPTIMIZED', + 'OBJECT_STORE', + 'LEGACY' +] as const; +export type BucketLayout = typeof BucketLayoutTypeList[number]; + + +export type Bucket = { + volumeName: string; + bucketName: string; + isVersionEnabled: boolean; + storageType: BucketStorage; + creationTime: number; + modificationTime: number; + sourceVolume?: string; + sourceBucket?: string; + usedBytes: number; + usedNamespace: number; + quotaInBytes: number; + quotaInNamespace: number; + owner: string; + acls?: Acl[]; + bucketLayout: BucketLayout; +} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/overview.types.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/overview.types.ts new file mode 100644 index 00000000000..f8390fd4346 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/overview.types.ts @@ -0,0 +1,66 @@ +/* + * 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. + */ + +export type ClusterStateResponse = { + missingContainers: number; + totalDatanodes: number; + healthyDatanodes: number; + pipelines: number; + storageReport: StorageReport; + containers: number; + volumes: number; + buckets: number; + keys: number; + openContainers: number; + deletedContainers: number; + keysPendingDeletion: number; + scmServiceId: string; + omServiceId: string; +} + +export type OverviewState = { + loading: boolean; + datanodes: string; + pipelines: number; + containers: number; + volumes: number; + buckets: number; + keys: number; + missingContainersCount: number; + lastRefreshed: number; + lastUpdatedOMDBDelta: number; + lastUpdatedOMDBFull: number; + omStatus: string; + openContainers: number; + deletedContainers: number; + openSummarytotalUnrepSize: number; + openSummarytotalRepSize: number; + openSummarytotalOpenKeys: number; + deletePendingSummarytotalUnrepSize: number; + deletePendingSummarytotalRepSize: number; + deletePendingSummarytotalDeletedKeys: number; + scmServiceId: string; + omServiceId: string; +} + +export type StorageReport = { + capacity: number; + used: number; + remaining: number; + committed: number; +} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/volume.types.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/volume.types.ts new file mode 100644 index 00000000000..67f007706a4 --- /dev/null +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/v2/types/volume.types.ts @@ -0,0 +1,44 @@ +/* +* 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 { Acl } from "@/v2/types/acl.types"; +import { Option } from "@/v2/components/select/multiSelect"; + +export type Volume = { + volume: string; + owner: string; + admin: string; + creationTime: number; + modificationTime: number; + quotaInBytes: number; + quotaInNamespace: number; + usedNamespace: number; + acls?: Acl[]; +} + +export type VolumesResponse = { + totalCount: number; + volumes: Volume[]; +} + +export type VolumesState = { + data: Volume[]; + lastUpdated: number; + columnOptions: Option[]; + currentRow: Volume | Record; +} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx index 47fe41dd7b3..d7fdf2b9eb8 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx @@ -560,7 +560,7 @@ export class Datanodes extends React.Component, IDatanode onShowSizeChange: this.onShowSizeChange }; return ( -
+
Datanodes ({totalCount})
@@ -575,6 +575,7 @@ export class Datanodes extends React.Component, IDatanode value={selectedColumns} allOption={allColumnsOption} onChange={this._handleColumnChange} + data-testid='datanodes-multiselect' /> Columns
, I 0) ? ` (${missingDataSource.length})` : ''}`}> + tab={`Missing (${missingDataSource?.length ?? 0})`}> {generateTable(missingDataSource)} 0) ? ` (${underReplicatedDataSource.length})` : ''}`}> + tab={`Under-Replicated (${underReplicatedDataSource?.length ?? 0})`}> {generateTable(underReplicatedDataSource)} 0) ? ` (${overReplicatedDataSource.length})` : ''}`}> + tab={`Over-Replicated (${overReplicatedDataSource?.length ?? 0})`}> {generateTable(overReplicatedDataSource)} 0) ? ` (${misReplicatedDataSource.length})` : ''}`}> + tab={`Mis-Replicated (${misReplicatedDataSource?.length ?? 0})`}> {generateTable(misReplicatedDataSource)} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx index e0167bd11bd..c53be4f7a20 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx @@ -116,8 +116,8 @@ const COLUMNS = [ render: (datanodes: string[]) =>
{datanodes && datanodes.map(datanode =>
- triggerNode}> - {datanode && datanode.hostName} + triggerNode}> + {datanode?.hostName ?? 'N/A'}
)} diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/tsconfig.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/tsconfig.json index 1d94b955da5..610a55148a3 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/tsconfig.json +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/tsconfig.json @@ -21,9 +21,14 @@ "rootDir": "src", "baseUrl": "src", "paths": { - "@/*": ["*"] + "@/*": ["*"], + "@tests/*": ["__tests__/*"] }, - "types": ["vite/client", "vite-plugin-svgr/client"] + "types": [ + "vite/client", + "vite-plugin-svgr/client", + "vitest/globals" + ] }, "include": [ "src" diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite-env.d.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite-env.d.ts deleted file mode 100644 index 46167caa7e9..00000000000 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite-env.d.ts +++ /dev/null @@ -1,18 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at - -* http://www.apache.org/licenses/LICENSE-2.0 - -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -/// \ No newline at end of file diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts index 068f27bb2d9..ddb2832f39b 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/vite.config.ts @@ -1,19 +1,23 @@ -/** - * 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 +/* + * 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. + */ -* 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 { defineConfig, splitVendorChunkPlugin } from 'vite'; import { resolve } from 'path'; @@ -54,7 +58,8 @@ export default defineConfig({ }, resolve: { alias: { - "@": pathResolve('src') + "@": pathResolve('src'), + "@tests": pathResolve('src/__tests__') } }, css: { @@ -71,13 +76,9 @@ export default defineConfig({ }, test: { globals: true, - setupFiles: './src/setupTests.ts', - css: true, - reporters: ['verbose'], - coverage: { - reporter: ['text', 'json', 'html'], - include: ['src/**/*'], - exclude: [] - } + environment: 'jsdom', + setupFiles: 'src/__tests__/vitest.setup.ts', + include: ["src/__tests__/**/*.test.tsx"], + reporters: ['verbose'] } -}) \ No newline at end of file +}); diff --git a/hadoop-ozone/s3-secret-store/pom.xml b/hadoop-ozone/s3-secret-store/pom.xml index df34dd8cdc8..b2da4c9e3c8 100644 --- a/hadoop-ozone/s3-secret-store/pom.xml +++ b/hadoop-ozone/s3-secret-store/pom.xml @@ -47,6 +47,14 @@ com.bettercloud vault-java-driver + + jakarta.annotation + jakarta.annotation-api + + + org.slf4j + slf4j-api + diff --git a/hadoop-ozone/s3gateway/pom.xml b/hadoop-ozone/s3gateway/pom.xml index 01fe0a257bd..c26171d98ac 100644 --- a/hadoop-ozone/s3gateway/pom.xml +++ b/hadoop-ozone/s3gateway/pom.xml @@ -31,11 +31,31 @@ + + org.apache.ozone + hdds-client + + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-interface-server + org.apache.ozone ozone-common compile + + org.apache.ozone + ozone-interface-client + org.javassist javassist @@ -44,14 +64,55 @@ org.apache.ozone hdds-server-framework + + commons-codec + commons-codec + + + commons-io + commons-io + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.kerby + kerby-util + + + org.apache.ratis + ratis-common + + org.jboss.weld.servlet weld-servlet-shaded + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-webapp + org.glassfish.jersey.containers jersey-container-servlet-core + + org.glassfish.jersey.core + jersey-common + + + org.glassfish.jersey.core + jersey-server + org.glassfish.jersey.ext.cdi jersey-cdi1x @@ -68,6 +129,14 @@ org.glassfish.hk2 hk2-api + + org.slf4j + slf4j-api + + + com.fasterxml.jackson.core + jackson-databind + com.fasterxml.jackson.dataformat jackson-dataformat-xml @@ -76,10 +145,30 @@ com.fasterxml.jackson.module jackson-module-jaxb-annotations + + com.google.guava + guava + + + javax.annotation + javax.annotation-api + javax.enterprise cdi-api + + javax.servlet + javax.servlet-api + + + jakarta.annotation + jakarta.annotation-api + + + jakarta.ws.rs + jakarta.ws.rs-api + jakarta.xml.bind jakarta.xml.bind-api @@ -92,6 +181,10 @@ jakarta.activation jakarta.activation-api + + info.picocli + picocli + io.grpc grpc-netty @@ -118,6 +211,18 @@ io.netty netty-transport + + io.opentracing + opentracing-api + + + io.opentracing + opentracing-noop + + + io.opentracing + opentracing-util + org.apache.ozone hdds-hadoop-dependency-server diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java index 21c3f8358f4..41876c6e245 100644 --- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.io.Text; +import org.apache.hadoop.ozone.OzoneFsServerDefaults; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneInputStream; @@ -504,6 +505,11 @@ public TenantStateList listTenant() throws IOException { return null; } + @Override + public OzoneFsServerDefaults getServerDefaults() throws IOException { + return null; + } + @Override public KeyProvider getKeyProvider() throws IOException { return null; diff --git a/hadoop-ozone/tools/pom.xml b/hadoop-ozone/tools/pom.xml index 98a6fef13e1..04c1c8602cb 100644 --- a/hadoop-ozone/tools/pom.xml +++ b/hadoop-ozone/tools/pom.xml @@ -29,6 +29,42 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> jar + + org.apache.ozone + hdds-client + + + org.apache.ozone + hdds-common + + + org.apache.ozone + hdds-config + + + org.apache.ozone + hdds-container-service + + + org.apache.ozone + hdds-interface-admin + + + org.apache.ozone + hdds-interface-client + + + org.apache.ozone + hdds-interface-server + + + org.apache.ozone + hdds-managed-rocksdb + + + org.apache.ozone + hdds-server-scm + org.apache.ozone ozone-manager @@ -45,10 +81,22 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.ozone ozone-client + + org.apache.ozone + ozone-filesystem-common + org.apache.ozone ozone-filesystem + + org.apache.ozone + ozone-interface-client + + + org.apache.ozone + ozone-interface-storage + org.apache.ozone hdds-server-framework @@ -68,6 +116,55 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-hadoop-dependency-server + + commons-codec + commons-codec + + + commons-io + commons-io + + + org.apache.commons + commons-lang3 + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.ratis + ratis-client + + + org.apache.ratis + ratis-common + + + org.apache.ratis + ratis-proto + + + org.apache.ratis + ratis-server-api + + + org.apache.ratis + ratis-thirdparty-misc + + + org.apache.ratis + ratis-tools + + + + info.picocli + picocli + jakarta.xml.bind jakarta.xml.bind-api @@ -76,6 +173,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.glassfish.jaxb jaxb-runtime + + jakarta.annotation + jakarta.annotation-api + jakarta.activation jakarta.activation-api @@ -84,6 +185,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> io.dropwizard.metrics metrics-core + + io.opentracing + opentracing-api + + + io.opentracing + opentracing-util + com.amazonaws aws-java-sdk-core @@ -92,10 +201,42 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> com.amazonaws aws-java-sdk-s3 + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-jsr310 + + + com.google.guava + guava + + + org.jooq + jooq + org.kohsuke.metainf-services metainf-services + + org.rocksdb + rocksdbjni + + + org.slf4j + slf4j-api + diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java index 5592926bf88..a0aba2a1b15 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.common.InconsistentStorageStateException; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile; import org.apache.hadoop.ozone.container.common.impl.ContainerData; @@ -151,7 +152,9 @@ public void loadContainersFromVolumes() throws IOException { volumeSet, metrics, containerReplicaProto -> { - }); + }, + // Since this is an Ozone debug CLI, this instance is not part of a running datanode. + new ContainerChecksumTreeManager(conf)); handler.setClusterID(clusterId); handlers.put(containerType, handler); } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java index d471c13462f..0c525457aac 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.hdfs.server.datanode.StorageLocation; +import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.Handler; @@ -193,7 +194,9 @@ private void initializeReplicationSupervisor( volumeSet, metrics, containerReplicaProto -> { - }); + }, + // Since this a Freon tool, this instance is not part of a running datanode. + new ContainerChecksumTreeManager(conf)); handler.setClusterID(UUID.randomUUID().toString()); handlers.put(containerType, handler); } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java index 3b4d25cddaf..f83b2a1a4a9 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java @@ -24,8 +24,8 @@ import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.XceiverClientCreator; import org.apache.hadoop.hdds.scm.XceiverClientFactory; -import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; @@ -152,11 +152,10 @@ public Void call() throws Exception { XceiverClientFactory xceiverClientManager; if (OzoneSecurityUtil.isSecurityEnabled(configuration)) { CACertificateProvider caCerts = () -> HAUtils.buildCAX509List(null, configuration); - xceiverClientManager = new XceiverClientManager(configuration, - configuration.getObject(XceiverClientManager.ScmClientConfig.class), + xceiverClientManager = new XceiverClientCreator(configuration, new ClientTrustManager(caCerts, null)); } else { - xceiverClientManager = new XceiverClientManager(configuration); + xceiverClientManager = new XceiverClientCreator(configuration); } clients = new ArrayList<>(numClients); for (int i = 0; i < numClients; i++) { diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java index f6a5c596501..3e613d2d2c5 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java @@ -29,7 +29,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; -import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientCreator; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; @@ -99,8 +100,8 @@ public Void call() throws Exception { Pipeline pipeline = findPipelineForTest(pipelineId, scmLocationClient, LOG); - try (XceiverClientManager xceiverClientManager = - new XceiverClientManager(ozoneConf)) { + try (XceiverClientFactory xceiverClientManager = + new XceiverClientCreator(ozoneConf)) { client = xceiverClientManager.acquireClient(pipeline); timer = getMetrics().timer("put-block"); diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java index 6362f32d049..7f0f5bb9e57 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java @@ -36,7 +36,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; -import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientCreator; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientReply; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; @@ -117,8 +118,8 @@ public Void call() throws Exception { try (StorageContainerLocationProtocol scmLocationClient = createStorageContainerLocationClient(ozoneConf); - XceiverClientManager xceiverClientManager = - new XceiverClientManager(ozoneConf)) { + XceiverClientFactory xceiverClientManager = + new XceiverClientCreator(ozoneConf)) { List pipelinesFromSCM = scmLocationClient.listPipelines(); Pipeline firstPipeline; init(); @@ -246,8 +247,7 @@ private void sendWriteChunkRequest(DatanodeBlockID blockId, if (async) { XceiverClientReply xceiverClientReply = xceiverClientSpi.sendCommandAsync(request); - xceiverClientSpi - .watchForCommit(xceiverClientReply.getLogIndex()); + xceiverClientSpi.watchForCommit(xceiverClientReply.getLogIndex()).get(); } else { xceiverClientSpi.sendCommand(request); diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java index 2bbf8b6d5b2..0b1e34efe78 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java @@ -24,7 +24,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; -import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientCreator; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; @@ -91,8 +92,8 @@ public Void call() throws Exception { createStorageContainerLocationClient(ozoneConf)) { Pipeline pipeline = findPipelineForTest(pipelineId, scmClient, LOG); - try (XceiverClientManager xceiverClientManager = - new XceiverClientManager(ozoneConf)) { + try (XceiverClientFactory xceiverClientManager = + new XceiverClientCreator(ozoneConf)) { xceiverClient = xceiverClientManager.acquireClientForReadData(pipeline); checksumProtobuf = ContainerProtos.ChecksumData.newBuilder() diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopBaseFreonGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopBaseFreonGenerator.java new file mode 100644 index 00000000000..eeb6514a725 --- /dev/null +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopBaseFreonGenerator.java @@ -0,0 +1,85 @@ +/* + * 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.freon; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import picocli.CommandLine.Option; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.util.Optional; + +/** + * Base class for Freon generator tests that requires {@link FileSystem} instance. + */ +public abstract class HadoopBaseFreonGenerator extends BaseFreonGenerator { + + @Option(names = {"-r", "--rpath", "--path"}, + description = "Hadoop FS file system path. Use full path.", + defaultValue = "o3fs://bucket1.vol1") + private String rootPath; + + private final ThreadLocal threadLocalFileSystem = + ThreadLocal.withInitial(this::createFS); + + private OzoneConfiguration configuration; + private URI uri; + + @Override + public void init() { + super.init(); + configuration = createOzoneConfiguration(); + uri = URI.create(rootPath); + String scheme = Optional.ofNullable(uri.getScheme()) + .orElseGet(() -> FileSystem.getDefaultUri(configuration) + .getScheme()); + String disableCacheName = + String.format("fs.%s.impl.disable.cache", scheme); + print("Disabling FS cache: " + disableCacheName); + configuration.setBoolean(disableCacheName, true); + } + + @Override + protected void taskLoopCompleted() { + FileSystem fileSystem = threadLocalFileSystem.get(); + try { + fileSystem.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + protected String getRootPath() { + return rootPath; + } + + protected FileSystem getFileSystem() { + return threadLocalFileSystem.get(); + } + + private FileSystem createFS() { + try { + return FileSystem.get(uri, configuration); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + +} diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopDirTreeGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopDirTreeGenerator.java index 91d78234ba9..3eb879d5c06 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopDirTreeGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopDirTreeGenerator.java @@ -19,17 +19,14 @@ import com.codahale.metrics.Timer; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdds.cli.HddsVersionProvider; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageSize; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import java.net.URI; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; @@ -45,17 +42,12 @@ mixinStandardHelpOptions = true, showDefaultValues = true) @SuppressWarnings("java:S2245") // no need for secure random -public class HadoopDirTreeGenerator extends BaseFreonGenerator +public class HadoopDirTreeGenerator extends HadoopBaseFreonGenerator implements Callable { private static final Logger LOG = LoggerFactory.getLogger(HadoopDirTreeGenerator.class); - @Option(names = {"-r", "--rpath"}, - description = "Hadoop FS root path", - defaultValue = "o3fs://bucket2.vol2") - private String rootPath; - @Option(names = {"-d", "--depth"}, description = "Number of directories to be generated recursively", defaultValue = "5") @@ -99,8 +91,6 @@ public class HadoopDirTreeGenerator extends BaseFreonGenerator private ContentGenerator contentGenerator; - private FileSystem fileSystem; - @Override public Void call() throws Exception { String s; @@ -111,10 +101,7 @@ public Void call() throws Exception { s = "Invalid span value, span value should be greater than zero!"; print(s); } else { - init(); - OzoneConfiguration configuration = createOzoneConfiguration(); - fileSystem = FileSystem.get(URI.create(rootPath), configuration); - + super.init(); contentGenerator = new ContentGenerator(fileSize.toBytes(), bufferSize); timer = getMetrics().timer("file-create"); @@ -152,7 +139,7 @@ public Void call() throws Exception { created. */ private void createDir(long counter) throws Exception { - String dir = makeDirWithGivenNumberOfFiles(rootPath); + String dir = makeDirWithGivenNumberOfFiles(getRootPath()); if (depth > 1) { createSubDirRecursively(dir, 1, 1); } @@ -196,8 +183,8 @@ private void createSubDirRecursively(String parent, int depthIndex, private String makeDirWithGivenNumberOfFiles(String parent) throws Exception { String dir = RandomStringUtils.randomAlphanumeric(length); - dir = parent.toString().concat("/").concat(dir); - fileSystem.mkdirs(new Path(dir)); + dir = parent.concat("/").concat(dir); + getFileSystem().mkdirs(new Path(dir)); totalDirsCnt.incrementAndGet(); // Add given number of files into the created directory. createFiles(dir); @@ -212,7 +199,7 @@ private void createFile(String dir, long counter) throws Exception { LOG.debug("FilePath:{}", file); } timer.time(() -> { - try (FSDataOutputStream output = fileSystem.create(file)) { + try (FSDataOutputStream output = getFileSystem().create(file)) { contentGenerator.write(output); } return null; diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java index dd386fb9262..1f910c9398a 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java @@ -16,17 +16,12 @@ */ package org.apache.hadoop.ozone.freon; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.net.URI; -import java.util.Optional; import java.util.concurrent.Callable; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdds.cli.HddsVersionProvider; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; import com.codahale.metrics.Timer; import org.apache.hadoop.hdds.conf.StorageSize; @@ -42,14 +37,9 @@ versionProvider = HddsVersionProvider.class, mixinStandardHelpOptions = true, showDefaultValues = true) -public class HadoopFsGenerator extends BaseFreonGenerator +public class HadoopFsGenerator extends HadoopBaseFreonGenerator implements Callable { - @Option(names = {"--path"}, - description = "Hadoop FS file system path. Use full path.", - defaultValue = "o3fs://bucket1.vol1") - private String rootPath; - @Option(names = {"-s", "--size"}, description = "Size of the generated files. " + StorageSizeConverter.STORAGE_SIZE_DESCRIPTION, @@ -77,29 +67,12 @@ public class HadoopFsGenerator extends BaseFreonGenerator private Timer timer; - private OzoneConfiguration configuration; - private URI uri; - private final ThreadLocal threadLocalFileSystem = - ThreadLocal.withInitial(this::createFS); - @Override public Void call() throws Exception { - init(); - - configuration = createOzoneConfiguration(); - uri = URI.create(rootPath); - String scheme = Optional.ofNullable(uri.getScheme()) - .orElseGet(() -> FileSystem.getDefaultUri(configuration) - .getScheme()); - String disableCacheName = - String.format("fs.%s.impl.disable.cache", scheme); - print("Disabling FS cache: " + disableCacheName); - configuration.setBoolean(disableCacheName, true); - - Path file = new Path(rootPath + "/" + generateObjectName(0)); - try (FileSystem fileSystem = threadLocalFileSystem.get()) { - fileSystem.mkdirs(file.getParent()); - } + super.init(); + + Path file = new Path(getRootPath() + "/" + generateObjectName(0)); + getFileSystem().mkdirs(file.getParent()); contentGenerator = new ContentGenerator(fileSize.toBytes(), bufferSize, copyBufferSize, @@ -113,8 +86,8 @@ public Void call() throws Exception { } private void createFile(long counter) throws Exception { - Path file = new Path(rootPath + "/" + generateObjectName(counter)); - FileSystem fileSystem = threadLocalFileSystem.get(); + Path file = new Path(getRootPath() + "/" + generateObjectName(counter)); + FileSystem fileSystem = getFileSystem(); timer.time(() -> { try (FSDataOutputStream output = fileSystem.create(file)) { @@ -123,22 +96,4 @@ private void createFile(long counter) throws Exception { return null; }); } - - private FileSystem createFS() { - try { - return FileSystem.get(uri, configuration); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - protected void taskLoopCompleted() { - FileSystem fileSystem = threadLocalFileSystem.get(); - try { - fileSystem.close(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsValidator.java index fe160efd73c..1566eaed8ae 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsValidator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsValidator.java @@ -16,22 +16,18 @@ */ package org.apache.hadoop.ozone.freon; -import java.net.URI; import java.security.MessageDigest; import java.util.concurrent.Callable; import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdds.cli.HddsVersionProvider; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; import com.codahale.metrics.Timer; import org.apache.commons.io.IOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; -import picocli.CommandLine.Option; /** * Data generator tool test om performance. @@ -42,36 +38,24 @@ versionProvider = HddsVersionProvider.class, mixinStandardHelpOptions = true, showDefaultValues = true) -public class HadoopFsValidator extends BaseFreonGenerator +public class HadoopFsValidator extends HadoopBaseFreonGenerator implements Callable { private static final Logger LOG = LoggerFactory.getLogger(HadoopFsValidator.class); - @Option(names = {"--path"}, - description = "Hadoop FS file system path", - defaultValue = "o3fs://bucket1.vol1") - private String rootPath; - private ContentGenerator contentGenerator; private Timer timer; - private FileSystem fileSystem; - private byte[] referenceDigest; @Override public Void call() throws Exception { + super.init(); - init(); - - OzoneConfiguration configuration = createOzoneConfiguration(); - - fileSystem = FileSystem.get(URI.create(rootPath), configuration); - - Path file = new Path(rootPath + "/" + generateObjectName(0)); - try (FSDataInputStream stream = fileSystem.open(file)) { + Path file = new Path(getRootPath() + "/" + generateObjectName(0)); + try (FSDataInputStream stream = getFileSystem().open(file)) { referenceDigest = getDigest(stream); } @@ -83,10 +67,10 @@ public Void call() throws Exception { } private void validateFile(long counter) throws Exception { - Path file = new Path(rootPath + "/" + generateObjectName(counter)); + Path file = new Path(getRootPath() + "/" + generateObjectName(counter)); byte[] content = timer.time(() -> { - try (FSDataInputStream input = fileSystem.open(file)) { + try (FSDataInputStream input = getFileSystem().open(file)) { return IOUtils.toByteArray(input); } }); diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopNestedDirGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopNestedDirGenerator.java index 8e631e5b6a9..ff74a54fbc6 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopNestedDirGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopNestedDirGenerator.java @@ -16,13 +16,10 @@ */ package org.apache.hadoop.ozone.freon; -import java.net.URI; import java.util.concurrent.Callable; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdds.cli.HddsVersionProvider; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.commons.lang3.RandomStringUtils; import org.slf4j.Logger; @@ -41,17 +38,12 @@ mixinStandardHelpOptions = true, showDefaultValues = true) @SuppressWarnings("java:S2245") // no need for secure random -public class HadoopNestedDirGenerator extends BaseFreonGenerator +public class HadoopNestedDirGenerator extends HadoopBaseFreonGenerator implements Callable { private static final Logger LOG = LoggerFactory.getLogger(HadoopNestedDirGenerator.class); - @Option(names = {"-r", "--rpath"}, - description = "Hadoop FS directory system path", - defaultValue = "o3fs://bucket2.vol2") - private String rootPath; - @Option(names = {"-d", "--depth"}, description = "Number of directories to be generated recursively", defaultValue = "5") @@ -70,8 +62,6 @@ public class HadoopNestedDirGenerator extends BaseFreonGenerator defaultValue = "10") private int length; - private FileSystem fileSystem; - @Override public Void call() throws Exception { String s; @@ -82,9 +72,7 @@ public Void call() throws Exception { s = "Invalid span value, span value should be greater or equal to zero!"; print(s); } else { - init(); - OzoneConfiguration configuration = createOzoneConfiguration(); - fileSystem = FileSystem.get(URI.create(rootPath), configuration); + super.init(); runTests(this::createDir); } return null; @@ -109,14 +97,14 @@ private void createDir(long counter) throws Exception { dirString = dirString.concat("/").concat(RandomStringUtils. randomAlphanumeric(length)); } - Path file = new Path(rootPath.concat("/").concat(dirString)); - fileSystem.mkdirs(file.getParent()); + Path file = new Path(getRootPath().concat("/").concat(dirString)); + getFileSystem().mkdirs(file.getParent()); String leafDir = dirString.substring(0, dirString.length() - length); String tmp = "/0"; for (int i = 1; i <= span; i++) { String childDir = leafDir.concat(Integer.toString(i)).concat(tmp); - Path dir = new Path(rootPath.concat("/").concat(childDir)); - fileSystem.mkdirs(dir.getParent()); + Path dir = new Path(getRootPath().concat("/").concat(childDir)); + getFileSystem().mkdirs(dir.getParent()); } String message = "\nSuccessfully created directories. " + "Total Directories with level = " + depth + " and span = " + span; diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HsyncGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HsyncGenerator.java index 962a7448060..8de2ee032d0 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HsyncGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HsyncGenerator.java @@ -16,7 +16,6 @@ */ package org.apache.hadoop.ozone.freon; -import java.net.URI; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicInteger; @@ -54,17 +53,12 @@ versionProvider = HddsVersionProvider.class, mixinStandardHelpOptions = true, showDefaultValues = true) -public class HsyncGenerator extends BaseFreonGenerator implements Callable { +public class HsyncGenerator extends HadoopNestedDirGenerator implements Callable { private static final Logger LOG = LoggerFactory.getLogger(HsyncGenerator.class); @CommandLine.ParentCommand private Freon freon; - @Option(names = {"--path"}, - description = "Hadoop FS file system path. Use full path.", - defaultValue = "o3fs://bucket1.vol1") - private String rootPath; - @Option(names = {"--bytes-per-write"}, description = "Size of each write", defaultValue = "1024") @@ -78,7 +72,6 @@ public class HsyncGenerator extends BaseFreonGenerator implements Callable private Timer timer; private OzoneConfiguration configuration; - private FileSystem[] fileSystems; private FSDataOutputStream[] outputStreams; private Path[] files; private AtomicInteger[] callsPerFile; @@ -94,23 +87,20 @@ public HsyncGenerator() { @Override public Void call() throws Exception { - init(); + super.init(); if (configuration == null) { configuration = freon.createOzoneConfiguration(); } - URI uri = URI.create(rootPath); - fileSystems = new FileSystem[numberOfFiles]; outputStreams = new FSDataOutputStream[numberOfFiles]; files = new Path[numberOfFiles]; callsPerFile = new AtomicInteger[numberOfFiles]; + FileSystem fileSystem = getFileSystem(); for (int i = 0; i < numberOfFiles; i++) { - FileSystem fileSystem = FileSystem.get(uri, configuration); - Path file = new Path(rootPath + "/" + generateObjectName(i)); + Path file = new Path(getRootPath() + "/" + generateObjectName(i)); fileSystem.mkdirs(file.getParent()); outputStreams[i] = fileSystem.create(file); - fileSystems[i] = fileSystem; files[i] = file; callsPerFile[i] = new AtomicInteger(); @@ -126,9 +116,6 @@ public Void call() throws Exception { for (FSDataOutputStream outputStream : outputStreams) { outputStream.close(); } - for (FileSystem fs : fileSystems) { - fs.close(); - } } StringBuilder distributionReport = new StringBuilder(); diff --git a/pom.xml b/pom.xml index c0f1dc843f8..b5a6323bed9 100644 --- a/pom.xml +++ b/pom.xml @@ -112,9 +112,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 1.8.0 1.17.0 3.2.2 - 1.26.2 + 1.27.0 2.10.1 - 1.5.6-3 + 1.5.6-4 1.4.0 2.16.1 3.14.0 @@ -125,7 +125,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 1.12.0 1.6 1.5 - 1.7.1 + 1.9.0 ${project.build.directory}/test-dir ${test.build.dir} @@ -145,6 +145,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 1.2.2 2.3.3 2.3.9 + 0.10.4 3.1.19 0.1.55 2.0 @@ -173,7 +174,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 4.4.16 - 2.0.13 + 2.0.16 2.23.1 3.4.4 1.2.25 @@ -200,10 +201,14 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 4.2.2 2.6.1 2.1.1 + 2.0.2 + 2.1.6 + 1 2.12.7 + 1.0 0.21.0 - 2.28.0 + 2.29.2 32.1.3-jre 6.0.0 2.10.1 @@ -267,7 +272,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 3.7.1 4.2.2 0.45.0 - 3.3.0 + 3.4.1 2.4.0 1.0-beta-1 1.0-M1 @@ -291,8 +296,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 1.15.0 2.5.0 1.4.0 + 3.9.8.1 - 5.3.37 + 5.3.39 3.11.10 @@ -409,6 +415,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs ${hadoop.version} test-jar + + aopalliance + aopalliance + ${aopalliance.version} + com.google.guava guava @@ -465,6 +476,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs kerb-util ${kerby.version} + + org.apache.kerby + kerby-util + ${kerby.version} + commons-codec commons-codec @@ -480,6 +496,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs commons-validator ${commons-validator.version} + + com.github.jnr + jnr-constants + ${jnr-constants.version} + com.github.jnr jnr-posix @@ -505,6 +526,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs cdi-api ${cdi-api.version} + + javax.inject + javax.inject + ${javax.inject.version} + javax.servlet javax.servlet-api @@ -593,6 +619,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs jersey-container-servlet-core ${jersey2.version} + + org.glassfish.jersey.core + jersey-common + ${jersey2.version} + org.glassfish.jersey.core jersey-server @@ -666,6 +697,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs ${bonecp.version} + + org.apache.ratis + ratis-thirdparty-misc + ${ratis.thirdparty.version} + org.apache.ratis ratis-proto-shaded @@ -686,6 +722,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs org.apache.ratis ${ratis.version} + + org.apache.ratis + ratis-metrics-api + ${ratis.version} + ratis-metrics-dropwizard3 org.apache.ratis @@ -701,6 +742,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs org.apache.ratis ${ratis.version} + + org.apache.ratis + ratis-proto + ${ratis.version} + org.apache.ratis ratis-tools @@ -914,7 +960,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs org.springframework - spring-jdbc + spring-core ${spring.version} @@ -923,6 +969,16 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs + + org.springframework + spring-jdbc + ${spring.version} + + + org.springframework + spring-tx + ${spring.version} + com.fasterxml.woodstox woodstox-core @@ -1009,6 +1065,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs + + io.jaegertracing + jaeger-core + ${jaeger.version} + org.jetbrains.kotlin kotlin-bom @@ -1016,6 +1077,16 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs pom import + + io.opentracing + opentracing-api + ${opentracing.version} + + + io.opentracing + opentracing-noop + ${opentracing.version} + io.opentracing opentracing-util @@ -1025,6 +1096,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs io.prometheus simpleclient_dropwizard ${prometheus.version} + + + io.prometheus + simpleclient + ${prometheus.version} io.prometheus @@ -1170,6 +1246,16 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs jakarta.annotation-api ${jakarta.annotation.version} + + jakarta.validation + jakarta.validation-api + ${jakarta.validation.version} + + + jakarta.ws.rs + jakarta.ws.rs-api + ${jakarta.ws.rs-api.version} + joda-time joda-time @@ -1215,6 +1301,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs hadoop-shaded-guava ${hadoop-shaded-guava.version} + + com.github.vlsi.mxgraph + jgraphx + ${jgraphx.version} + com.nimbusds nimbus-jose-jwt @@ -1528,6 +1619,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs org.jetbrains.annotations.Nullable + + true + Use commons-lang v3 + + org.apache.commons.lang.** + +