diff --git a/NOTICE.txt b/NOTICE.txt index 7a1e855f6a3..cc4e3c58b39 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -1,5 +1,5 @@ Apache Ozone -Copyright 2022 The Apache Software Foundation +Copyright 2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/dev-support/pom.xml b/dev-support/pom.xml new file mode 100644 index 00000000000..71f76628a6f --- /dev/null +++ b/dev-support/pom.xml @@ -0,0 +1,98 @@ + + + + + ozone-main + org.apache.ozone + 1.4.1 + + 4.0.0 + ozone-dev-support + Helper module for sharing resources among projects + Apache Ozone Dev Support + + + false + + + + + ${project.build.directory}/extra-resources + META-INF + + LICENSE.txt + NOTICE.txt + + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + org.apache.maven.plugins + maven-resources-plugin + + + copy-resources + validate + + copy-resources + + + ${project.build.directory}/extra-resources + + + ../ + + LICENSE.txt + NOTICE.txt + + + + + + + + + + org.apache.maven.plugins + maven-remote-resources-plugin + + + process-resources + + bundle + + + + + ${project.build.outputDirectory} + + META-INF/LICENSE.txt + META-INF/NOTICE.txt + + + + + + 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 f8214687648..04e69bce0cf 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 @@ -38,9 +38,13 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Objects; +import java.util.Optional; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; @@ -141,8 +145,34 @@ ContainerCommandResponseProto> executePutBlock(boolean close, } if (checksumBlockData != null) { - List currentChunks = getContainerBlockData().getChunksList(); + + // For the same BlockGroupLength, we need to find the larger value of Block DataSize. + // This is because we do not send empty chunks to the DataNode, so the larger value is more accurate. + Map> maxDataSizeByGroup = Arrays.stream(blockData) + .filter(Objects::nonNull) + .collect(Collectors.groupingBy(BlockData::getBlockGroupLength, + Collectors.maxBy(Comparator.comparingLong(BlockData::getSize)))); + BlockData maxBlockData = maxDataSizeByGroup.get(blockGroupLength).get(); + + // When calculating the checksum size, + // We need to consider both blockGroupLength and the actual size of blockData. + // + // We use the smaller value to determine the size of the ChunkList. + // + // 1. In most cases, blockGroupLength is equal to the size of blockData. + // 2. Occasionally, blockData is not fully filled; if a chunk is empty, + // it is not sent to the DN, resulting in blockData size being smaller than blockGroupLength. + // 3. In cases with 'dirty data', + // if an error occurs when writing to the EC-Stripe (e.g., DN reports Container Closed), + // and the length confirmed with OM is smaller, blockGroupLength may be smaller than blockData size. + long blockDataSize = Math.min(maxBlockData.getSize(), blockGroupLength); + int chunkSize = (int) Math.ceil(((double) blockDataSize / repConfig.getEcChunkSize())); List checksumBlockDataChunks = checksumBlockData.getChunks(); + if (chunkSize > 0) { + checksumBlockDataChunks = checksumBlockData.getChunks().subList(0, chunkSize); + } + + List currentChunks = getContainerBlockData().getChunksList(); Preconditions.checkArgument( currentChunks.size() == checksumBlockDataChunks.size(), @@ -268,7 +298,7 @@ ContainerCommandResponseProto> executePutBlock(boolean close, throw ce; }); } catch (IOException | ExecutionException e) { - throw new IOException(EXCEPTION_MSG + e.toString(), e); + throw new IOException(EXCEPTION_MSG + e, e); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); handleInterruptedException(ex, false); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java index 985b1b80ee9..55beb335dd4 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java @@ -902,4 +902,17 @@ public static HddsProtos.UUID toProtobuf(UUID uuid) { ? Thread.currentThread().getStackTrace() : null; } + + /** + * Logs a warning to report that the class is not closed properly. + */ + public static void reportLeak(Class clazz, String stackTrace, Logger log) { + String warning = String.format("%s is not closed properly", clazz.getSimpleName()); + if (stackTrace != null && LOG.isDebugEnabled()) { + String debugMessage = String.format("%nStackTrace for unclosed instance: %s", + stackTrace); + warning = warning.concat(debugMessage); + } + log.warn(warning); + } } 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/ozone/container/common/helpers/BlockData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java index 8f89be3c118..fa17541d85e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.utils.db.Codec; import org.apache.hadoop.hdds.utils.db.DelegatedCodec; import org.apache.hadoop.hdds.utils.db.Proto3Codec; +import org.apache.hadoop.ozone.OzoneConsts; import java.io.IOException; import java.util.Collections; @@ -296,4 +297,14 @@ public void appendTo(StringBuilder sb) { sb.append(", size=").append(size); sb.append("]"); } + + public long getBlockGroupLength() { + String lenStr = getMetadata() + .get(OzoneConsts.BLOCK_GROUP_LEN_KEY_IN_PUT_BLOCK); + // If we don't have the length, then it indicates a problem with the stripe. + // All replica should carry the length, so if it is not there, we return 0, + // which will cause us to set the length of the block to zero and not + // attempt to reconstruct it. + return (lenStr == null) ? 0 : Long.parseLong(lenStr); + } } diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 4cc32eb336c..88880acbce5 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -2097,7 +2097,7 @@ - ozone.om.ratis.server.leaderelection.pre-vote + ozone.om.ratis.server.leaderelection.pre-vote true OZONE, OM, RATIS, MANAGEMENT Enable/disable OM HA leader election pre-vote phase. @@ -2114,6 +2114,15 @@ + + ozone.om.ratis.server.close.threshold + 60s + OZONE, OM, RATIS + + Raft Server will close if JVM pause longer than the threshold. + + + ozone.om.ratis.snapshot.dir 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/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 13f7ad61502..2169d4a5886 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; @@ -46,6 +50,7 @@ import com.google.protobuf.Descriptors.Descriptor; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.CRLStatusReport; +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; @@ -54,10 +59,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; @@ -117,7 +124,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; @@ -182,7 +189,7 @@ public StateContext(ConfigurationSource conf, crlStatusReport = new AtomicReference<>(); // Certificate Revocation List endpoints = new HashSet<>(); containerActions = new HashMap<>(); - pipelineActions = new HashMap<>(); + pipelineActions = new ConcurrentHashMap<>(); lock = new ReentrantLock(); stateExecutionCount = new AtomicLong(0); threadPoolNotAvailableCount = new AtomicLong(0); @@ -524,47 +531,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); } } @@ -577,34 +553,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); } /** @@ -922,7 +881,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/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 58c8655454b..772381b4c2b 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 @@ -44,6 +44,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; @@ -260,11 +261,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) { @@ -295,11 +299,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); @@ -391,17 +390,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); @@ -430,6 +428,7 @@ private void setRaftSegmentAndWriteBufferSize(RaftProperties properties) { SizeInBytes.valueOf(raftSegmentSize)); RaftServerConfigKeys.Log.setWriteBufferSize(properties, SizeInBytes.valueOf(raftSegmentBufferSize)); + return logAppenderQueueByteLimit; } private void setStateMachineDataConfigurations(RaftProperties properties) { diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisGrpcConfig.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisGrpcConfig.java index 5b283c3a1a3..fde555208b3 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisGrpcConfig.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisGrpcConfig.java @@ -31,23 +31,6 @@ @ConfigGroup(prefix = HDDS_DATANODE_RATIS_PREFIX_KEY + "." + GrpcConfigKeys.PREFIX) public class DatanodeRatisGrpcConfig { - @Config(key = "message.size.max", - defaultValue = "32MB", - type = ConfigType.SIZE, - tags = {OZONE, CLIENT, PERFORMANCE}, - description = "Maximum message size allowed to be received by Grpc " + - "Channel (Server)." - ) - private int maximumMessageSize = 32 * 1024 * 1024; - - public int getMaximumMessageSize() { - return maximumMessageSize; - } - - public void setMaximumMessageSize(int maximumMessageSize) { - this.maximumMessageSize = maximumMessageSize; - } - @Config(key = "flow.control.window", defaultValue = "5MB", type = ConfigType.SIZE, diff --git a/hadoop-hdds/hadoop-dependency-test/pom.xml b/hadoop-hdds/hadoop-dependency-test/pom.xml index c226c184720..308c9dad5ab 100644 --- a/hadoop-hdds/hadoop-dependency-test/pom.xml +++ b/hadoop-hdds/hadoop-dependency-test/pom.xml @@ -67,5 +67,15 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> assertj-core compile + + + org.junit.jupiter + junit-jupiter-engine + + + org.junit.vintage + junit-vintage-engine + test + diff --git a/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObjectUtils.java b/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObjectUtils.java index a7fa827573d..f953eb9870a 100644 --- a/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObjectUtils.java +++ b/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObjectUtils.java @@ -59,12 +59,7 @@ static UncheckedAutoCloseable track(AutoCloseable object) { static void reportLeak(Class clazz, String stackTrace) { ManagedRocksObjectMetrics.INSTANCE.increaseLeakObject(); - String warning = String.format("%s is not closed properly", clazz.getSimpleName()); - if (stackTrace != null && LOG.isDebugEnabled()) { - String debugMessage = String.format("%nStackTrace for unclosed instance: %s", stackTrace); - warning = warning.concat(debugMessage); - } - LOG.warn(warning); + HddsUtils.reportLeak(clazz, stackTrace, LOG); } private static @Nullable StackTraceElement[] getStackTrace() { diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml index c07cf30da7f..4db9ea9191f 100644 --- a/hadoop-hdds/pom.xml +++ b/hadoop-hdds/pom.xml @@ -290,6 +290,30 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + + org.apache.maven.plugins + maven-remote-resources-plugin + + + org.apache.ozone:ozone-dev-support:${ozone.version} + + + + + org.apache.ozone + ozone-dev-support + ${ozone.version} + + + + + + process + + + + diff --git a/hadoop-hdds/server-scm/pom.xml b/hadoop-hdds/server-scm/pom.xml index cbf80b07f2d..219d60ad91a 100644 --- a/hadoop-hdds/server-scm/pom.xml +++ b/hadoop-hdds/server-scm/pom.xml @@ -167,6 +167,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.mockito mockito-junit-jupiter + + jakarta.annotation + jakarta.annotation-api + 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/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java new file mode 100644 index 00000000000..b290b950442 --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java @@ -0,0 +1,318 @@ +/** + * 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.container.balancer; + +import com.google.protobuf.ByteString; +import jakarta.annotation.Nonnull; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.scm.PlacementPolicy; +import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerManager; +import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; +import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException; +import org.apache.hadoop.hdds.scm.container.MockNodeManager; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics; +import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; +import org.apache.hadoop.hdds.scm.exceptions.SCMException; +import org.apache.hadoop.hdds.scm.ha.SCMContext; +import org.apache.hadoop.hdds.scm.ha.SCMService; +import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; +import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManager; +import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManagerImpl; +import org.apache.hadoop.hdds.scm.net.NetworkTopology; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; +import org.apache.hadoop.hdds.server.events.EventPublisher; +import org.mockito.Mockito; + +import java.io.IOException; +import java.time.Clock; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Class for test used for setting up testable StorageContainerManager. + * Provides an access to {@link TestableCluster} and to necessary mocked instances + */ +public final class MockedSCM { + private final StorageContainerManager scm; + private final TestableCluster cluster; + private final MockNodeManager mockNodeManager; + private MockedReplicationManager mockedReplicaManager; + private MoveManager moveManager; + private ContainerManager containerManager; + + private MockedPlacementPolicies mockedPlacementPolicies; + + public MockedSCM(@Nonnull TestableCluster testableCluster) { + scm = mock(StorageContainerManager.class); + cluster = testableCluster; + mockNodeManager = new MockNodeManager(cluster.getDatanodeToContainersMap()); + } + + public void init(@Nonnull ContainerBalancerConfiguration balancerConfig) { + init(balancerConfig, new OzoneConfiguration()); + } + + public void init(@Nonnull ContainerBalancerConfiguration balancerConfig, @Nonnull OzoneConfiguration ozoneCfg) { + ozoneCfg.setFromObject(balancerConfig); + try { + doMock(balancerConfig, ozoneCfg); + } catch (IOException | NodeNotFoundException | TimeoutException e) { + throw new RuntimeException("Can't initialize TestOzoneHDDS: ", e); + } + } + + /** + * Mock some instances that will be used for MockedStorageContainerManager. + */ + private void doMock(@Nonnull ContainerBalancerConfiguration cfg, @Nonnull OzoneConfiguration ozoneCfg) + throws IOException, NodeNotFoundException, TimeoutException { + containerManager = mockContainerManager(cluster); + mockedReplicaManager = MockedReplicationManager.doMock(); + moveManager = mockMoveManager(); + StatefulServiceStateManager stateManager = MockedServiceStateManager.doMock(); + SCMServiceManager scmServiceManager = mockSCMServiceManger(); + + mockedPlacementPolicies = MockedPlacementPolicies.doMock(ozoneCfg, mockNodeManager); + + when(scm.getConfiguration()).then(invocationOnMock -> { + ozoneCfg.setFromObject(cfg); + return ozoneCfg; + }); + when(scm.getMoveManager()).thenReturn(moveManager); + when(scm.getScmNodeManager()).thenReturn(mockNodeManager); + when(scm.getContainerManager()).thenReturn(containerManager); + when(scm.getReplicationManager()).thenReturn(mockedReplicaManager.manager); + when(scm.getContainerPlacementPolicy()).thenReturn(mockedPlacementPolicies.placementPolicy); + when(scm.getPlacementPolicyValidateProxy()).thenReturn(mockedPlacementPolicies.validateProxyPolicy); + when(scm.getSCMServiceManager()).thenReturn(scmServiceManager); + when(scm.getScmContext()).thenReturn(SCMContext.emptyContext()); + when(scm.getClusterMap()).thenReturn(null); + when(scm.getEventQueue()).thenReturn(mock(EventPublisher.class)); + when(scm.getStatefulServiceStateManager()).thenReturn(stateManager); + } + + @Override + public String toString() { + return cluster.toString(); + } + + public @Nonnull ContainerBalancerTask startBalancerTask( + @Nonnull ContainerBalancer containerBalancer, + @Nonnull ContainerBalancerConfiguration config + ) { + ContainerBalancerTask task = new ContainerBalancerTask(scm, 0, containerBalancer, + containerBalancer.getMetrics(), config, false); + task.run(); + return task; + } + + public @Nonnull ContainerBalancerTask startBalancerTask(@Nonnull ContainerBalancerConfiguration config) { + init(config); + return startBalancerTask(new ContainerBalancer(scm), config); + } + + public void enableLegacyReplicationManager() { + mockedReplicaManager.conf.setEnableLegacy(true); + } + + public void disableLegacyReplicationManager() { + mockedReplicaManager.conf.setEnableLegacy(false); + } + + public @Nonnull MoveManager getMoveManager() { + return moveManager; + } + + public @Nonnull ReplicationManager getReplicationManager() { + return mockedReplicaManager.manager; + } + + public @Nonnull MockNodeManager getNodeManager() { + return mockNodeManager; + } + + public @Nonnull StorageContainerManager getStorageContainerManager() { + return scm; + } + + public @Nonnull TestableCluster getCluster() { + return cluster; + } + + public @Nonnull ContainerManager getContainerManager() { + return containerManager; + } + + public @Nonnull PlacementPolicy getPlacementPolicy() { + return mockedPlacementPolicies.placementPolicy; + } + + public @Nonnull PlacementPolicy getEcPlacementPolicy() { + return mockedPlacementPolicies.ecPlacementPolicy; + } + + private static @Nonnull ContainerManager mockContainerManager(@Nonnull TestableCluster cluster) + throws ContainerNotFoundException { + ContainerManager containerManager = mock(ContainerManager.class); + Mockito + .when(containerManager.getContainerReplicas(any(ContainerID.class))) + .thenAnswer(invocationOnMock -> { + ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0]; + return cluster.getCidToReplicasMap().get(cid); + }); + + Mockito + .when(containerManager.getContainer(any(ContainerID.class))) + .thenAnswer(invocationOnMock -> { + ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0]; + return cluster.getCidToInfoMap().get(cid); + }); + + Mockito + .when(containerManager.getContainers()) + .thenReturn(new ArrayList<>(cluster.getCidToInfoMap().values())); + return containerManager; + } + + private static @Nonnull SCMServiceManager mockSCMServiceManger() { + SCMServiceManager scmServiceManager = mock(SCMServiceManager.class); + + Mockito + .doNothing() + .when(scmServiceManager) + .register(Mockito.any(SCMService.class)); + + return scmServiceManager; + } + + private static @Nonnull MoveManager mockMoveManager() + throws NodeNotFoundException, ContainerReplicaNotFoundException, ContainerNotFoundException { + MoveManager moveManager = mock(MoveManager.class); + Mockito + .when(moveManager.move( + any(ContainerID.class), + any(DatanodeDetails.class), + any(DatanodeDetails.class))) + .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED)); + return moveManager; + } + + private static final class MockedReplicationManager { + private final ReplicationManager manager; + private final ReplicationManager.ReplicationManagerConfiguration conf; + + private MockedReplicationManager() { + manager = mock(ReplicationManager.class); + conf = new ReplicationManager.ReplicationManagerConfiguration(); + // Disable LegacyReplicationManager. This means balancer should select RATIS as well as + // EC containers for balancing. Also, MoveManager will be used. + conf.setEnableLegacy(false); + } + + private static @Nonnull MockedReplicationManager doMock() + throws NodeNotFoundException, ContainerNotFoundException, TimeoutException { + MockedReplicationManager mockedManager = new MockedReplicationManager(); + + Mockito + .when(mockedManager.manager.getConfig()) + .thenReturn(mockedManager.conf); + + Mockito + .when(mockedManager.manager.isContainerReplicatingOrDeleting(Mockito.any(ContainerID.class))) + .thenReturn(false); + + Mockito + .when(mockedManager.manager.move( + Mockito.any(ContainerID.class), + Mockito.any(DatanodeDetails.class), + Mockito.any(DatanodeDetails.class))) + .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED)); + + Mockito + .when(mockedManager.manager.getClock()) + .thenReturn(Clock.system(ZoneId.systemDefault())); + + return mockedManager; + } + } + + private static final class MockedServiceStateManager { + private final Map serviceToConfigMap = new HashMap<>(); + private final StatefulServiceStateManager serviceStateManager = Mockito.mock(StatefulServiceStateManagerImpl.class); + + private static @Nonnull StatefulServiceStateManager doMock() throws IOException { + MockedServiceStateManager manager = new MockedServiceStateManager(); + + // When StatefulServiceStateManager#saveConfiguration is called, save to in-memory serviceToConfigMap instead. + Map map = manager.serviceToConfigMap; + StatefulServiceStateManager stateManager = manager.serviceStateManager; + Mockito + .doAnswer(i -> { + map.put(i.getArgument(0, String.class), i.getArgument(1, ByteString.class)); + return null; + }) + .when(stateManager) + .saveConfiguration(Mockito.any(String.class), Mockito.any(ByteString.class)); + + // When StatefulServiceStateManager#readConfiguration is called, read from serviceToConfigMap instead. + Mockito + .when(stateManager.readConfiguration(Mockito.anyString())) + .thenAnswer(i -> map.get(i.getArgument(0, String.class))); + return stateManager; + } + } + + private static final class MockedPlacementPolicies { + private final PlacementPolicy placementPolicy; + private final PlacementPolicy ecPlacementPolicy; + private final PlacementPolicyValidateProxy validateProxyPolicy; + + private MockedPlacementPolicies(@Nonnull PlacementPolicy placementPolicy, @Nonnull PlacementPolicy ecPolicy) { + this.placementPolicy = placementPolicy; + ecPlacementPolicy = ecPolicy; + validateProxyPolicy = new PlacementPolicyValidateProxy(this.placementPolicy, ecPlacementPolicy); + } + + private static @Nonnull MockedPlacementPolicies doMock( + @Nonnull OzoneConfiguration ozoneConfig, + @Nonnull NodeManager nodeManager + ) throws SCMException { + NetworkTopology clusterMap = nodeManager.getClusterNetworkTopologyMap(); + PlacementPolicy policy = ContainerPlacementPolicyFactory.getPolicy( + ozoneConfig, nodeManager, clusterMap, true, SCMContainerPlacementMetrics.create()); + PlacementPolicy ecPolicy = ContainerPlacementPolicyFactory.getECPolicy( + ozoneConfig, nodeManager, clusterMap, true, SCMContainerPlacementMetrics.create()); + return new MockedPlacementPolicies(policy, ecPolicy); + } + } +} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java new file mode 100644 index 00000000000..8f1db615dfe --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java @@ -0,0 +1,186 @@ +/* + * 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.container.balancer; + +import jakarta.annotation.Nonnull; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.event.Level; + +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests for {@link ContainerBalancerTask} moved from {@link TestContainerBalancerTask} to run them on clusters + * with different datanode count. + */ +public class TestContainerBalancerDatanodeNodeLimit { + private static final long STORAGE_UNIT = OzoneConsts.GB; + private static final int DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER = 15; + + @BeforeAll + public static void setup() { + GenericTestUtils.setLogLevel(ContainerBalancerTask.LOG, Level.DEBUG); + } + + private static Stream createMockedSCMs() { + return Stream.of( + Arguments.of(getMockedSCM(4)), + Arguments.of(getMockedSCM(5)), + Arguments.of(getMockedSCM(6)), + Arguments.of(getMockedSCM(7)), + Arguments.of(getMockedSCM(8)), + Arguments.of(getMockedSCM(9)), + Arguments.of(getMockedSCM(10)), + Arguments.of(getMockedSCM(11)), + Arguments.of(getMockedSCM(12)), + Arguments.of(getMockedSCM(13)), + Arguments.of(getMockedSCM(14)), + Arguments.of(getMockedSCM(15)), + Arguments.of(getMockedSCM(17)), + Arguments.of(getMockedSCM(19)), + Arguments.of(getMockedSCM(20)), + Arguments.of(getMockedSCM(30))); + } + + @ParameterizedTest(name = "MockedSCM #{index}: {0}") + @MethodSource("createMockedSCMs") + public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit(@Nonnull MockedSCM mockedSCM) { + ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); + int nodeCount = mockedSCM.getCluster().getNodeCount(); + if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { + config.setMaxDatanodesPercentageToInvolvePerIteration(100); + } + config.setIterations(1); + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); + ContainerBalancerMetrics metrics = task.getMetrics(); + + int maxDatanodePercentage = config.getMaxDatanodesPercentageToInvolvePerIteration(); + int number = maxDatanodePercentage * nodeCount / 100; + int datanodesInvolvedPerIteration = task.getCountDatanodesInvolvedPerIteration(); + assertThat(datanodesInvolvedPerIteration).isGreaterThan(0); + assertThat(datanodesInvolvedPerIteration).isLessThanOrEqualTo(number); + long numDatanodesInvolvedInLatestIteration = metrics.getNumDatanodesInvolvedInLatestIteration(); + assertThat(numDatanodesInvolvedInLatestIteration).isGreaterThan(0); + assertThat(numDatanodesInvolvedInLatestIteration).isLessThanOrEqualTo(number); + } + + @ParameterizedTest(name = "MockedSCM #{index}: {0}") + @MethodSource("createMockedSCMs") + public void balancerShouldObeyMaxSizeEnteringTargetLimit(@Nonnull MockedSCM mockedSCM) { + OzoneConfiguration ozoneConfig = new OzoneConfiguration(); + ozoneConfig.set("ozone.scm.container.size", "1MB"); + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(ozoneConfig); + if (mockedSCM.getCluster().getNodeCount() < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { + config.setMaxDatanodesPercentageToInvolvePerIteration(100); + } + config.setIterations(1); + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + // No containers should be selected when the limit is just 2 MB. + config.setMaxSizeEnteringTarget(2 * OzoneConsts.MB); + + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); + // Container balancer still has unbalanced nodes due to MaxSizeEnteringTarget limit + assertTrue(stillHaveUnbalancedNodes(task)); + // ContainerToSourceMap is empty due to MaxSizeEnteringTarget limit + assertTrue(task.getContainerToSourceMap().isEmpty()); + // SizeScheduledForMoveInLatestIteration equals to 0 because there are no containers was selected + assertEquals(0, task.getSizeScheduledForMoveInLatestIteration()); + + // Some containers should be selected when using default values. + ContainerBalancerConfiguration balancerConfig = balancerConfigByOzoneConfig(new OzoneConfiguration()); + if (mockedSCM.getCluster().getNodeCount() < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { + balancerConfig.setMaxDatanodesPercentageToInvolvePerIteration(100); + } + balancerConfig.setIterations(1); + + task = mockedSCM.startBalancerTask(balancerConfig); + // Balancer should have identified unbalanced nodes. + assertTrue(stillHaveUnbalancedNodes(task)); + // ContainerToSourceMap is not empty due to some containers should be selected + assertFalse(task.getContainerToSourceMap().isEmpty()); + // SizeScheduledForMoveInLatestIteration doesn't equal to 0 because some containers should be selected + assertNotEquals(0, task.getSizeScheduledForMoveInLatestIteration()); + } + + @ParameterizedTest(name = "MockedSCM #{index}: {0}") + @MethodSource("createMockedSCMs") + public void balancerShouldObeyMaxSizeLeavingSourceLimit(@Nonnull MockedSCM mockedSCM) { + OzoneConfiguration ozoneConfig = new OzoneConfiguration(); + ozoneConfig.set("ozone.scm.container.size", "1MB"); + ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(ozoneConfig); + if (mockedSCM.getCluster().getNodeCount() < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { + config.setMaxDatanodesPercentageToInvolvePerIteration(100); + } + config.setIterations(1); + config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); + config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + // No source containers should be selected when the limit is just 2 MB. + config.setMaxSizeLeavingSource(2 * OzoneConsts.MB); + + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); + // Container balancer still has unbalanced nodes due to MaxSizeLeavingSource limit + assertTrue(stillHaveUnbalancedNodes(task)); + // ContainerToSourceMap is empty due to MaxSizeLeavingSource limit + assertTrue(task.getContainerToSourceMap().isEmpty()); + // SizeScheduledForMoveInLatestIteration equals to 0 because there are no containers was selected + assertEquals(0, task.getSizeScheduledForMoveInLatestIteration()); + + // Some containers should be selected when using default values. + ContainerBalancerConfiguration newBalancerConfig = balancerConfigByOzoneConfig(new OzoneConfiguration()); + if (mockedSCM.getCluster().getNodeCount() < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) { + newBalancerConfig.setMaxDatanodesPercentageToInvolvePerIteration(100); + } + newBalancerConfig.setIterations(1); + + task = mockedSCM.startBalancerTask(newBalancerConfig); + // Balancer should have identified unbalanced nodes. + assertTrue(stillHaveUnbalancedNodes(task)); + // ContainerToSourceMap is not empty due to some containers should be selected + assertFalse(task.getContainerToSourceMap().isEmpty()); + // SizeScheduledForMoveInLatestIteration doesn't equal to 0 because some containers should be selected + assertNotEquals(0, task.getSizeScheduledForMoveInLatestIteration()); + } + + private static boolean stillHaveUnbalancedNodes(@Nonnull ContainerBalancerTask task) { + return !task.getUnBalancedNodes().isEmpty(); + } + + public static @Nonnull MockedSCM getMockedSCM(int datanodeCount) { + return new MockedSCM(new TestableCluster(datanodeCount, STORAGE_UNIT)); + } + + private static @Nonnull ContainerBalancerConfiguration balancerConfigByOzoneConfig( + @Nonnull OzoneConfiguration ozoneConfiguration + ) { + return ozoneConfiguration.getObject(ContainerBalancerConfiguration.class); + } +} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java index 56d02dabb5f..f8024ea8e04 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java @@ -325,33 +325,6 @@ public void unBalancedNodesListShouldBeEmptyWhenClusterIsBalanced() Assertions.assertEquals(0, metrics.getNumDatanodesUnbalanced()); } - /** - * ContainerBalancer should not involve more datanodes than the - * maxDatanodesRatioToInvolvePerIteration limit. - */ - @Test - public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit() - throws IllegalContainerBalancerStateException, IOException, - InvalidContainerBalancerConfigurationException, TimeoutException { - int percent = 40; - balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration( - percent); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); - balancerConfiguration.setThreshold(1); - balancerConfiguration.setIterations(1); - startBalancer(balancerConfiguration); - - int number = percent * numberOfNodes / 100; - ContainerBalancerMetrics metrics = containerBalancerTask.getMetrics(); - Assertions.assertFalse( - containerBalancerTask.getCountDatanodesInvolvedPerIteration() > number); - Assertions.assertTrue( - metrics.getNumDatanodesInvolvedInLatestIteration() > 0); - Assertions.assertFalse( - metrics.getNumDatanodesInvolvedInLatestIteration() > number); - stopBalancer(); - } - @Test public void containerBalancerShouldSelectOnlyClosedContainers() throws IllegalContainerBalancerStateException, IOException, @@ -608,86 +581,6 @@ public void balancerShouldNotSelectConfiguredExcludeContainers() } } - @Test - public void balancerShouldObeyMaxSizeEnteringTargetLimit() - throws IllegalContainerBalancerStateException, IOException, - InvalidContainerBalancerConfigurationException, TimeoutException { - conf.set("ozone.scm.container.size", "1MB"); - balancerConfiguration = - conf.getObject(ContainerBalancerConfiguration.class); - balancerConfiguration.setThreshold(10); - balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); - - // no containers should be selected when the limit is just 2 MB - balancerConfiguration.setMaxSizeEnteringTarget(2 * OzoneConsts.MB); - startBalancer(balancerConfiguration); - - Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes() - .isEmpty()); - Assertions.assertTrue(containerBalancerTask.getContainerToSourceMap() - .isEmpty()); - stopBalancer(); - - // some containers should be selected when using default values - OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); - ContainerBalancerConfiguration cbc = ozoneConfiguration. - getObject(ContainerBalancerConfiguration.class); - cbc.setBalancingInterval(1); - ContainerBalancer sb = new ContainerBalancer(scm); - containerBalancerTask = new ContainerBalancerTask(scm, 0, sb, - sb.getMetrics(), cbc, false); - containerBalancerTask.run(); - - stopBalancer(); - // balancer should have identified unbalanced nodes - Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes() - .isEmpty()); - Assertions.assertFalse(containerBalancerTask.getContainerToSourceMap() - .isEmpty()); - } - - @Test - public void balancerShouldObeyMaxSizeLeavingSourceLimit() - throws IllegalContainerBalancerStateException, IOException, - InvalidContainerBalancerConfigurationException, TimeoutException { - conf.set("ozone.scm.container.size", "1MB"); - balancerConfiguration = - conf.getObject(ContainerBalancerConfiguration.class); - balancerConfiguration.setThreshold(10); - balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); - - // no source containers should be selected when the limit is just 2 MB - balancerConfiguration.setMaxSizeLeavingSource(2 * OzoneConsts.MB); - startBalancer(balancerConfiguration); - - Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes() - .isEmpty()); - Assertions.assertTrue(containerBalancerTask.getContainerToSourceMap() - .isEmpty()); - stopBalancer(); - - // some containers should be selected when using default values - OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); - ContainerBalancerConfiguration cbc = ozoneConfiguration. - getObject(ContainerBalancerConfiguration.class); - cbc.setBalancingInterval(1); - ContainerBalancer sb = new ContainerBalancer(scm); - containerBalancerTask = new ContainerBalancerTask(scm, 0, sb, - sb.getMetrics(), cbc, false); - containerBalancerTask.run(); - - stopBalancer(); - // balancer should have identified unbalanced nodes - Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes() - .isEmpty()); - Assertions.assertFalse(containerBalancerTask.getContainerToSourceMap() - .isEmpty()); - Assertions.assertTrue(0 != - containerBalancerTask.getSizeScheduledForMoveInLatestIteration()); - } - @Test public void testMetrics() throws IllegalContainerBalancerStateException, IOException, diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestableCluster.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestableCluster.java new file mode 100644 index 00000000000..60ee45535f0 --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestableCluster.java @@ -0,0 +1,253 @@ +/** + * 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.container.balancer; + +import jakarta.annotation.Nonnull; +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.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; +import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; + +/** + * Class is used for creating test cluster with a required number of datanodes. + * 1. Fill the cluster by generating some data. + * 2. Nodes in the cluster have utilization values determined by generateUtilization method. + */ +public final class TestableCluster { + static final ThreadLocalRandom RANDOM = ThreadLocalRandom.current(); + private static final Logger LOG = LoggerFactory.getLogger(TestableCluster.class); + private final int nodeCount; + private final double[] nodeUtilizationList; + private final DatanodeUsageInfo[] nodesInCluster; + private final Map cidToInfoMap = new HashMap<>(); + private final Map> cidToReplicasMap = new HashMap<>(); + private final Map> dnUsageToContainersMap = new HashMap<>(); + private final double averageUtilization; + + TestableCluster(int numberOfNodes, long storageUnit) { + nodeCount = numberOfNodes; + nodeUtilizationList = createUtilizationList(nodeCount); + nodesInCluster = new DatanodeUsageInfo[nodeCount]; + + generateData(storageUnit); + createReplicasForContainers(); + long clusterCapacity = 0, clusterUsedSpace = 0; + + // For each node utilization, calculate that datanode's used space and capacity. + for (int i = 0; i < nodeUtilizationList.length; i++) { + Set containerIDSet = dnUsageToContainersMap.get(nodesInCluster[i]); + long datanodeUsedSpace = 0; + for (ContainerID containerID : containerIDSet) { + datanodeUsedSpace += cidToInfoMap.get(containerID).getUsedBytes(); + } + // Use node utilization and used space to determine node capacity. + long datanodeCapacity = (nodeUtilizationList[i] == 0) + ? storageUnit * RANDOM.nextInt(10, 60) + : (long) (datanodeUsedSpace / nodeUtilizationList[i]); + + SCMNodeStat stat = new SCMNodeStat(datanodeCapacity, datanodeUsedSpace, + datanodeCapacity - datanodeUsedSpace, 0, + datanodeCapacity - datanodeUsedSpace - 1); + nodesInCluster[i].setScmNodeStat(stat); + clusterUsedSpace += datanodeUsedSpace; + clusterCapacity += datanodeCapacity; + } + + averageUtilization = (double) clusterUsedSpace / clusterCapacity; + } + + @Override + public String toString() { + return "cluster of " + nodeCount + " nodes"; + } + + @Nonnull Map> getDatanodeToContainersMap() { + return dnUsageToContainersMap; + } + + @Nonnull Map getCidToInfoMap() { + return cidToInfoMap; + } + + int getNodeCount() { + return nodeCount; + } + + double getAverageUtilization() { + return averageUtilization; + } + + @Nonnull DatanodeUsageInfo[] getNodesInCluster() { + return nodesInCluster; + } + + double[] getNodeUtilizationList() { + return nodeUtilizationList; + } + + @Nonnull Map> getCidToReplicasMap() { + return cidToReplicasMap; + } + + /** + * Determines unBalanced nodes, that is, over and under utilized nodes, + * according to the generated utilization values for nodes and the threshold. + * + * @param threshold a percentage in the range 0 to 100 + * @return list of DatanodeUsageInfo containing the expected(correct) unBalanced nodes. + */ + @Nonnull List getUnBalancedNodes(double threshold) { + threshold /= 100; + double lowerLimit = averageUtilization - threshold; + double upperLimit = averageUtilization + threshold; + + // Use node utilization to determine over and under utilized nodes. + List expectedUnBalancedNodes = new ArrayList<>(); + for (int i = 0; i < nodeCount; i++) { + double nodeUtilization = nodeUtilizationList[i]; + if (nodeUtilization < lowerLimit || nodeUtilization > upperLimit) { + expectedUnBalancedNodes.add(nodesInCluster[i]); + } + } + return expectedUnBalancedNodes; + } + + /** + * Create some datanodes and containers for each node. + */ + private void generateData(long storageUnit) { + // Create datanodes and add containers to them. + for (int i = 0; i < nodeCount; i++) { + DatanodeUsageInfo usageInfo = + new DatanodeUsageInfo(MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat()); + nodesInCluster[i] = usageInfo; + + // Create containers with varying used space. + Set containerIDSet = new HashSet<>(); + int sizeMultiple = 0; + for (int j = 0; j < i; j++) { + sizeMultiple %= 5; + sizeMultiple++; + ContainerInfo container = createContainer((long) i * i + j, storageUnit * sizeMultiple); + + cidToInfoMap.put(container.containerID(), container); + containerIDSet.add(container.containerID()); + + // Create initial replica for this container and add it. + Set containerReplicaSet = new HashSet<>(); + containerReplicaSet.add( + createReplica(container.containerID(), usageInfo.getDatanodeDetails(), container.getUsedBytes())); + cidToReplicasMap.put(container.containerID(), containerReplicaSet); + } + dnUsageToContainersMap.put(usageInfo, containerIDSet); + } + } + + private @Nonnull ContainerInfo createContainer(long id, long usedBytes) { + ContainerInfo.Builder builder = new ContainerInfo.Builder() + .setContainerID(id) + .setState(HddsProtos.LifeCycleState.CLOSED) + .setOwner("TestContainerBalancer") + .setUsedBytes(usedBytes); + + // Make it a RATIS container if id is even, else make it an EC container. + ReplicationConfig config = (id % 2 == 0) + ? RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE) + : new ECReplicationConfig(3, 2); + + builder.setReplicationConfig(config); + return builder.build(); + } + + /** + * Create the required number of replicas for each container. Note that one replica already exists and + * nodes with utilization value 0 should not have any replicas. + */ + private void createReplicasForContainers() { + for (ContainerInfo container : cidToInfoMap.values()) { + // One replica already exists; create the remaining ones. + ReplicationConfig replicationConfig = container.getReplicationConfig(); + ContainerID key = container.containerID(); + for (int i = 0; i < replicationConfig.getRequiredNodes() - 1; i++) { + // Randomly pick a datanode for this replica. + int dnIndex = RANDOM.nextInt(0, nodeCount); + // Don't put replicas in DNs that are supposed to have 0 utilization. + if (Math.abs(nodeUtilizationList[dnIndex] - 0.0d) > 0.00001) { + DatanodeDetails node = nodesInCluster[dnIndex].getDatanodeDetails(); + Set replicas = cidToReplicasMap.get(key); + replicas.add(createReplica(key, node, container.getUsedBytes())); + cidToReplicasMap.put(key, replicas); + dnUsageToContainersMap.get(nodesInCluster[dnIndex]).add(key); + } + } + } + } + + /** + * Generates a range of equally spaced utilization(that is, used / capacity) values from 0 to 1. + * + * @param count Number of values to generate. Count must be greater than or equal to 1. + * @return double array of node utilization values + * @throws IllegalArgumentException If the value of the parameter count is less than 1. + */ + private static double[] createUtilizationList(int count) throws IllegalArgumentException { + if (count < 1) { + LOG.warn("The value of argument count is {}. However, count must be greater than 0.", count); + throw new IllegalArgumentException(); + } + double[] result = new double[count]; + for (int i = 0; i < count; i++) { + result[i] = (i / (double) count); + } + return result; + } + + private @Nonnull ContainerReplica createReplica( + @Nonnull ContainerID containerID, + @Nonnull DatanodeDetails datanodeDetails, + long usedBytes + ) { + return ContainerReplica.newBuilder() + .setContainerID(containerID) + .setContainerState(ContainerReplicaProto.State.CLOSED) + .setDatanodeDetails(datanodeDetails) + .setOriginNodeId(datanodeDetails.getUuid()) + .setSequenceId(1000L) + .setBytesUsed(usedBytes) + .build(); + } +} diff --git a/hadoop-hdds/test-utils/src/main/java/org/apache/ozone/test/GenericTestUtils.java b/hadoop-hdds/test-utils/src/main/java/org/apache/ozone/test/GenericTestUtils.java index 2be39ea9b9c..9072bffbb0a 100644 --- a/hadoop-hdds/test-utils/src/main/java/org/apache/ozone/test/GenericTestUtils.java +++ b/hadoop-hdds/test-utils/src/main/java/org/apache/ozone/test/GenericTestUtils.java @@ -27,6 +27,7 @@ import java.io.UnsupportedEncodingException; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.TimeoutException; import com.google.common.base.Preconditions; @@ -231,6 +232,20 @@ public static void waitFor(BooleanSupplier check, int checkEveryMillis, } } + public static T assertThrows( + Class expectedType, + Callable func) { + return Assertions.assertThrows(expectedType, () -> { + final AutoCloseable closeable = func.call(); + try { + if (closeable != null) { + closeable.close(); + } + } catch (Exception ignored) { + } + }); + } + /** * @deprecated use sl4fj based version */ diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java index 3a63a593469..8bd648545d4 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java @@ -26,6 +26,7 @@ import java.io.IOException; import com.google.common.annotations.VisibleForTesting; +import org.apache.ratis.util.UncheckedAutoCloseable; /** * OzoneClient connects to Ozone Cluster and @@ -76,6 +77,7 @@ public class OzoneClient implements Closeable { private final ClientProtocol proxy; private final ObjectStore objectStore; private ConfigurationSource conf; + private final UncheckedAutoCloseable leakTracker = OzoneClientFactory.track(this); /** * Creates a new OzoneClient object, generally constructed @@ -119,7 +121,11 @@ public ConfigurationSource getConfiguration() { */ @Override public void close() throws IOException { - proxy.close(); + try { + proxy.close(); + } finally { + leakTracker.close(); + } } /** diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java index 44239aafceb..5fc8e3a9345 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java @@ -23,9 +23,11 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.MutableConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.utils.LeakDetector; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneConsts; @@ -34,13 +36,17 @@ import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import org.apache.hadoop.security.token.Token; -import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY; +import org.apache.ratis.util.UncheckedAutoCloseable; + +import com.google.common.base.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY; + /** * Factory class to create OzoneClients. */ @@ -54,6 +60,21 @@ public final class OzoneClientFactory { */ private OzoneClientFactory() { } + private static final LeakDetector OZONE_CLIENT_LEAK_DETECTOR = + new LeakDetector("OzoneClientObject"); + + public static UncheckedAutoCloseable track(AutoCloseable object) { + final Class clazz = object.getClass(); + final StackTraceElement[] stackTrace = HddsUtils.getStackTrace(LOG); + return OZONE_CLIENT_LEAK_DETECTOR.track(object, + () -> HddsUtils.reportLeak(clazz, + HddsUtils.formatStackTrace(stackTrace, 4), LOG)); + } + + public static Logger getLogger() { + return LOG; + } + /** * Constructs and return an OzoneClient with default configuration. diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java index 519edf20f13..191b158fe9d 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java @@ -247,6 +247,10 @@ private OMConfigKeys() { public static final boolean OZONE_OM_RATIS_SERVER_ELECTION_PRE_VOTE_DEFAULT = true; + public static final String OZONE_OM_RATIS_SERVER_CLOSE_THRESHOLD_KEY = + "ozone.om.ratis.server.close.threshold"; + public static final TimeDuration OZONE_OM_RATIS_SERVER_CLOSE_THRESHOLD_DEFAULT = + TimeDuration.valueOf(60, TimeUnit.SECONDS); // OM SnapshotProvider configurations public static final String OZONE_OM_RATIS_SNAPSHOT_DIR = diff --git a/hadoop-ozone/dist/src/main/assemblies/ozone-src.xml b/hadoop-ozone/dist/src/main/assemblies/ozone-src.xml index 3c97d3add76..3450b387393 100644 --- a/hadoop-ozone/dist/src/main/assemblies/ozone-src.xml +++ b/hadoop-ozone/dist/src/main/assemblies/ozone-src.xml @@ -67,6 +67,13 @@ dev-support true + + **/.classpath + **/.project + **/.settings + **/*.iml + **/target/** + hadoop-hdds diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/.env b/hadoop-ozone/dist/src/main/compose/ozone-balancer/.env new file mode 100644 index 00000000000..0e99fab82fd --- /dev/null +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/.env @@ -0,0 +1,20 @@ +# 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. + +HDDS_VERSION=${hdds.version} +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} +OZONE_RUNNER_IMAGE=apache/ozone-runner +OZONE_OPTS= \ No newline at end of file diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-compose.yaml new file mode 100644 index 00000000000..dc6bae7822e --- /dev/null +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-compose.yaml @@ -0,0 +1,179 @@ +# 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. + +version: "3.8" + +# reusable fragments (see https://docs.docker.com/compose/compose-file/#extension-fields) +x-common-config: + &common-config + image: ${OZONE_RUNNER_IMAGE}:${OZONE_RUNNER_VERSION} + volumes: + - ../..:/opt/hadoop + env_file: + - docker-config + +x-replication: + &replication + OZONE-SITE.XML_ozone.server.default.replication: ${OZONE_REPLICATION_FACTOR:-3} + +services: + datanode1: + <<: *common-config + ports: + - 19864 + - 9882 + environment: + <<: *replication + command: ["ozone","datanode"] + volumes: + - tmpfs1:/data + - ../..:/opt/hadoop + datanode2: + <<: *common-config + ports: + - 19864 + - 9882 + environment: + <<: *replication + command: [ "ozone","datanode" ] + volumes: + - tmpfs2:/data + - ../..:/opt/hadoop + datanode3: + <<: *common-config + ports: + - 19864 + - 9882 + environment: + <<: *replication + command: [ "ozone","datanode" ] + volumes: + - tmpfs3:/data + - ../..:/opt/hadoop + datanode4: + <<: *common-config + ports: + - 19864 + - 9882 + environment: + <<: *replication + command: [ "ozone","datanode" ] + volumes: + - tmpfs4:/data + - ../..:/opt/hadoop + om1: + <<: *common-config + environment: + WAITFOR: scm3:9894 + ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION + <<: *replication + ports: + - 9874:9874 + - 9862 + hostname: om1 + command: ["ozone","om"] + om2: + <<: *common-config + environment: + WAITFOR: scm3:9894 + ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION + <<: *replication + ports: + - 9874 + - 9862 + hostname: om2 + command: ["ozone","om"] + om3: + <<: *common-config + environment: + WAITFOR: scm3:9894 + ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION + <<: *replication + ports: + - 9874 + - 9862 + hostname: om3 + command: ["ozone","om"] + scm1: + <<: *common-config + ports: + - 9876:9876 + environment: + ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION + OZONE-SITE.XML_hdds.scm.safemode.min.datanode: ${OZONE_SAFEMODE_MIN_DATANODES:-1} + <<: *replication + command: ["ozone","scm"] + scm2: + <<: *common-config + ports: + - 9876 + environment: + WAITFOR: scm1:9894 + ENSURE_SCM_BOOTSTRAPPED: /data/metadata/scm/current/VERSION + OZONE-SITE.XML_hdds.scm.safemode.min.datanode: ${OZONE_SAFEMODE_MIN_DATANODES:-1} + <<: *replication + command: ["ozone","scm"] + scm3: + <<: *common-config + ports: + - 9876 + environment: + WAITFOR: scm2:9894 + ENSURE_SCM_BOOTSTRAPPED: /data/metadata/scm/current/VERSION + OZONE-SITE.XML_hdds.scm.safemode.min.datanode: ${OZONE_SAFEMODE_MIN_DATANODES:-1} + <<: *replication + command: ["ozone","scm"] + httpfs: + <<: *common-config + environment: + OZONE-SITE.XML_hdds.scm.safemode.min.datanode: ${OZONE_SAFEMODE_MIN_DATANODES:-1} + <<: *replication + ports: + - 14000:14000 + command: [ "ozone","httpfs" ] + s3g: + <<: *common-config + environment: + OZONE_OPTS: + <<: *replication + ports: + - 9878:9878 + command: ["ozone","s3g"] +volumes: + tmpfs1: + driver: local + driver_opts: + o: "size=1g,uid=1000" + device: tmpfs + type: tmpfs + tmpfs2: + driver: local + driver_opts: + o: "size=1g,uid=2000" + device: tmpfs + type: tmpfs + tmpfs3: + driver: local + driver_opts: + o: "size=1g,uid=3000" + device: tmpfs + type: tmpfs + tmpfs4: + driver: local + driver_opts: + o: "size=1g,uid=4000" + device: tmpfs + type: tmpfs diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config new file mode 100644 index 00000000000..60e8afe6e1f --- /dev/null +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config @@ -0,0 +1,62 @@ +# 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. + +# For HttpFS service it is required to enable proxying users. +CORE-SITE.XML_hadoop.proxyuser.hadoop.hosts=* +CORE-SITE.XML_hadoop.proxyuser.hadoop.groups=* + +CORE-SITE.XML_fs.defaultFS=ofs://om/ +CORE-SITE.XML_fs.trash.interval=1 + +OZONE-SITE.XML_ozone.om.service.ids=om +OZONE-SITE.XML_ozone.om.nodes.om=om1,om2,om3 +OZONE-SITE.XML_ozone.om.address.om.om1=om1 +OZONE-SITE.XML_ozone.om.address.om.om2=om2 +OZONE-SITE.XML_ozone.om.address.om.om3=om3 +OZONE-SITE.XML_ozone.om.ratis.enable=true + +OZONE-SITE.XML_ozone.scm.service.ids=scmservice +OZONE-SITE.XML_ozone.scm.nodes.scmservice=scm1,scm2,scm3 +OZONE-SITE.XML_ozone.scm.address.scmservice.scm1=scm1 +OZONE-SITE.XML_ozone.scm.address.scmservice.scm2=scm2 +OZONE-SITE.XML_ozone.scm.address.scmservice.scm3=scm3 +OZONE-SITE.XML_ozone.scm.ratis.enable=true +OZONE-SITE.XML_ozone.scm.datanode.id.dir=/data +OZONE-SITE.XML_ozone.scm.container.size=100MB +OZONE-SITE.XML_ozone.scm.block.size=20MB +OZONE-SITE.XML_ozone.scm.datanode.ratis.volume.free-space.min=10MB +OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata +OZONE-SITE.XML_hdds.node.report.interval=20s +OZONE-SITE.XML_hdds.heartbeat.interval=20s +OZONE-SITE.XML_hdds.datanode.du.refresh.period=20s +OZONE-SITE.XML_hdds.datanode.dir=/data/hdds +OZONE-SITE.XML_hdds.datanode.volume.min.free.space=100MB +OZONE-SITE.XML_ozone.scm.pipeline.creation.auto.factor.one=false +OZONE-SITE.XML_ozone.datanode.pipeline.limit=1 +OZONE-SITE.XML_hdds.scmclient.max.retry.timeout=30s +OZONE-SITE.XML_ozone.scm.primordial.node.id=scm1 +OZONE-SITE.XML_hdds.container.report.interval=30s +OZONE-SITE.XML_ozone.om.s3.grpc.server_enabled=true +OZONE-SITE.XML_ozone.recon.db.dir=/data/metadata/recon +OZONE-SITE.XML_ozone.recon.address=recon:9891 +OZONE-SITE.XML_ozone.recon.http-address=0.0.0.0:9888 +OZONE-SITE.XML_ozone.recon.https-address=0.0.0.0:9889 +OZONE-SITE.XML_dfs.container.ratis.datastream.enabled=true + +OZONE_CONF_DIR=/etc/hadoop +OZONE_LOG_DIR=/var/log/hadoop + +no_proxy=om1,om2,om3,scm,s3g,recon,kdc,localhost,127.0.0.1 diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/test.sh b/hadoop-ozone/dist/src/main/compose/ozone-balancer/test.sh new file mode 100644 index 00000000000..e79979877ba --- /dev/null +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/test.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash +# 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. + +#suite:balancer + +COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" +export COMPOSE_DIR +export OM_SERVICE_ID="om" +export OM=om1 +export SCM=scm1 +export OZONE_REPLICATION_FACTOR=3 + +# shellcheck source=/dev/null +source "$COMPOSE_DIR/../testlib.sh" + +# We need 4 dataNodes in this tests +start_docker_env 4 + +execute_robot_test ${OM} balancer/testBalancer.robot diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot new file mode 100644 index 00000000000..6e2fb9d85a5 --- /dev/null +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -0,0 +1,143 @@ +# 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. + +*** Settings *** +Documentation Smoketest ozone cluster startup +Library OperatingSystem +Library Collections +Resource ../commonlib.robot +Resource ../ozone-lib/shell.robot + +Test Timeout 20 minutes + +*** Variables *** +${SECURITY_ENABLED} false +${HOST} datanode1 +${VOLUME} volume1 +${BUCKET} bucket1 +${SIZE} 104857600 + + +** Keywords *** +Prepare For Tests + Execute dd if=/dev/urandom of=/tmp/100mb bs=1048576 count=100 + Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab + Execute ozone sh volume create /${VOLUME} + Execute ozone sh bucket create /${VOLUME}/${BUCKET} + + +Datanode In Maintenance Mode + ${result} = Execute ozone admin datanode maintenance ${HOST} + Should Contain ${result} Entering maintenance mode on datanode + ${result} = Execute ozone admin datanode list | grep "Operational State:*" + Wait Until Keyword Succeeds 30sec 5sec Should contain ${result} ENTERING_MAINTENANCE + Wait Until Keyword Succeeds 3min 10sec Related pipelines are closed + Sleep 60000ms + +Related pipelines are closed + ${result} = Execute ozone admin datanode list | awk -v RS= '{$1=$1}1'|grep MAINT | sed -e 's/^.*pipelines: \\(.*\\)$/\\1/' + Should Contain Any ${result} CLOSED No related pipelines or the node is not in Healthy state. + +Datanode Recommission + ${result} = Execute ozone admin datanode recommission ${HOST} + Should Contain ${result} Started recommissioning datanode + Wait Until Keyword Succeeds 1min 10sec Datanode Recommission is Finished + Sleep 300000ms + +Datanode Recommission is Finished + ${result} = Execute ozone admin datanode list | grep "Operational State:*" + Should Not Contain ${result} ENTERING_MAINTENANCE + +Run Container Balancer + ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 + Should Contain ${result} Container Balancer started successfully. + ${result} = Execute ozone admin containerbalancer status + Should Contain ${result} ContainerBalancer is Running. + Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running + Sleep 60000ms + +ContainerBalancer is Not Running + ${result} = Execute ozone admin containerbalancer status + Should contain ${result} ContainerBalancer is Not Running. + +Create Multiple Keys + [arguments] ${NUM_KEYS} + ${file} = Set Variable /tmp/100mb + FOR ${INDEX} IN RANGE ${NUM_KEYS} + ${fileName} = Set Variable file-${INDEX}.txt + ${key} = Set Variable /${VOLUME}/${BUCKET}/${fileName} + LOG ${fileName} + Create Key ${key} ${file} + Key Should Match Local File ${key} ${file} + END + +Datanode Usageinfo + [arguments] ${uuid} + ${result} = Execute ozone admin datanode usageinfo --uuid=${uuid} + Should Contain ${result} Ozone Used + +Get Uuid + ${result} = Execute ozone admin datanode list | awk -v RS= '{$1=$1}1'| grep ${HOST} | sed -e 's/Datanode: //'|sed -e 's/ .*$//' + [return] ${result} + +Close All Containers + FOR ${INDEX} IN RANGE 15 + ${container} = Execute ozone admin container list --state OPEN | jq -r 'select(.replicationConfig.replicationFactor == "THREE") | .containerID' | head -1 + EXIT FOR LOOP IF "${container}" == "" + Execute ozone admin container close "${container}" + ${output} = Execute ozone admin container info "${container}" + Should contain ${output} CLOS + END + Wait until keyword succeeds 3min 10sec All container is closed + +All container is closed + ${output} = Execute ozone admin container list --state OPEN + Should Be Empty ${output} + +Get Datanode Ozone Used Bytes Info + [arguments] ${uuid} + ${output} = Execute export DATANODES=$(ozone admin datanode list --json) && for datanode in $(echo "$\{DATANODES\}" | jq -r '.[].datanodeDetails.uuid'); do ozone admin datanode usageinfo --uuid=$\{datanode\} --json | jq '{(.[0].datanodeDetails.uuid) : .[0].ozoneUsed}'; done | jq -s add + ${result} = Execute echo '${output}' | jq '. | to_entries | .[] | select(.key == "${uuid}") | .value' + [return] ${result} + +** Test Cases *** +Verify Container Balancer for RATIS containers + Prepare For Tests + + Datanode In Maintenance Mode + + ${uuid} = Get Uuid + Datanode Usageinfo ${uuid} + + Create Multiple Keys 3 + + Close All Containers + + ${datanodeOzoneUsedBytesInfo} = Get Datanode Ozone Used Bytes Info ${uuid} + Should Be True ${datanodeOzoneUsedBytesInfo} < ${SIZE} + + Datanode Recommission + + Run Container Balancer + + ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} = Get Datanode Ozone Used Bytes Info ${uuid} + Should Not Be Equal As Integers ${datanodeOzoneUsedBytesInfo} ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} + Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} < ${SIZE} * 3.5 + Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} > ${SIZE} * 3 + + + + + diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java index a83c7bf16a2..f088beb362e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java @@ -388,7 +388,7 @@ public void testCreateWithInvalidPaths() throws Exception { } private void checkInvalidPath(Path path) { - InvalidPathException pathException = assertThrows( + InvalidPathException pathException = GenericTestUtils.assertThrows( InvalidPathException.class, () -> fs.create(path, false) ); assertTrue(pathException.getMessage().contains("Invalid path Name")); @@ -1703,12 +1703,14 @@ public void testLoopInLinkBuckets() throws Exception { String rootPath = String.format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, linkBucket1Name, linksVolume); - try { - FileSystem.get(URI.create(rootPath), cluster.getConf()); - fail("Should throw Exception due to loop in Link Buckets"); + try (FileSystem fileSystem = FileSystem.get(URI.create(rootPath), + cluster.getConf())) { + fail("Should throw Exception due to loop in Link Buckets" + + " while initialising fs with URI " + fileSystem.getUri()); } catch (OMException oe) { // Expected exception - assertEquals(OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS, oe.getResult()); + assertEquals(OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS, + oe.getResult()); } finally { volume.deleteBucket(linkBucket1Name); volume.deleteBucket(linkBucket2Name); @@ -1726,13 +1728,17 @@ public void testLoopInLinkBuckets() throws Exception { String rootPath2 = String.format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, danglingLinkBucketName, linksVolume); + FileSystem fileSystem = null; try { - FileSystem.get(URI.create(rootPath2), cluster.getConf()); + fileSystem = FileSystem.get(URI.create(rootPath2), cluster.getConf()); } catch (OMException oe) { // Expected exception fail("Should not throw Exception and show orphan buckets"); } finally { volume.deleteBucket(danglingLinkBucketName); + if (fileSystem != null) { + fileSystem.close(); + } } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/OzoneFileSystemTests.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/OzoneFileSystemTests.java index 64ce7c148c5..11bc1ec408f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/OzoneFileSystemTests.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/OzoneFileSystemTests.java @@ -62,15 +62,17 @@ public static void listStatusIteratorOnPageSize(OzoneConfiguration conf, URI uri = FileSystem.getDefaultUri(config); config.setBoolean( String.format("fs.%s.impl.disable.cache", uri.getScheme()), true); - FileSystem subject = FileSystem.get(uri, config); - Path dir = new Path(Objects.requireNonNull(rootPath), "listStatusIterator"); - try { - Set paths = new TreeSet<>(); - for (int dirCount : dirCounts) { - listStatusIterator(subject, dir, paths, dirCount); + try (FileSystem subject = FileSystem.get(uri, config)) { + Path dir = new Path(Objects.requireNonNull(rootPath), + "listStatusIterator"); + try { + Set paths = new TreeSet<>(); + for (int dirCount : dirCounts) { + listStatusIterator(subject, dir, paths, dirCount); + } + } finally { + subject.delete(dir, true); } - } finally { - subject.delete(dir, true); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java index d9d347dac3e..5b9024ca716 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java @@ -125,8 +125,16 @@ void teardown() throws IOException { void fileSystemWithUnsupportedDefaultBucketLayout(String layout) { OzoneConfiguration conf = configWithDefaultBucketLayout(layout); - OMException e = assertThrows(OMException.class, - () -> FileSystem.newInstance(conf)); + OMException e = assertThrows(OMException.class, () -> { + FileSystem fileSystem = null; + try { + fileSystem = FileSystem.newInstance(conf); + } finally { + if (fileSystem != null) { + fileSystem.close(); + } + } + }); assertThat(e.getMessage(), containsString(ERROR_MAP.get(layout))); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java index a35e2198c11..a0eafc4fdf3 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java @@ -61,6 +61,7 @@ import org.apache.hadoop.ozone.client.OzoneKeyDetails; import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.SecretKeyTestClient; +import org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry; import org.apache.hadoop.ozone.client.io.InsufficientLocationsException; import org.apache.hadoop.ozone.client.io.KeyOutputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; @@ -85,6 +86,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; @@ -101,6 +103,7 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -114,6 +117,8 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto.READ; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto.WRITE; import static org.apache.hadoop.ozone.container.ContainerTestHelper.newWriteChunkRequestBuilder; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.params.provider.Arguments.arguments; /** * This class tests container commands on EC containers. @@ -612,30 +617,33 @@ private static byte[] getBytesWith(int singleDigitNumber, int total) { @ParameterizedTest @MethodSource("recoverableMissingIndexes") - void testECReconstructionCoordinatorWith(List missingIndexes) + void testECReconstructionCoordinatorWith(List missingIndexes, boolean triggerRetry) throws Exception { - testECReconstructionCoordinator(missingIndexes, 3); + testECReconstructionCoordinator(missingIndexes, 3, triggerRetry); } @ParameterizedTest @MethodSource("recoverableMissingIndexes") - void testECReconstructionCoordinatorWithPartialStripe(List missingIndexes) - throws Exception { - testECReconstructionCoordinator(missingIndexes, 1); + void testECReconstructionCoordinatorWithPartialStripe(List missingIndexes, + boolean triggerRetry) throws Exception { + testECReconstructionCoordinator(missingIndexes, 1, triggerRetry); } @ParameterizedTest @MethodSource("recoverableMissingIndexes") - void testECReconstructionCoordinatorWithFullAndPartialStripe(List missingIndexes) - throws Exception { - testECReconstructionCoordinator(missingIndexes, 4); + void testECReconstructionCoordinatorWithFullAndPartialStripe(List missingIndexes, + boolean triggerRetry) throws Exception { + testECReconstructionCoordinator(missingIndexes, 4, triggerRetry); } - static Stream> recoverableMissingIndexes() { - return Stream - .concat(IntStream.rangeClosed(1, 5).mapToObj(ImmutableList::of), Stream - .of(ImmutableList.of(2, 3), ImmutableList.of(2, 4), - ImmutableList.of(3, 5), ImmutableList.of(4, 5))); + static Stream recoverableMissingIndexes() { + Stream args = IntStream.rangeClosed(1, 5).mapToObj(i -> arguments(ImmutableList.of(i), true)); + Stream args1 = IntStream.rangeClosed(1, 5).mapToObj(i -> arguments(ImmutableList.of(i), false)); + Stream args2 = Stream.of(arguments(ImmutableList.of(2, 3), true), + arguments(ImmutableList.of(2, 4), true), arguments(ImmutableList.of(3, 5), true)); + Stream args3 = Stream.of(arguments(ImmutableList.of(2, 3), false), + arguments(ImmutableList.of(2, 4), false), arguments(ImmutableList.of(3, 5), false)); + return Stream.concat(Stream.concat(args, args1), Stream.concat(args2, args3)); } /** @@ -646,7 +654,7 @@ static Stream> recoverableMissingIndexes() { public void testECReconstructionCoordinatorWithMissingIndexes135() { InsufficientLocationsException exception = Assert.assertThrows(InsufficientLocationsException.class, () -> { - testECReconstructionCoordinator(ImmutableList.of(1, 3, 5), 3); + testECReconstructionCoordinator(ImmutableList.of(1, 3, 5), 3, false); }); String expectedMessage = @@ -657,7 +665,7 @@ public void testECReconstructionCoordinatorWithMissingIndexes135() { } private void testECReconstructionCoordinator(List missingIndexes, - int numInputChunks) throws Exception { + int numInputChunks, boolean triggerRetry) throws Exception { ObjectStore objectStore = rpcClient.getObjectStore(); String keyString = UUID.randomUUID().toString(); String volumeName = UUID.randomUUID().toString(); @@ -666,7 +674,7 @@ private void testECReconstructionCoordinator(List missingIndexes, objectStore.getVolume(volumeName).createBucket(bucketName); OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - createKeyAndWriteData(keyString, bucket, numInputChunks); + createKeyAndWriteData(keyString, bucket, numInputChunks, triggerRetry); try ( XceiverClientManager xceiverClientManager = @@ -778,7 +786,7 @@ private void testECReconstructionCoordinator(List missingIndexes, .getReplicationConfig(), cToken); Assert.assertEquals(blockDataArrList.get(i).length, reconstructedBlockData.length); - checkBlockData(blockDataArrList.get(i), reconstructedBlockData); + checkBlockDataWithRetry(blockDataArrList.get(i), reconstructedBlockData, triggerRetry); XceiverClientSpi client = xceiverClientManager.acquireClient( newTargetPipeline); try { @@ -799,7 +807,7 @@ private void testECReconstructionCoordinator(List missingIndexes, } private void createKeyAndWriteData(String keyString, OzoneBucket bucket, - int numChunks) throws IOException { + int numChunks, boolean triggerRetry) throws IOException { for (int i = 0; i < numChunks; i++) { inputChunks[i] = getBytesWith(i + 1, EC_CHUNK_SIZE); } @@ -808,11 +816,48 @@ private void createKeyAndWriteData(String keyString, OzoneBucket bucket, new HashMap<>())) { Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream); for (int i = 0; i < numChunks; i++) { + // We generally wait until the data is written to the last chunk + // before attempting to trigger CloseContainer. + // We use an asynchronous approach for this trigger, + // aiming to ensure that closing the container does not interfere with the write operation. + // However, this process often needs to be executed multiple times before it takes effect. + if (i == numChunks - 1 && triggerRetry) { + triggerRetryByCloseContainer(out); + } out.write(inputChunks[i]); } } } + private void triggerRetryByCloseContainer(OzoneOutputStream out) { + CompletableFuture.runAsync(() -> { + BlockOutputStreamEntry blockOutputStreamEntry = out.getKeyOutputStream().getStreamEntries().get(0); + BlockID entryBlockID = blockOutputStreamEntry.getBlockID(); + long entryContainerID = entryBlockID.getContainerID(); + Pipeline entryPipeline = blockOutputStreamEntry.getPipeline(); + Map replicaIndexes = entryPipeline.getReplicaIndexes(); + try { + for (Map.Entry entry : replicaIndexes.entrySet()) { + DatanodeDetails key = entry.getKey(); + Integer value = entry.getValue(); + XceiverClientManager xceiverClientManager = new XceiverClientManager(config); + Token cToken = containerTokenGenerator + .generateToken(ANY_USER, ContainerID.valueOf(entryContainerID)); + XceiverClientSpi client = xceiverClientManager.acquireClient( + createSingleNodePipeline(entryPipeline, key, value)); + try { + ContainerProtocolCalls.closeContainer(client, entryContainerID, cToken.encodeToUrlString()); + } finally { + xceiverClientManager.releaseClient(client, false); + } + break; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + @Test public void testECReconstructionCoordinatorShouldCleanupContainersOnFailure() throws Exception { @@ -825,7 +870,7 @@ public void testECReconstructionCoordinatorShouldCleanupContainersOnFailure() objectStore.getVolume(volumeName).createBucket(bucketName); OzoneVolume volume = objectStore.getVolume(volumeName); OzoneBucket bucket = volume.getBucket(bucketName); - createKeyAndWriteData(keyString, bucket, 3); + createKeyAndWriteData(keyString, bucket, 3, false); OzoneKeyDetails key = bucket.getKey(keyString); long conID = key.getOzoneKeyLocations().get(0).getContainerID(); @@ -899,6 +944,25 @@ private void closeContainer(long conID) HddsProtos.LifeCycleEvent.CLOSE); } + private void checkBlockDataWithRetry( + org.apache.hadoop.ozone.container.common.helpers.BlockData[] blockData, + org.apache.hadoop.ozone.container.common.helpers.BlockData[] + reconstructedBlockData, boolean triggerRetry) { + if (triggerRetry) { + for (int i = 0; i < reconstructedBlockData.length; i++) { + assertEquals(blockData[i].getBlockID(), reconstructedBlockData[i].getBlockID()); + List oldBlockDataChunks = blockData[i].getChunks(); + List newBlockDataChunks = reconstructedBlockData[i].getChunks(); + for (int j = 0; j < newBlockDataChunks.size(); j++) { + ContainerProtos.ChunkInfo chunkInfo = oldBlockDataChunks.get(j); + assertEquals(chunkInfo, newBlockDataChunks.get(j)); + } + } + return; + } + checkBlockData(blockData, reconstructedBlockData); + } + private void checkBlockData( org.apache.hadoop.ozone.container.common.helpers.BlockData[] blockData, org.apache.hadoop.ozone.container.common.helpers.BlockData[] @@ -967,8 +1031,7 @@ public static void prepareData(int[][] ranges) throws Exception { out.write(values[i]); } } -// List containerIDs = -// new ArrayList<>(scm.getContainerManager().getContainerIDs()); + List containerIDs = scm.getContainerManager().getContainers() .stream() diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java index 3a8d117bc6d..8b2367c6b35 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java @@ -60,6 +60,7 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; import org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine; import org.apache.hadoop.ozone.om.ratis.metrics.OzoneManagerStateMachineMetrics; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; import org.apache.ozone.test.GenericTestUtils; import org.junit.Assert; import org.junit.jupiter.api.AfterAll; @@ -340,9 +341,11 @@ public void testParallelDeleteBucketAndCreateKey() throws IOException, omSM.getHandler().setInjector(injector); thread1.start(); thread2.start(); - GenericTestUtils.waitFor(() -> metrics.getApplyTransactionMapSize() > 0, - 100, 5000); - Thread.sleep(2000); + // Wait long enough for createKey's preExecute to finish executing + GenericTestUtils.waitFor(() -> { + return getCluster().getOzoneManager().getOmServerProtocol().getLastRequestToSubmit().getCmdType().equals( + Type.CreateKey); + }, 100, 10000); injector.resume(); try { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java index b878f920193..c3c09884822 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java @@ -72,6 +72,7 @@ import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; @@ -166,6 +167,7 @@ public class TestKeyManagerImpl { private static final String VERSIONED_BUCKET_NAME = "versionedBucket1"; private static final String VOLUME_NAME = "vol1"; private static OzoneManagerProtocol writeClient; + private static OzoneClient rpcClient; private static OzoneManager om; @BeforeAll @@ -211,6 +213,7 @@ public static void setUp() throws Exception { keyManager = (KeyManagerImpl)omTestManagers.getKeyManager(); prefixManager = omTestManagers.getPrefixManager(); writeClient = omTestManagers.getWriteClient(); + rpcClient = omTestManagers.getRpcClient(); mockContainerClient(); @@ -230,6 +233,8 @@ public static void setUp() throws Exception { @AfterAll public static void cleanup() throws Exception { + writeClient.close(); + rpcClient.close(); scm.stop(); scm.join(); om.stop(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java index cd932f6efde..526ff8e2013 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.hdds.utils.db.DBCheckpoint; import org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils; +import org.apache.hadoop.hdds.utils.db.RDBStore; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl; import org.apache.hadoop.ozone.client.BucketArgs; @@ -60,8 +61,8 @@ import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -198,11 +199,12 @@ public void shutdown() { } } - @ParameterizedTest - @ValueSource(ints = {100}) // tried up to 1000 snapshots and this test works, but some of the // timeouts have to be increased. - public void testInstallSnapshot(int numSnapshotsToCreate) throws Exception { + private static final int SNAPSHOTS_TO_CREATE = 100; + + @Test + public void testInstallSnapshot(@TempDir Path tempDir) throws Exception { // Get the leader OM String leaderOMNodeId = OmFailoverProxyUtil .getFailoverProxyProvider(objectStore.getClientProxy()) @@ -230,8 +232,7 @@ public void testInstallSnapshot(int numSnapshotsToCreate) throws Exception { String snapshotName = ""; List keys = new ArrayList<>(); SnapshotInfo snapshotInfo = null; - for (int snapshotCount = 0; snapshotCount < numSnapshotsToCreate; - snapshotCount++) { + for (int snapshotCount = 0; snapshotCount < SNAPSHOTS_TO_CREATE; snapshotCount++) { snapshotName = snapshotNamePrefix + snapshotCount; keys = writeKeys(keyIncrement); snapshotInfo = createOzoneSnapshot(leaderOM, snapshotName); @@ -326,7 +327,7 @@ public void testInstallSnapshot(int numSnapshotsToCreate) throws Exception { private void checkSnapshot(OzoneManager leaderOM, OzoneManager followerOM, String snapshotName, List keys, SnapshotInfo snapshotInfo) - throws IOException { + throws IOException, RocksDBException { // Read back data from snapshot. OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) @@ -347,10 +348,19 @@ private void checkSnapshot(OzoneManager leaderOM, OzoneManager followerOM, Path leaderActiveDir = Paths.get(leaderMetaDir.toString(), OM_DB_NAME); Path leaderSnapshotDir = Paths.get(getSnapshotPath(leaderOM.getConfiguration(), snapshotInfo)); + + // Get list of live files on the leader. + RocksDB activeRocksDB = ((RDBStore) leaderOM.getMetadataManager().getStore()) + .getDb().getManagedRocksDb().get(); + // strip the leading "/". + Set liveSstFiles = activeRocksDB.getLiveFiles().files.stream() + .map(s -> s.substring(1)) + .collect(Collectors.toSet()); + // Get the list of hardlinks from the leader. Then confirm those links // are on the follower int hardLinkCount = 0; - try (Streamlist = Files.list(leaderSnapshotDir)) { + try (Stream list = Files.list(leaderSnapshotDir)) { for (Path leaderSnapshotSST: list.collect(Collectors.toList())) { String fileName = leaderSnapshotSST.getFileName().toString(); if (fileName.toLowerCase().endsWith(".sst")) { @@ -358,7 +368,8 @@ private void checkSnapshot(OzoneManager leaderOM, OzoneManager followerOM, Path leaderActiveSST = Paths.get(leaderActiveDir.toString(), fileName); // Skip if not hard link on the leader - if (!leaderActiveSST.toFile().exists()) { + // First confirm it is live + if (!liveSstFiles.contains(fileName)) { continue; } // If it is a hard link on the leader, it should be a hard 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 62931e03b63..d7de66331c2 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; @@ -58,6 +60,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneKeyDetails; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.rpc.RpcClient; @@ -91,8 +94,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; @@ -107,6 +113,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -145,12 +154,19 @@ 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); - + private static OzoneClient ozoneClient; @BeforeAll public static void setUp() throws Exception { @@ -169,6 +185,7 @@ public static void setUp() throws Exception { OmTestManagers omTestManagers = new OmTestManagers(conf, mockScmBlockLocationProtocol, mockScmContainerClient); om = omTestManagers.getOzoneManager(); + ozoneClient = omTestManagers.getRpcClient(); metadataManager = omTestManagers.getMetadataManager(); rpcClient = new RpcClient(conf, null) { @@ -189,6 +206,7 @@ protected XceiverClientFactory createXceiverClientFactory( @AfterAll public static void cleanup() throws Exception { + ozoneClient.close(); om.stop(); FileUtils.deleteDirectory(dir); } @@ -197,6 +215,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(); @@ -214,6 +234,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; } @@ -228,6 +253,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 { @@ -253,9 +283,11 @@ private static void createVolume(String volumeName) throws IOException { public void beforeEach() { CONTAINER_ID.getAndIncrement(); Mockito.reset(mockScmBlockLocationProtocol, mockScmContainerClient, - mockDn1Protocol, mockDn2Protocol); + mockDn1Protocol, mockDn2Protocol, mockDnEcProtocol); 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))); } /** @@ -569,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) @@ -625,11 +699,27 @@ private void mockScmAllocationOnDn1(long containerID, .setContainerBlockID(blockId) .build(); when(mockScmBlockLocationProtocol - .allocateBlock(Mockito.anyLong(), Mockito.anyInt(), + .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), - Mockito.anyString(), + anyString(), any(ExcludeList.class), - Mockito.anyString())) + anyString())) + .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)); } @@ -662,6 +752,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, @@ -760,4 +864,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/TestOzoneManagerHASnapshot.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHASnapshot.java index 4dc0ab111be..a5ac7efb7fd 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHASnapshot.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHASnapshot.java @@ -21,31 +21,41 @@ import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl; +import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.TestDataUtil; import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse; +import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.io.File; import java.io.IOException; import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; +import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPath; import static org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus.DONE; import static org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus.IN_PROGRESS; import static org.awaitility.Awaitility.await; @@ -72,6 +82,8 @@ public static void staticInit() throws Exception { String clusterId = UUID.randomUUID().toString(); String scmId = UUID.randomUUID().toString(); conf.setBoolean(OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_KEY, true); + conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 1, TimeUnit.SECONDS); + conf.setTimeDuration(OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1, TimeUnit.SECONDS); cluster = (MiniOzoneHAClusterImpl) MiniOzoneCluster.newOMHABuilder(conf) .setClusterId(clusterId) @@ -271,4 +283,97 @@ private void createFileKey(OzoneBucket bucket, String keyName) fileKey.write(value); } } + + /** + * This is to simulate HDDS-11152 scenario. In which a follower's doubleBuffer is lagging and accumulates purgeKey + * and purgeSnapshot in same batch. + */ + @Test + public void testKeyAndSnapshotDeletionService() throws IOException, InterruptedException, TimeoutException { + OzoneManager omLeader = cluster.getOMLeader(); + OzoneManager omFollower; + + if (omLeader != cluster.getOzoneManager(0)) { + omFollower = cluster.getOzoneManager(0); + } else { + omFollower = cluster.getOzoneManager(1); + } + + int numKeys = 5; + List keys = new ArrayList<>(); + for (int i = 0; i < numKeys; i++) { + String keyName = "key-" + RandomStringUtils.randomNumeric(10); + createFileKey(ozoneBucket, keyName); + keys.add(keyName); + } + + // Stop the key deletion service so that deleted keys get trapped in the snapshots. + omLeader.getKeyManager().getDeletingService().suspend(); + // Stop the snapshot deletion service so that deleted keys get trapped in the snapshots. + omLeader.getKeyManager().getSnapshotDeletingService().suspend(); + + // Delete half of the keys + for (int i = 0; i < numKeys / 2; i++) { + ozoneBucket.deleteKey(keys.get(i)); + } + + String snapshotName = "snap-" + RandomStringUtils.randomNumeric(10); + createSnapshot(volumeName, bucketName, snapshotName); + + store.deleteSnapshot(volumeName, bucketName, snapshotName); + + // Pause double buffer on follower node to accumulate all the key purge, snapshot delete and purge transactions. + omFollower.getOmRatisServer().getOmStateMachine().getOzoneManagerDoubleBuffer().stopDaemon(); + + long keyDeleteServiceCount = omLeader.getKeyManager().getDeletingService().getRunCount().get(); + omLeader.getKeyManager().getDeletingService().resume(); + + GenericTestUtils.waitFor( + () -> omLeader.getKeyManager().getDeletingService().getRunCount().get() > keyDeleteServiceCount, + 1000, 60000); + + long snapshotDeleteServiceCount = omLeader.getKeyManager().getSnapshotDeletingService().getRunCount().get(); + omLeader.getKeyManager().getSnapshotDeletingService().resume(); + + GenericTestUtils.waitFor( + () -> omLeader.getKeyManager().getSnapshotDeletingService().getRunCount().get() > snapshotDeleteServiceCount, + 1000, 60000); + + String tableKey = SnapshotInfo.getTableKey(volumeName, bucketName, snapshotName); + checkSnapshotIsPurgedFromDB(omLeader, tableKey); + + // Resume the DoubleBuffer and flush the pending transactions. + OzoneManagerDoubleBuffer omDoubleBuffer = + omFollower.getOmRatisServer().getOmStateMachine().getOzoneManagerDoubleBuffer(); + omDoubleBuffer.resume(); + CompletableFuture.supplyAsync(() -> { + omDoubleBuffer.flushTransactions(); + return null; + }); + omDoubleBuffer.awaitFlush(); + checkSnapshotIsPurgedFromDB(omFollower, tableKey); + } + + private void createSnapshot(String volName, String buckName, String snapName) throws IOException { + store.createSnapshot(volName, buckName, snapName); + + String tableKey = SnapshotInfo.getTableKey(volName, buckName, snapName); + SnapshotInfo snapshotInfo = SnapshotUtils.getSnapshotInfo(cluster.getOMLeader(), tableKey); + String fileName = getSnapshotPath(cluster.getOMLeader().getConfiguration(), snapshotInfo); + File snapshotDir = new File(fileName); + if (!RDBCheckpointUtils.waitForCheckpointDirectoryExist(snapshotDir)) { + throw new IOException("Snapshot directory doesn't exist"); + } + } + + private void checkSnapshotIsPurgedFromDB(OzoneManager ozoneManager, String snapshotTableKey) + throws InterruptedException, TimeoutException { + GenericTestUtils.waitFor(() -> { + try { + return ozoneManager.getMetadataManager().getSnapshotInfoTable().get(snapshotTableKey) == null; + } catch (IOException e) { + throw new RuntimeException(e); + } + }, 1000, 60000); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java index 8bd78460c1b..528dcec4def 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java @@ -572,7 +572,7 @@ private void checkIfSnapshotGetsProcessedBySFS(OzoneManager ozoneManager) } catch (IOException e) { Assertions.fail(); } - return snapshotInfo.isSstFiltered(); + return SstFilteringService.isSstFiltered(ozoneManager.getConfiguration(), snapshotInfo); }, 1000, 10000); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshot.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshot.java index 72b8126c8cd..014ad97e602 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshot.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshot.java @@ -204,11 +204,7 @@ private void init() throws Exception { conf.setBoolean(OZONE_OM_ENABLE_FILESYSTEM_PATHS, enabledFileSystemPaths); conf.set(OZONE_DEFAULT_BUCKET_LAYOUT, bucketLayout.name()); conf.setBoolean(OZONE_OM_SNAPSHOT_FORCE_FULL_DIFF, forceFullSnapshotDiff); - conf.setBoolean(OZONE_OM_SNAPSHOT_DIFF_DISABLE_NATIVE_LIBS, - disableNativeDiff); - conf.setBoolean(OZONE_OM_ENABLE_FILESYSTEM_PATHS, enabledFileSystemPaths); - conf.set(OZONE_DEFAULT_BUCKET_LAYOUT, bucketLayout.name()); - conf.setBoolean(OZONE_OM_SNAPSHOT_FORCE_FULL_DIFF, forceFullSnapshotDiff); + conf.setBoolean(OZONE_OM_SNAPSHOT_DIFF_DISABLE_NATIVE_LIBS, disableNativeDiff); conf.setEnum(HDDS_DB_PROFILE, DBProfile.TEST); // Enable filesystem snapshot feature for the test regardless of the default conf.setBoolean(OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_KEY, true); @@ -1486,10 +1482,8 @@ public void testSnapDiffCancel() throws Exception { String toSnapshotTableKey = SnapshotInfo.getTableKey(volumeName, bucketName, toSnapName); - UUID fromSnapshotID = ozoneManager.getOmSnapshotManager() - .getSnapshotInfo(fromSnapshotTableKey).getSnapshotId(); - UUID toSnapshotID = ozoneManager.getOmSnapshotManager() - .getSnapshotInfo(toSnapshotTableKey).getSnapshotId(); + UUID fromSnapshotID = SnapshotUtils.getSnapshotInfo(ozoneManager, fromSnapshotTableKey).getSnapshotId(); + UUID toSnapshotID = SnapshotUtils.getSnapshotInfo(ozoneManager, toSnapshotTableKey).getSnapshotId(); // Construct SnapshotDiffJob table key. String snapDiffJobKey = fromSnapshotID + DELIMITER + toSnapshotID; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java index 965dd2f5255..44385698c5c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java @@ -72,8 +72,8 @@ public void init() throws Exception { taskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(15)); conf.setFromObject(taskConfig); - conf.set("ozone.scm.stale.node.interval", "10s"); - conf.set("ozone.scm.dead.node.interval", "20s"); + conf.set("ozone.scm.stale.node.interval", "6s"); + conf.set("ozone.scm.dead.node.interval", "10s"); cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1) .includeRecon(true).build(); cluster.waitForClusterToBeReady(); @@ -102,9 +102,6 @@ public void testSyncSCMContainerInfo() throws Exception { final ContainerInfo container2 = scmContainerManager.allocateContainer( RatisReplicationConfig.getInstance( HddsProtos.ReplicationFactor.ONE), "admin"); - reconContainerManager.allocateContainer( - RatisReplicationConfig.getInstance( - HddsProtos.ReplicationFactor.ONE), "admin"); scmContainerManager.updateContainerState(container1.containerID(), HddsProtos.LifeCycleEvent.FINALIZE); scmContainerManager.updateContainerState(container2.containerID(), diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java index 0d73f43772a..d958ca36115 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSecretKeysApi.java @@ -34,6 +34,7 @@ import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.util.ExitUtil; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.tag.Flaky; import org.apache.ratis.util.ExitUtils; import org.jetbrains.annotations.NotNull; import org.junit.After; @@ -192,6 +193,7 @@ private void setSecureConfig() throws IOException { * Test secret key apis in happy case. */ @Test + @Flaky("HDDS-8900") public void testSecretKeyApiSuccess() throws Exception { enableBlockToken(); // set a low rotation period, of 1s, expiry is 3s, expect 3 active keys 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 37a726beddb..9b251a3cec3 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 @@ -23,7 +23,6 @@ import java.io.IOException; import java.io.PrintStream; import java.io.UnsupportedEncodingException; -import java.net.URI; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; @@ -749,8 +748,6 @@ public void testListBucket() throws Exception { getClientConfForOFS(hostPrefix, cluster.getConf()); int pageSize = 20; clientConf.setInt(OZONE_FS_LISTING_PAGE_SIZE, pageSize); - URI uri = FileSystem.getDefaultUri(clientConf); - clientConf.setBoolean(String.format("fs.%s.impl.disable.cache", uri.getScheme()), true); OzoneFsShell shell = new OzoneFsShell(clientConf); String volName = "testlistbucket"; diff --git a/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml b/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml index 9687ed06ae4..91eeb76fc2d 100644 --- a/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml +++ b/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml @@ -54,7 +54,7 @@ hdds.container.ratis.log.appender.queue.byte-limit - 8MB + 32MB ozone.om.ratis.log.appender.queue.byte-limit diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java index 68c5cf758eb..1ff185fa355 100644 --- a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java +++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java @@ -46,8 +46,7 @@ import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ExpiredMultipartUploadsBucket; -import org.apache.hadoop.ozone.storage.proto. - OzoneManagerStorageProtos.PersistedUserVolumeInfo; +import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos.PersistedUserVolumeInfo; import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.Table; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java index 9d8352d2834..02a9e8babe1 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java @@ -97,6 +97,7 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_DIFF_REPORT_MAX_PAGE_SIZE; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_DIFF_REPORT_MAX_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KEY_NAME; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_SNAPSHOT_ERROR; import static org.apache.hadoop.ozone.om.snapshot.SnapshotDiffManager.getSnapshotRootPath; import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotActive; import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.dropColumnFamilyHandle; @@ -691,19 +692,38 @@ private ReferenceCounted getSnapshot(String snapshotTableKey, boolea } /** - * Returns true if the snapshot is in given status. - * @param key DB snapshot table key - * @param status SnapshotStatus - * @return true if the snapshot is in given status, false otherwise + * Returns OmSnapshot object and skips active check. + * This should only be used for API calls initiated by background service e.g. purgeKeys, purgeSnapshot, + * snapshotMoveDeletedKeys, and SetSnapshotProperty. */ - public boolean isSnapshotStatus(String key, - SnapshotInfo.SnapshotStatus status) - throws IOException { - return getSnapshotInfo(key).getSnapshotStatus().equals(status); + public ReferenceCounted getSnapshot(UUID snapshotId) throws IOException { + return snapshotCache.get(snapshotId); } - public SnapshotInfo getSnapshotInfo(String key) throws IOException { - return SnapshotUtils.getSnapshotInfo(ozoneManager, key); + /** + * Returns snapshotInfo from cache if it is present in cache, otherwise it checks RocksDB and return value from there. + * ################################################# + * NOTE: THIS SHOULD BE USED BY SNAPSHOT CACHE ONLY. + * ################################################# + * Sometimes, the follower OM node may be lagging that it gets purgeKeys or snapshotMoveDeletedKeys from a Snapshot, + * and purgeSnapshot for the same Snapshot one after another. And purgeSnapshot's validateAndUpdateCache gets + * executed before doubleBuffer flushes purgeKeys or snapshotMoveDeletedKeys from that Snapshot. + * This should not be a case on the leader node because SnapshotDeletingService checks that deletedTable and + * deletedDirectoryTable in DB don't have entries for the bucket before it sends a purgeSnapshot on a snapshot. + * If that happens, and we just look into the cache, the addToBatch operation will fail when it tries to open + * the DB and purgeKeys from the Snapshot because snapshot is already purged from the SnapshotInfoTable cache. + * Hence, it is needed to look into the table to make sure that snapshot exists somewhere either in cache or in DB. + */ + private SnapshotInfo getSnapshotInfo(String snapshotKey) throws IOException { + SnapshotInfo snapshotInfo = ozoneManager.getMetadataManager().getSnapshotInfoTable().get(snapshotKey); + + if (snapshotInfo == null) { + snapshotInfo = ozoneManager.getMetadataManager().getSnapshotInfoTable().getSkipCache(snapshotKey); + } + if (snapshotInfo == null) { + throw new OMException("Snapshot '" + snapshotKey + "' is not found.", INVALID_SNAPSHOT_ERROR); + } + return snapshotInfo; } public static String getSnapshotPrefix(String snapshotName) { @@ -711,6 +731,13 @@ public static String getSnapshotPrefix(String snapshotName) { snapshotName + OM_KEY_PREFIX; } + public static Path getSnapshotPath(OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo) { + RDBStore store = (RDBStore) omMetadataManager.getStore(); + String checkpointPrefix = store.getDbLocation().getName(); + return Paths.get(store.getSnapshotsParentDir(), + checkpointPrefix + snapshotInfo.getCheckpointDir()); + } + public static String getSnapshotPath(OzoneConfiguration conf, SnapshotInfo snapshotInfo) { return OMStorage.getOmDbDir(conf) + 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 3a15f2e8d54..e9e8067a278 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/SnapshotChainManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java index 18deca1a4ff..5a7a1371b15 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java @@ -354,13 +354,16 @@ public synchronized void addSnapshot(SnapshotInfo snapshotInfo) public synchronized boolean deleteSnapshot(SnapshotInfo snapshotInfo) throws IOException { validateSnapshotChain(); - boolean status = deleteSnapshotGlobal(snapshotInfo.getSnapshotId()) && - deleteSnapshotPath(snapshotInfo.getSnapshotPath(), - snapshotInfo.getSnapshotId()); - if (status) { - snapshotIdToTableKey.remove(snapshotInfo.getSnapshotId()); - } - return status; + return deleteSnapshotGlobal(snapshotInfo.getSnapshotId()) && + deleteSnapshotPath(snapshotInfo.getSnapshotPath(), snapshotInfo.getSnapshotId()); + } + + /** + * Remove the snapshot from snapshotIdToSnapshotTableKey map. + */ + public synchronized void removeFromSnapshotIdToTable(UUID snapshotId) throws IOException { + validateSnapshotChain(); + snapshotIdToTableKey.remove(snapshotId); } /** diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java index 20d0ab0e53e..52e5da50403 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.om; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.BackgroundService; import org.apache.hadoop.hdds.utils.BackgroundTask; @@ -38,6 +39,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Map; import java.util.Optional; import java.util.concurrent.TimeUnit; @@ -69,6 +73,10 @@ public class SstFilteringService extends BackgroundService // multiple times. private static final int SST_FILTERING_CORE_POOL_SIZE = 1; + public static final String SST_FILTERED_FILE = "sstFiltered"; + private static final byte[] SST_FILTERED_FILE_CONTENT = StringUtils.string2Bytes("This file holds information " + + "if a particular snapshot has filtered out the relevant sst files or not.\nDO NOT add, change or delete " + + "any files in this directory unless you know what you are doing.\n"); private final OzoneManager ozoneManager; // Number of files to be batched in an iteration. @@ -78,6 +86,12 @@ public class SstFilteringService extends BackgroundService private AtomicBoolean running; + public static boolean isSstFiltered(OzoneConfiguration ozoneConfiguration, SnapshotInfo snapshotInfo) { + Path sstFilteredFile = Paths.get(OmSnapshotManager.getSnapshotPath(ozoneConfiguration, + snapshotInfo), SST_FILTERED_FILE); + return snapshotInfo.isSstFiltered() || sstFilteredFile.toFile().exists(); + } + public SstFilteringService(long interval, TimeUnit unit, long serviceTimeout, OzoneManager ozoneManager, OzoneConfiguration configuration) { super("SstFilteringService", interval, unit, SST_FILTERING_CORE_POOL_SIZE, @@ -112,33 +126,35 @@ public void resume() { private class SstFilteringTask implements BackgroundTask { + private boolean isSnapshotDeleted(SnapshotInfo snapshotInfo) { + return snapshotInfo == null || snapshotInfo.getSnapshotStatus() == SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED; + } + /** - * Marks the SSTFiltered flag corresponding to the snapshot. - * @param volume Volume name of the snapshot - * @param bucket Bucket name of the snapshot - * @param snapshotName Snapshot name + * Marks the snapshot as SSTFiltered by creating a file in snapshot directory. + * @param snapshotInfo snapshotInfo * @throws IOException */ - private void markSSTFilteredFlagForSnapshot(String volume, String bucket, - String snapshotName) throws IOException { + private void markSSTFilteredFlagForSnapshot(SnapshotInfo snapshotInfo) throws IOException { + // Acquiring read lock to avoid race condition with the snapshot directory deletion occurring + // in OmSnapshotPurgeResponse. Any operation apart from delete can run in parallel along with this operation. + //TODO. Revisit other SNAPSHOT_LOCK and see if we can change write locks to read locks to further optimize it. OMLockDetails omLockDetails = ozoneManager.getMetadataManager().getLock() - .acquireWriteLock(SNAPSHOT_LOCK, volume, bucket, snapshotName); + .acquireReadLock(SNAPSHOT_LOCK, snapshotInfo.getVolumeName(), snapshotInfo.getBucketName(), + snapshotInfo.getName()); boolean acquiredSnapshotLock = omLockDetails.isLockAcquired(); if (acquiredSnapshotLock) { - Table snapshotInfoTable = - ozoneManager.getMetadataManager().getSnapshotInfoTable(); + String snapshotDir = OmSnapshotManager.getSnapshotPath(ozoneManager.getConfiguration(), snapshotInfo); try { - // mark the snapshot as filtered by writing to the file - String snapshotTableKey = SnapshotInfo.getTableKey(volume, bucket, - snapshotName); - SnapshotInfo snapshotInfo = snapshotInfoTable.get(snapshotTableKey); - - snapshotInfo.setSstFiltered(true); - snapshotInfoTable.put(snapshotTableKey, snapshotInfo); + // mark the snapshot as filtered by creating a file. + if (Files.exists(Paths.get(snapshotDir))) { + Files.write(Paths.get(snapshotDir, SST_FILTERED_FILE), SST_FILTERED_FILE_CONTENT); + } } finally { ozoneManager.getMetadataManager().getLock() - .releaseWriteLock(SNAPSHOT_LOCK, volume, bucket, snapshotName); + .releaseReadLock(SNAPSHOT_LOCK, snapshotInfo.getVolumeName(), + snapshotInfo.getBucketName(), snapshotInfo.getName()); } } } @@ -163,12 +179,11 @@ public BackgroundTaskResult call() throws Exception { long snapshotLimit = snapshotLimitPerTask; while (iterator.hasNext() && snapshotLimit > 0 && running.get()) { + Table.KeyValue keyValue = iterator.next(); + String snapShotTableKey = keyValue.getKey(); + SnapshotInfo snapshotInfo = keyValue.getValue(); try { - Table.KeyValue keyValue = iterator.next(); - String snapShotTableKey = keyValue.getKey(); - SnapshotInfo snapshotInfo = keyValue.getValue(); - - if (snapshotInfo.isSstFiltered()) { + if (isSstFiltered(ozoneManager.getConfiguration(), snapshotInfo)) { continue; } @@ -194,6 +209,9 @@ public BackgroundTaskResult call() throws Exception { .lock()) { db.deleteFilesNotMatchingPrefix(columnFamilyNameToPrefixMap); } + markSSTFilteredFlagForSnapshot(snapshotInfo); + snapshotLimit--; + snapshotFilteredCount.getAndIncrement(); } catch (OMException ome) { // FILE_NOT_FOUND is obtained when the snapshot is deleted // In this case, get the snapshotInfo from the db, check if @@ -202,20 +220,22 @@ public BackgroundTaskResult call() throws Exception { SnapshotInfo snapshotInfoToCheck = ozoneManager.getMetadataManager().getSnapshotInfoTable() .get(snapShotTableKey); - if (snapshotInfoToCheck.getSnapshotStatus() == - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED) { + if (isSnapshotDeleted(snapshotInfoToCheck)) { LOG.info("Snapshot with name: '{}', id: '{}' has been " + "deleted.", snapshotInfo.getName(), snapshotInfo .getSnapshotId()); } } } - markSSTFilteredFlagForSnapshot(snapshotInfo.getVolumeName(), - snapshotInfo.getBucketName(), snapshotInfo.getName()); - snapshotLimit--; - snapshotFilteredCount.getAndIncrement(); } catch (RocksDBException | IOException e) { - LOG.error("Exception encountered while filtering a snapshot", e); + if (isSnapshotDeleted(snapshotInfoTable.get(snapShotTableKey))) { + LOG.info("Exception encountered while filtering a snapshot: {} since it was deleted midway", + snapShotTableKey, e); + } else { + LOG.error("Exception encountered while filtering a snapshot", e); + } + + } } } catch (IOException 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 f26a5e300aa..48e41925a13 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 @@ -43,6 +43,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; @@ -596,15 +597,12 @@ public void stop() { } } - //TODO simplify it to make it shorter - @SuppressWarnings("methodlength") public static RaftProperties newRaftProperties(ConfigurationSource conf, int port, String ratisStorageDir) { // Set RPC type - final String rpcType = conf.get( + final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(conf.get( OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_KEY, - OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_DEFAULT); - final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType); + OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_DEFAULT)); final RaftProperties properties = RatisHelper.newRaftProperties(rpc); // Set the ratis port number @@ -615,129 +613,116 @@ public static RaftProperties newRaftProperties(ConfigurationSource conf, } // Set Ratis storage directory - RaftServerConfigKeys.setStorageDir(properties, - Collections.singletonList(new File(ratisStorageDir))); + RaftServerConfigKeys.setStorageDir(properties, Collections.singletonList(new File(ratisStorageDir))); + + 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 + RatisConfUtils.Grpc.setMessageSizeMax(properties, logAppenderBufferByteLimit); + + setRaftLeaderElectionProperties(properties, conf); + + setRaftRpcProperties(properties, conf); + + setRaftRetryCacheProperties(properties, conf); + + setRaftSnapshotProperties(properties, conf); + + setRaftCloseThreshold(properties, conf); + + getOMHAConfigs(conf).forEach(properties::set); + return properties; + } + + private static void setRaftLeaderElectionProperties(RaftProperties properties, ConfigurationSource conf) { // Disable/enable the pre vote feature in Ratis - RaftServerConfigKeys.LeaderElection.setPreVote(properties, - conf.getBoolean(OMConfigKeys.OZONE_OM_RATIS_SERVER_ELECTION_PRE_VOTE, - OMConfigKeys.OZONE_OM_RATIS_SERVER_ELECTION_PRE_VOTE_DEFAULT)); + RaftServerConfigKeys.LeaderElection.setPreVote(properties, conf.getBoolean( + OMConfigKeys.OZONE_OM_RATIS_SERVER_ELECTION_PRE_VOTE, + OMConfigKeys.OZONE_OM_RATIS_SERVER_ELECTION_PRE_VOTE_DEFAULT)); + } + private static void setRaftLogProperties(RaftProperties properties, + int logAppenderQueueByteLimit, ConfigurationSource conf) { // Set RAFT segment size - final long raftSegmentSize = (long) conf.getStorageSize( + RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf((long) conf.getStorageSize( OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY, - OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_DEFAULT, - StorageUnit.BYTES); - RaftServerConfigKeys.Log.setSegmentSizeMax(properties, - SizeInBytes.valueOf(raftSegmentSize)); + OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_DEFAULT, StorageUnit.BYTES))); // Set to enable RAFT to purge logs up to Snapshot Index - RaftServerConfigKeys.Log.setPurgeUptoSnapshotIndex(properties, - conf.getBoolean( - OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_UPTO_SNAPSHOT_INDEX, - OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_UPTO_SNAPSHOT_INDEX_DEFAULT - ) - ); + RaftServerConfigKeys.Log.setPurgeUptoSnapshotIndex(properties, conf.getBoolean( + OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_UPTO_SNAPSHOT_INDEX, + OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_UPTO_SNAPSHOT_INDEX_DEFAULT)); + // Set number of last RAFT logs to not be purged - RaftServerConfigKeys.Log.setPurgePreservationLogNum(properties, - conf.getLong( - OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_PRESERVATION_LOG_NUM, - OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_PRESERVATION_LOG_NUM_DEFAULT - ) - ); + RaftServerConfigKeys.Log.setPurgePreservationLogNum(properties, conf.getLong( + OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_PRESERVATION_LOG_NUM, + OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_PRESERVATION_LOG_NUM_DEFAULT)); // Set RAFT segment pre-allocated size - final long raftSegmentPreallocatedSize = (long) conf.getStorageSize( + RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf((long) conf.getStorageSize( OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, - OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT, - StorageUnit.BYTES); - int logAppenderQueueNumElements = conf.getInt( + OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT, StorageUnit.BYTES))); + + // Set RAFT buffer element limit + RaftServerConfigKeys.Log.Appender.setBufferElementLimit(properties, conf.getInt( OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS, - OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT); - final int logAppenderQueueByteLimit = (int) conf.getStorageSize( - OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT, - OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT, - StorageUnit.BYTES); - RaftServerConfigKeys.Log.Appender.setBufferElementLimit(properties, - logAppenderQueueNumElements); - RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties, - SizeInBytes.valueOf(logAppenderQueueByteLimit)); - RaftServerConfigKeys.Log.setWriteBufferSize(properties, - SizeInBytes.valueOf(logAppenderQueueByteLimit + 8)); - RaftServerConfigKeys.Log.setPreallocatedSize(properties, - SizeInBytes.valueOf(raftSegmentPreallocatedSize)); - RaftServerConfigKeys.Log.Appender.setInstallSnapshotEnabled(properties, - false); - final int logPurgeGap = conf.getInt( + OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT)); + + RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties, SizeInBytes.valueOf(logAppenderQueueByteLimit)); + RaftServerConfigKeys.Log.setWriteBufferSize(properties, SizeInBytes.valueOf(logAppenderQueueByteLimit + 8)); + RaftServerConfigKeys.Log.Appender.setInstallSnapshotEnabled(properties, false); + + RaftServerConfigKeys.Log.setPurgeGap(properties, conf.getInt( OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP, - OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP_DEFAULT); - RaftServerConfigKeys.Log.setPurgeGap(properties, logPurgeGap); + OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP_DEFAULT)); - // For grpc set the maximum message size - // TODO: calculate the optimal max message size - GrpcConfigKeys.setMessageSizeMax(properties, - SizeInBytes.valueOf(logAppenderQueueByteLimit)); + // Set the number of maximum cached segments + RaftServerConfigKeys.Log.setSegmentCacheNumMax(properties, 2); + } + private static void setRaftRpcProperties(RaftProperties properties, ConfigurationSource conf) { // Set the server request timeout - TimeUnit serverRequestTimeoutUnit = - OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getUnit(); - long serverRequestTimeoutDuration = conf.getTimeDuration( + TimeUnit serverRequestTimeoutUnit = OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getUnit(); + final TimeDuration serverRequestTimeout = TimeDuration.valueOf(conf.getTimeDuration( OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_KEY, - OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT - .getDuration(), serverRequestTimeoutUnit); - final TimeDuration serverRequestTimeout = TimeDuration.valueOf( - serverRequestTimeoutDuration, serverRequestTimeoutUnit); - RaftServerConfigKeys.Rpc.setRequestTimeout(properties, - serverRequestTimeout); - - // Set timeout for server retry cache entry - TimeUnit retryCacheTimeoutUnit = OMConfigKeys - .OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT.getUnit(); - long retryCacheTimeoutDuration = conf.getTimeDuration( - OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_KEY, - OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT - .getDuration(), retryCacheTimeoutUnit); - final TimeDuration retryCacheTimeout = TimeDuration.valueOf( - retryCacheTimeoutDuration, retryCacheTimeoutUnit); - RaftServerConfigKeys.RetryCache.setExpiryTime(properties, - retryCacheTimeout); + OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getDuration(), serverRequestTimeoutUnit), + serverRequestTimeoutUnit); + RaftServerConfigKeys.Rpc.setRequestTimeout(properties, serverRequestTimeout); // Set the server min and max timeout - TimeUnit serverMinTimeoutUnit = - OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT.getUnit(); - long serverMinTimeoutDuration = conf.getTimeDuration( + TimeUnit serverMinTimeoutUnit = OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT.getUnit(); + final TimeDuration serverMinTimeout = TimeDuration.valueOf(conf.getTimeDuration( OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_KEY, - OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT - .getDuration(), serverMinTimeoutUnit); - final TimeDuration serverMinTimeout = TimeDuration.valueOf( - serverMinTimeoutDuration, serverMinTimeoutUnit); - long serverMaxTimeoutDuration = - serverMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200; - final TimeDuration serverMaxTimeout = TimeDuration.valueOf( - serverMaxTimeoutDuration, TimeUnit.MILLISECONDS); - RaftServerConfigKeys.Rpc.setTimeoutMin(properties, - serverMinTimeout); - RaftServerConfigKeys.Rpc.setTimeoutMax(properties, - serverMaxTimeout); - - // Set the number of maximum cached segments - RaftServerConfigKeys.Log.setSegmentCacheNumMax(properties, 2); - - // TODO: set max write buffer size - - TimeUnit nodeFailureTimeoutUnit = - OMConfigKeys.OZONE_OM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_DEFAULT - .getUnit(); - long nodeFailureTimeoutDuration = conf.getTimeDuration( + OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT.getDuration(), serverMinTimeoutUnit), + serverMinTimeoutUnit); + final TimeDuration serverMaxTimeout = serverMinTimeout.add(200, TimeUnit.MILLISECONDS); + RaftServerConfigKeys.Rpc.setTimeoutMin(properties, serverMinTimeout); + RaftServerConfigKeys.Rpc.setTimeoutMax(properties, serverMaxTimeout); + + // Set the server Rpc slowness timeout and Notification noLeader timeout + TimeUnit nodeFailureTimeoutUnit = OMConfigKeys.OZONE_OM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_DEFAULT.getUnit(); + final TimeDuration nodeFailureTimeout = TimeDuration.valueOf(conf.getTimeDuration( OMConfigKeys.OZONE_OM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_KEY, - OMConfigKeys.OZONE_OM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_DEFAULT - .getDuration(), nodeFailureTimeoutUnit); - final TimeDuration nodeFailureTimeout = TimeDuration.valueOf( - nodeFailureTimeoutDuration, nodeFailureTimeoutUnit); - RaftServerConfigKeys.Notification.setNoLeaderTimeout(properties, - nodeFailureTimeout); - RaftServerConfigKeys.Rpc.setSlownessTimeout(properties, - nodeFailureTimeout); + OMConfigKeys.OZONE_OM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_DEFAULT.getDuration(), nodeFailureTimeoutUnit), + nodeFailureTimeoutUnit); + RaftServerConfigKeys.Notification.setNoLeaderTimeout(properties, nodeFailureTimeout); + RaftServerConfigKeys.Rpc.setSlownessTimeout(properties, nodeFailureTimeout); + } + + private static void setRaftRetryCacheProperties(RaftProperties properties, ConfigurationSource conf) { + // Set timeout for server retry cache entry + TimeUnit retryCacheTimeoutUnit = OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT.getUnit(); + final TimeDuration retryCacheTimeout = TimeDuration.valueOf(conf.getTimeDuration( + OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_KEY, + OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT.getDuration(), retryCacheTimeoutUnit), + retryCacheTimeoutUnit); + RaftServerConfigKeys.RetryCache.setExpiryTime(properties, retryCacheTimeout); + } + private static void setRaftSnapshotProperties(RaftProperties properties, ConfigurationSource conf) { // Set auto trigger snapshot. We don't need to configure auto trigger // threshold in OM, as last applied index is flushed during double buffer // flush automatically. (But added this property internally, so that this @@ -747,18 +732,22 @@ public static RaftProperties newRaftProperties(ConfigurationSource conf, // The transaction info value in OM DB is used as // snapshot value after restart. - RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled( - properties, true); + RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(properties, true); - long snapshotAutoTriggerThreshold = conf.getLong( + RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold(properties, conf.getLong( OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_KEY, - OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_DEFAULT); - - RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold(properties, - snapshotAutoTriggerThreshold); - - getOMHAConfigs(conf).forEach(properties::set); - return properties; + OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_DEFAULT)); + } + + private static void setRaftCloseThreshold(RaftProperties properties, ConfigurationSource conf) { + // Set RAFT server close threshold + TimeUnit closeThresholdUnit = OMConfigKeys.OZONE_OM_RATIS_SERVER_CLOSE_THRESHOLD_DEFAULT.getUnit(); + final int closeThreshold = (int) TimeDuration.valueOf(conf.getTimeDuration( + OMConfigKeys.OZONE_OM_RATIS_SERVER_CLOSE_THRESHOLD_KEY, + OMConfigKeys.OZONE_OM_RATIS_SERVER_CLOSE_THRESHOLD_DEFAULT.getDuration(), closeThresholdUnit), + closeThresholdUnit).toLong(TimeUnit.SECONDS); + // TODO: update to new api setCloseThreshold(RaftProperties, TimeDuration) if available + RaftServerConfigKeys.setCloseThreshold(properties, closeThreshold); } private static Map getOMHAConfigs( diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java index 33708450294..0c52cf4857f 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java @@ -26,6 +26,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; @@ -69,9 +70,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager(); try { if (fromSnapshot != null) { - fromSnapshotInfo = ozoneManager.getMetadataManager() - .getSnapshotInfoTable() - .get(fromSnapshot); + fromSnapshotInfo = SnapshotUtils.getSnapshotInfo(ozoneManager, fromSnapshot); } for (OzoneManagerProtocolProtos.PurgePathRequest path : purgeRequests) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java index 778f526d67d..8bfa8439a64 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; @@ -74,14 +75,12 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, try { SnapshotInfo fromSnapshotInfo = null; if (fromSnapshot != null) { - fromSnapshotInfo = ozoneManager.getMetadataManager() - .getSnapshotInfoTable().get(fromSnapshot); + fromSnapshotInfo = SnapshotUtils.getSnapshotInfo(ozoneManager, fromSnapshot); } omClientResponse = new OMKeyPurgeResponse(omResponse.build(), keysToBePurgedList, fromSnapshotInfo, keysToUpdateList); } catch (IOException ex) { - omClientResponse = new OMKeyPurgeResponse( - createErrorOMResponse(omResponse, ex)); + omClientResponse = new OMKeyPurgeResponse(createErrorOMResponse(omResponse, ex)); } return omClientResponse; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java index 7e1e469aa25..7b039849430 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; @@ -60,7 +59,6 @@ public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) { @DisallowedUntilLayoutVersion(FILESYSTEM_SNAPSHOT) public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, long trxnLogIndex) { - OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager(); OmMetadataManagerImpl omMetadataManager = (OmMetadataManagerImpl) ozoneManager.getMetadataManager(); SnapshotChainManager snapshotChainManager = @@ -78,8 +76,10 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, OzoneManagerProtocolProtos.OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(getOmRequest()); try { - nextSnapshot = SnapshotUtils.getNextActiveSnapshot(fromSnapshot, - snapshotChainManager, omSnapshotManager); + // Check the snapshot exists. + SnapshotUtils.getSnapshotInfo(ozoneManager, fromSnapshot.getTableKey()); + + nextSnapshot = SnapshotUtils.getNextActiveSnapshot(fromSnapshot, snapshotChainManager, ozoneManager); // Get next non-deleted snapshot. List nextDBKeysList = 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 ed4842c7eab..f8b27961e66 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,14 +19,11 @@ 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.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; @@ -43,15 +40,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. @@ -60,6 +53,13 @@ public class OMSnapshotPurgeRequest extends OMClientRequest { private static final Logger LOG = LoggerFactory.getLogger(OMSnapshotPurgeRequest.class); + /** + * This map contains up to date snapshotInfo and works as a local cache for OMSnapshotPurgeRequest. + * Since purge and other updates happen in sequence inside validateAndUpdateCache, we can get updated snapshotInfo + * from this map rather than getting form snapshotInfoTable which creates a deep copy for every get call. + */ + private final Map updatedSnapshotInfos = new HashMap<>(); + public OMSnapshotPurgeRequest(OMRequest omRequest) { super(omRequest); } @@ -69,7 +69,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, long trxnLogIndex) { OMMetrics omMetrics = ozoneManager.getMetrics(); - OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager(); OmMetadataManagerImpl omMetadataManager = (OmMetadataManagerImpl) ozoneManager.getMetadataManager(); SnapshotChainManager snapshotChainManager = @@ -85,77 +84,40 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, try { List snapshotDbKeys = snapshotPurgeRequest .getSnapshotDBKeysList(); - Map updatedSnapInfos = new HashMap<>(); - Map updatedPathPreviousAndGlobalSnapshots = - 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 = getUpdatedSnapshotInfo(snapTableKey, omMetadataManager); + 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, ozoneManager); + + // Step 1: Update the deep clean flag for the next active snapshot + updateSnapshotInfoAndCache(nextSnapshot, omMetadataManager, trxnLogIndex); + // Step 2: Update the snapshot chain. + updateSnapshotChainAndCache(omMetadataManager, fromSnapshot, trxnLogIndex); + // Step 3: Purge the snapshot from SnapshotInfoTable cache. + omMetadataManager.getSnapshotInfoTable() + .addCacheEntry(new CacheKey<>(fromSnapshot.getTableKey()), CacheValue.get(trxnLogIndex)); } - omClientResponse = new OMSnapshotPurgeResponse(omResponse.build(), - snapshotDbKeys, updatedSnapInfos, - updatedPathPreviousAndGlobalSnapshots); + omClientResponse = new OMSnapshotPurgeResponse(omResponse.build(), snapshotDbKeys, updatedSnapshotInfos); omMetrics.incNumSnapshotPurges(); - LOG.info("Successfully executed snapshotPurgeRequest: {{}} along with updating deep clean flags for " + - "snapshots: {} and global and previous for snapshots:{}.", - snapshotPurgeRequest, updatedSnapInfos.keySet(), updatedPathPreviousAndGlobalSnapshots.keySet()); + LOG.info("Successfully executed snapshotPurgeRequest: {{}} along with updating snapshots:{}.", + snapshotPurgeRequest, updatedSnapshotInfos); } catch (IOException ex) { omClientResponse = new OMSnapshotPurgeResponse( createErrorOMResponse(omResponse, ex)); @@ -166,41 +128,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, 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 { + private void updateSnapshotInfoAndCache(SnapshotInfo snapInfo, OmMetadataManagerImpl omMetadataManager, + long trxnLogIndex) 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)); + updatedSnapshotInfos.put(snapInfo.getTableKey(), snapInfo); } } @@ -211,11 +150,9 @@ private void updateSnapshotInfoAndCache(SnapshotInfo snapInfo, * update in DB. */ private void updateSnapshotChainAndCache( - Set> lockSet, OmMetadataManagerImpl metadataManager, SnapshotInfo snapInfo, - long trxnLogIndex, - Map updatedPathPreviousAndGlobalSnapshots + long trxnLogIndex ) throws IOException { if (snapInfo == null) { return; @@ -245,58 +182,45 @@ 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 = - nextPathSnapshotKey != null ? metadataManager.getSnapshotInfoTable().get(nextPathSnapshotKey) : null; - - SnapshotInfo nextGlobalSnapInfo = - nextGlobalSnapshotKey != null ? metadataManager.getSnapshotInfoTable().get(nextGlobalSnapshotKey) : null; + nextPathSnapshotKey != null ? getUpdatedSnapshotInfo(nextPathSnapshotKey, metadataManager) : null; - // Updates next path snapshot's previous snapshot ID if (nextPathSnapInfo != null) { nextPathSnapInfo.setPathPreviousSnapshotId(snapInfo.getPathPreviousSnapshotId()); metadataManager.getSnapshotInfoTable().addCacheEntry( new CacheKey<>(nextPathSnapInfo.getTableKey()), CacheValue.get(trxnLogIndex, nextPathSnapInfo)); - updatedPathPreviousAndGlobalSnapshots - .put(nextPathSnapInfo.getTableKey(), nextPathSnapInfo); } - // Updates next global snapshot's previous snapshot ID - // If both next global and path snapshot are same, it may overwrite - // nextPathSnapInfo.setPathPreviousSnapshotID(), adding this check - // will prevent it. - if (nextGlobalSnapInfo != null && nextPathSnapInfo != null && - nextGlobalSnapInfo.getSnapshotId().equals(nextPathSnapInfo.getSnapshotId())) { - nextPathSnapInfo.setGlobalPreviousSnapshotId(snapInfo.getGlobalPreviousSnapshotId()); - metadataManager.getSnapshotInfoTable().addCacheEntry( - new CacheKey<>(nextPathSnapInfo.getTableKey()), - CacheValue.get(trxnLogIndex, nextPathSnapInfo)); - updatedPathPreviousAndGlobalSnapshots - .put(nextPathSnapInfo.getTableKey(), nextPathSnapInfo); - } else if (nextGlobalSnapInfo != null) { - nextGlobalSnapInfo.setGlobalPreviousSnapshotId( - snapInfo.getGlobalPreviousSnapshotId()); + SnapshotInfo nextGlobalSnapInfo = + nextGlobalSnapshotKey != null ? getUpdatedSnapshotInfo(nextGlobalSnapshotKey, metadataManager) : null; + + if (nextGlobalSnapInfo != null) { + nextGlobalSnapInfo.setGlobalPreviousSnapshotId(snapInfo.getGlobalPreviousSnapshotId()); metadataManager.getSnapshotInfoTable().addCacheEntry( new CacheKey<>(nextGlobalSnapInfo.getTableKey()), CacheValue.get(trxnLogIndex, nextGlobalSnapInfo)); - updatedPathPreviousAndGlobalSnapshots - .put(nextGlobalSnapInfo.getTableKey(), nextGlobalSnapInfo); } snapshotChainManager.deleteSnapshot(snapInfo); } + + private SnapshotInfo getUpdatedSnapshotInfo(String snapshotTableKey, OMMetadataManager omMetadataManager) + throws IOException { + SnapshotInfo snapshotInfo = updatedSnapshotInfos.get(snapshotTableKey); + + if (snapshotInfo == null) { + snapshotInfo = omMetadataManager.getSnapshotInfoTable().get(snapshotTableKey); + updatedSnapshotInfos.put(snapshotTableKey, snapshotInfo); + } + return snapshotInfo; + } } 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 3d4873e6735..8f9c3f3afa6 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 @@ -37,7 +37,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 OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, long trxnLogIndex) { 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, 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, 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/response/key/OMDirectoriesPurgeResponseWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java index 5d3875baaf8..be9d5668734 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java @@ -84,10 +84,7 @@ public void addToDBBatch(OMMetadataManager metadataManager, .getOzoneManager().getOmSnapshotManager(); try (ReferenceCounted - rcFromSnapshotInfo = omSnapshotManager.getSnapshot( - fromSnapshotInfo.getVolumeName(), - fromSnapshotInfo.getBucketName(), - fromSnapshotInfo.getName())) { + rcFromSnapshotInfo = omSnapshotManager.getSnapshot(fromSnapshotInfo.getSnapshotId())) { OmSnapshot fromSnapshot = rcFromSnapshotInfo.get(); DBStore fromSnapshotStore = fromSnapshot.getMetadataManager() .getStore(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java index b606b610dae..f19d361e2ce 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java @@ -75,18 +75,13 @@ public void addToDBBatch(OMMetadataManager omMetadataManager, if (fromSnapshot != null) { OmSnapshotManager omSnapshotManager = - ((OmMetadataManagerImpl) omMetadataManager) - .getOzoneManager().getOmSnapshotManager(); + ((OmMetadataManagerImpl) omMetadataManager).getOzoneManager().getOmSnapshotManager(); try (ReferenceCounted rcOmFromSnapshot = - omSnapshotManager.getSnapshot( - fromSnapshot.getVolumeName(), - fromSnapshot.getBucketName(), - fromSnapshot.getName())) { + omSnapshotManager.getSnapshot(fromSnapshot.getSnapshotId())) { OmSnapshot fromOmSnapshot = rcOmFromSnapshot.get(); - DBStore fromSnapshotStore = - fromOmSnapshot.getMetadataManager().getStore(); + DBStore fromSnapshotStore = fromOmSnapshot.getMetadataManager().getStore(); // Init Batch Operation for snapshot db. try (BatchOperation writeBatch = fromSnapshotStore.initBatchOperation()) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java index 043b647e088..64618ea8853 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java @@ -91,19 +91,13 @@ protected void addToDBBatch(OMMetadataManager omMetadataManager, .getOzoneManager().getOmSnapshotManager(); try (ReferenceCounted rcOmFromSnapshot = - omSnapshotManager.getSnapshot( - fromSnapshot.getVolumeName(), - fromSnapshot.getBucketName(), - fromSnapshot.getName())) { + omSnapshotManager.getSnapshot(fromSnapshot.getSnapshotId())) { OmSnapshot fromOmSnapshot = rcOmFromSnapshot.get(); if (nextSnapshot != null) { try (ReferenceCounted - rcOmNextSnapshot = omSnapshotManager.getSnapshot( - nextSnapshot.getVolumeName(), - nextSnapshot.getBucketName(), - nextSnapshot.getName())) { + rcOmNextSnapshot = omSnapshotManager.getSnapshot(nextSnapshot.getSnapshotId())) { OmSnapshot nextOmSnapshot = rcOmNextSnapshot.get(); RDBStore nextSnapshotStore = 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 1dc27cc5f6b..c634198df0a 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 @@ -20,10 +20,11 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.hdds.utils.db.BatchOperation; -import org.apache.hadoop.hdds.utils.db.RDBStore; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.OMLockDetails; 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; @@ -33,11 +34,11 @@ import javax.annotation.Nonnull; import java.io.IOException; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.List; import java.util.Map; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE; +import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK; /** * Response for OMSnapshotPurgeRequest. @@ -48,18 +49,15 @@ public class OMSnapshotPurgeResponse extends OMClientResponse { LoggerFactory.getLogger(OMSnapshotPurgeResponse.class); private final List snapshotDbKeys; private final Map updatedSnapInfos; - private final Map updatedPreviousAndGlobalSnapInfos; public OMSnapshotPurgeResponse( @Nonnull OMResponse omResponse, @Nonnull List snapshotDbKeys, - Map updatedSnapInfos, - Map updatedPreviousAndGlobalSnapInfos + Map updatedSnapInfos ) { super(omResponse); this.snapshotDbKeys = snapshotDbKeys; this.updatedSnapInfos = updatedSnapInfos; - this.updatedPreviousAndGlobalSnapInfos = updatedPreviousAndGlobalSnapInfos; } /** @@ -71,7 +69,6 @@ public OMSnapshotPurgeResponse(@Nonnull OMResponse omResponse) { checkStatusNotOK(); this.snapshotDbKeys = null; this.updatedSnapInfos = null; - this.updatedPreviousAndGlobalSnapInfos = null; } @Override @@ -80,8 +77,6 @@ protected void addToDBBatch(OMMetadataManager omMetadataManager, OmMetadataManagerImpl metadataManager = (OmMetadataManagerImpl) omMetadataManager; - updateSnapInfo(metadataManager, batchOperation, - updatedPreviousAndGlobalSnapInfos); updateSnapInfo(metadataManager, batchOperation, updatedSnapInfos); for (String dbKey: snapshotDbKeys) { // Skip the cache here because snapshot is purged from cache in OMSnapshotPurgeRequest. @@ -95,8 +90,15 @@ protected void addToDBBatch(OMMetadataManager omMetadataManager, continue; } + // Remove and close snapshot's RocksDB instance from SnapshotCache. + ((OmMetadataManagerImpl) omMetadataManager).getOzoneManager().getOmSnapshotManager() + .invalidateCacheEntry(snapshotInfo.getSnapshotId()); + // Remove the snapshot from snapshotId to snapshotTableKey map. + ((OmMetadataManagerImpl) omMetadataManager).getSnapshotChainManager() + .removeFromSnapshotIdToTable(snapshotInfo.getSnapshotId()); // Delete Snapshot checkpoint directory. deleteCheckpointDirectory(omMetadataManager, snapshotInfo); + // Delete snapshotInfo from the table. omMetadataManager.getSnapshotInfoTable().deleteWithBatch(batchOperation, dbKey); } } @@ -116,15 +118,24 @@ private void updateSnapInfo(OmMetadataManagerImpl metadataManager, */ private void deleteCheckpointDirectory(OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo) { - RDBStore store = (RDBStore) omMetadataManager.getStore(); - String checkpointPrefix = store.getDbLocation().getName(); - Path snapshotDirPath = Paths.get(store.getSnapshotsParentDir(), - checkpointPrefix + snapshotInfo.getCheckpointDir()); - try { - FileUtils.deleteDirectory(snapshotDirPath.toFile()); - } catch (IOException ex) { - LOG.error("Failed to delete snapshot directory {} for snapshot {}", - snapshotDirPath, snapshotInfo.getTableKey(), ex); + // Acquiring write lock to avoid race condition with sst filtering service which creates a sst filtered file + // inside the snapshot directory. Any operation apart which doesn't create/delete files under this snapshot + // directory can run in parallel along with this operation. + OMLockDetails omLockDetails = omMetadataManager.getLock() + .acquireWriteLock(SNAPSHOT_LOCK, snapshotInfo.getVolumeName(), snapshotInfo.getBucketName(), + snapshotInfo.getName()); + boolean acquiredSnapshotLock = omLockDetails.isLockAcquired(); + if (acquiredSnapshotLock) { + Path snapshotDirPath = OmSnapshotManager.getSnapshotPath(omMetadataManager, snapshotInfo); + try { + FileUtils.deleteDirectory(snapshotDirPath.toFile()); + } catch (IOException ex) { + LOG.error("Failed to delete snapshot directory {} for snapshot {}", + snapshotDirPath, snapshotInfo.getTableKey(), ex); + } finally { + omMetadataManager.getLock().releaseWriteLock(SNAPSHOT_LOCK, snapshotInfo.getVolumeName(), + snapshotInfo.getBucketName(), snapshotInfo.getName()); + } } } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java index 429e286287c..154bd474b6d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java @@ -31,7 +31,6 @@ import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OMMetadataManager; -import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.helpers.OMRatisHelper; @@ -39,6 +38,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeletedKeys; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; @@ -576,8 +576,7 @@ protected boolean isBufferLimitCrossed( return cLimit + increment >= maxLimit; } - protected SnapshotInfo getPreviousActiveSnapshot(SnapshotInfo snapInfo, - SnapshotChainManager chainManager, OmSnapshotManager omSnapshotManager) + protected SnapshotInfo getPreviousActiveSnapshot(SnapshotInfo snapInfo, SnapshotChainManager chainManager) throws IOException { SnapshotInfo currSnapInfo = snapInfo; while (chainManager.hasPreviousPathSnapshot( @@ -586,7 +585,7 @@ protected SnapshotInfo getPreviousActiveSnapshot(SnapshotInfo snapInfo, UUID prevPathSnapshot = chainManager.previousPathSnapshot( currSnapInfo.getSnapshotPath(), currSnapInfo.getSnapshotId()); String tableKey = chainManager.getTableKey(prevPathSnapshot); - SnapshotInfo prevSnapInfo = omSnapshotManager.getSnapshotInfo(tableKey); + SnapshotInfo prevSnapInfo = SnapshotUtils.getSnapshotInfo(ozoneManager, tableKey); if (prevSnapInfo.getSnapshotStatus() == SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE) { return prevSnapInfo; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java index 83991668c9f..f3e7ab588cd 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java @@ -289,13 +289,11 @@ private void processSnapshotDeepClean(int delCount) } String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX; - SnapshotInfo previousSnapshot = getPreviousActiveSnapshot( - currSnapInfo, snapChainManager, omSnapshotManager); + SnapshotInfo previousSnapshot = getPreviousActiveSnapshot(currSnapInfo, snapChainManager); SnapshotInfo previousToPrevSnapshot = null; if (previousSnapshot != null) { - previousToPrevSnapshot = getPreviousActiveSnapshot( - previousSnapshot, snapChainManager, omSnapshotManager); + previousToPrevSnapshot = getPreviousActiveSnapshot(previousSnapshot, snapChainManager); } Table previousKeyTable = null; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java index 29b2b319532..a98081c63a1 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java @@ -36,7 +36,6 @@ import org.apache.hadoop.ozone.lock.BootstrapStateHandler; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.KeyManagerImpl; import org.apache.hadoop.ozone.om.OmSnapshot; import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; @@ -153,12 +152,9 @@ public BackgroundTaskResult call() throws InterruptedException { while (iterator.hasNext() && snapshotLimit > 0) { SnapshotInfo snapInfo = iterator.next().getValue(); - boolean isSstFilteringServiceEnabled = - ((KeyManagerImpl) ozoneManager.getKeyManager()) - .isSstFilteringSvcEnabled(); // Only Iterate in deleted snapshot - if (shouldIgnoreSnapshot(snapInfo, isSstFilteringServiceEnabled)) { + if (shouldIgnoreSnapshot(snapInfo)) { continue; } @@ -211,8 +207,7 @@ public BackgroundTaskResult call() throws InterruptedException { } //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB. - SnapshotInfo previousSnapshot = getPreviousActiveSnapshot( - snapInfo, chainManager, omSnapshotManager); + SnapshotInfo previousSnapshot = getPreviousActiveSnapshot(snapInfo, chainManager); Table previousKeyTable = null; Table previousDirTable = null; OmSnapshot omPreviousSnapshot = null; @@ -591,11 +586,10 @@ public void submitRequest(OMRequest omRequest) { } } - public static boolean shouldIgnoreSnapshot(SnapshotInfo snapInfo, - boolean isSstFilteringServiceEnabled) { + @VisibleForTesting + boolean shouldIgnoreSnapshot(SnapshotInfo snapInfo) { SnapshotInfo.SnapshotStatus snapshotStatus = snapInfo.getSnapshotStatus(); - return !(snapshotStatus == SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED) - || (isSstFilteringServiceEnabled && !snapInfo.isSstFiltered()); + return snapshotStatus != SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED; } // TODO: Move this util class. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java index fe0f6e111ed..9746b4421b7 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java @@ -176,8 +176,7 @@ public BackgroundTaskResult call() { "unexpected state."); } - SnapshotInfo previousSnapshot = getPreviousActiveSnapshot( - currSnapInfo, snapChainManager, omSnapshotManager); + SnapshotInfo previousSnapshot = getPreviousActiveSnapshot(currSnapInfo, snapChainManager); SnapshotInfo previousToPrevSnapshot = null; Table previousKeyTable = null; @@ -194,8 +193,7 @@ public BackgroundTaskResult call() { .getKeyTable(bucketInfo.getBucketLayout()); prevRenamedTable = omPreviousSnapshot .getMetadataManager().getSnapshotRenamedTable(); - previousToPrevSnapshot = getPreviousActiveSnapshot( - previousSnapshot, snapChainManager, omSnapshotManager); + previousToPrevSnapshot = getPreviousActiveSnapshot(previousSnapshot, snapChainManager); } Table previousToPrevKeyTable = null; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java index 2041fa791a7..e0f40dabd8a 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.exceptions.OMException; @@ -143,7 +142,7 @@ public static void checkSnapshotActive(SnapshotInfo snapInfo, * Get the next non deleted snapshot in the snapshot chain. */ public static SnapshotInfo getNextActiveSnapshot(SnapshotInfo snapInfo, - SnapshotChainManager chainManager, OmSnapshotManager omSnapshotManager) + SnapshotChainManager chainManager, OzoneManager ozoneManager) throws IOException { // If the snapshot is deleted in the previous run, then the in-memory @@ -162,8 +161,7 @@ public static SnapshotInfo getNextActiveSnapshot(SnapshotInfo snapInfo, snapInfo.getSnapshotPath(), snapInfo.getSnapshotId()); String tableKey = chainManager.getTableKey(nextPathSnapshot); - SnapshotInfo nextSnapshotInfo = - omSnapshotManager.getSnapshotInfo(tableKey); + SnapshotInfo nextSnapshotInfo = getSnapshotInfo(ozoneManager, tableKey); if (nextSnapshotInfo.getSnapshotStatus().equals( SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java index 11b6cc26c81..fe98a73ea6c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java @@ -88,6 +88,9 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements // always true, only used in tests private boolean shouldFlushCache = true; + private OMRequest lastRequestToSubmit; + + /** * Constructs an instance of the server handler. * @@ -225,6 +228,7 @@ private OMResponse internalProcessRequest(OMRequest request) throws assert (omClientRequest != null); OMClientRequest finalOmClientRequest = omClientRequest; requestToSubmit = preExecute(finalOmClientRequest); + this.lastRequestToSubmit = requestToSubmit; } catch (IOException ex) { if (omClientRequest != null) { omClientRequest.handleRequestFailure(ozoneManager); @@ -248,6 +252,11 @@ private OMRequest preExecute(OMClientRequest finalOmClientRequest) () -> finalOmClientRequest.preExecute(ozoneManager)); } + @VisibleForTesting + public OMRequest getLastRequestToSubmit() { + return lastRequestToSubmit; + } + /** * Submits request to OM's Ratis server. */ diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/OmTestManagers.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/OmTestManagers.java index 17f575f43b5..f1b9deb9c0b 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/OmTestManagers.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/OmTestManagers.java @@ -17,6 +17,9 @@ package org.apache.hadoop.ozone.om; +import static org.apache.ozone.test.GenericTestUtils.waitFor; + +import java.util.concurrent.TimeoutException; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.HddsWhiteboxTestUtils; @@ -24,9 +27,11 @@ import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenSecretManager; +import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.RaftServerStatus; import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.mockito.Mockito; @@ -43,10 +48,11 @@ public final class OmTestManagers { private KeyManager keyManager; private OMMetadataManager metadataManager; private KeyProviderCryptoExtension kmsProvider; - private VolumeManager volumeManager; - private BucketManager bucketManager; - private PrefixManager prefixManager; - private ScmBlockLocationProtocol scmBlockClient; + private final VolumeManager volumeManager; + private final BucketManager bucketManager; + private final PrefixManager prefixManager; + private final ScmBlockLocationProtocol scmBlockClient; + private final OzoneClient rpcClient; public OzoneManager getOzoneManager() { return om; @@ -72,6 +78,9 @@ public KeyManager getKeyManager() { public ScmBlockLocationProtocol getScmBlockClient() { return scmBlockClient; } + public OzoneClient getRpcClient() { + return rpcClient; + } public OmTestManagers(OzoneConfiguration conf) throws AuthenticationException, IOException { @@ -110,7 +119,17 @@ public OmTestManagers(OzoneConfiguration conf, "secretManager", Mockito.mock(OzoneBlockTokenSecretManager.class)); om.start(); - writeClient = OzoneClientFactory.getRpcClient(conf) + try { + waitFor(() -> om.getOmRatisServer().checkLeaderStatus() == RaftServerStatus.LEADER_AND_READY, + 10, 10_000); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + rpcClient = OzoneClientFactory.getRpcClient(conf); + writeClient = rpcClient .getObjectStore().getClientProxy().getOzoneManagerClient(); metadataManager = (OmMetadataManagerImpl) HddsWhiteboxTestUtils .getInternalState(om, "metadataManager"); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java index 678efabc318..ae32268e5e2 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSstFilteringService.java @@ -35,19 +35,22 @@ import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; -import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.apache.ratis.util.ExitUtils; import org.awaitility.core.ConditionTimeoutException; -import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.TestMethodOrder; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.MethodOrderer.OrderAnnotation; import org.junit.jupiter.api.io.TempDir; import org.rocksdb.LiveFileMetaData; import java.io.File; import java.io.IOException; import java.time.Duration; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -63,6 +66,7 @@ import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL; import static org.awaitility.Awaitility.with; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -71,24 +75,21 @@ /** * Test SST Filtering Service. */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +@TestMethodOrder(OrderAnnotation.class) public class TestSstFilteringService { - public static final String SST_FILE_EXTENSION = ".sst"; - @TempDir - private File folder; + private static final String SST_FILE_EXTENSION = ".sst"; private OzoneManagerProtocol writeClient; private OzoneManager om; private OzoneConfiguration conf; private KeyManager keyManager; + private short countTotalSnapshots = 0; @BeforeAll - public static void setup() { + void setup(@TempDir Path folder) throws Exception { ExitUtils.disableSystemExit(); - } - - @BeforeEach - public void init() throws AuthenticationException, IOException { conf = new OzoneConfiguration(); - conf.set(OZONE_METADATA_DIRS, folder.getAbsolutePath()); + conf.set(OZONE_METADATA_DIRS, folder.toString()); conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 200, TimeUnit.MILLISECONDS); conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 100, @@ -102,7 +103,7 @@ public void init() throws AuthenticationException, IOException { om = omTestManagers.getOzoneManager(); } - @AfterEach + @AfterAll public void cleanup() throws Exception { if (keyManager != null) { keyManager.stop(); @@ -133,6 +134,7 @@ public void cleanup() throws Exception { * @throws IOException - on Failure. */ @Test + @Order(1) public void testIrrelevantSstFileDeletion() throws IOException, InterruptedException { RDBStore activeDbStore = (RDBStore) om.getMetadataManager().getStore(); @@ -142,7 +144,8 @@ public void testIrrelevantSstFileDeletion() final int keyCount = 100; String volumeName = "vol1"; String bucketName1 = "buck1"; - createVolumeAndBucket(volumeName, bucketName1); + createVolume(volumeName); + addBucketToVolume(volumeName, bucketName1); createKeys(volumeName, bucketName1, keyCount / 2); activeDbStore.getDb().flush(OmMetadataManagerImpl.KEY_TABLE); @@ -180,16 +183,16 @@ public void testIrrelevantSstFileDeletion() assertTrue(nonLevel0FilesCountAfterCompact > 0); String bucketName2 = "buck2"; - createVolumeAndBucket(volumeName, bucketName2); + addBucketToVolume(volumeName, bucketName2); createKeys(volumeName, bucketName2, keyCount); activeDbStore.getDb().flush(OmMetadataManagerImpl.KEY_TABLE); List allFiles = activeDbStore.getDb().getSstFileList(); String snapshotName1 = "snapshot1"; - writeClient.createSnapshot(volumeName, bucketName2, snapshotName1); + createSnapshot(volumeName, bucketName2, snapshotName1); SnapshotInfo snapshotInfo = om.getMetadataManager().getSnapshotInfoTable() .get(SnapshotInfo.getTableKey(volumeName, bucketName2, snapshotName1)); - assertFalse(snapshotInfo.isSstFiltered()); + assertFalse(SstFilteringService.isSstFiltered(om.getConfiguration(), snapshotInfo)); with().atMost(Duration.ofSeconds(120)) .pollInterval(Duration.ofSeconds(1)) .await() @@ -220,14 +223,18 @@ public void testIrrelevantSstFileDeletion() } } - assertTrue(snapshotInfo.isSstFiltered()); + // Need to read the sstFiltered flag which is set in background process and + // hence snapshotInfo.isSstFiltered() may not work sometimes. + assertTrue(SstFilteringService.isSstFiltered(om.getConfiguration(), + om.getMetadataManager().getSnapshotInfoTable().get(SnapshotInfo + .getTableKey(volumeName, bucketName2, snapshotName1)))); String snapshotName2 = "snapshot2"; long count; try (BootstrapStateHandler.Lock lock = filteringService.getBootstrapStateLock().lock()) { count = filteringService.getSnapshotFilteredCount().get(); - writeClient.createSnapshot(volumeName, bucketName2, snapshotName2); + createSnapshot(volumeName, bucketName2, snapshotName2); assertThrows(ConditionTimeoutException.class, () -> with() .atMost(Duration.ofSeconds(10)) @@ -252,14 +259,16 @@ public void testIrrelevantSstFileDeletion() } @Test + @Order(2) public void testActiveAndDeletedSnapshotCleanup() throws IOException { RDBStore activeDbStore = (RDBStore) om.getMetadataManager().getStore(); String volumeName = "volume1"; List bucketNames = Arrays.asList("bucket1", "bucket2"); + createVolume(volumeName); // Create 2 Buckets for (String bucketName : bucketNames) { - createVolumeAndBucket(volumeName, bucketName); + addBucketToVolume(volumeName, bucketName); } // Write 25 keys in each bucket, 2 sst files would be generated each for // keys in a single bucket @@ -277,8 +286,8 @@ public void testActiveAndDeletedSnapshotCleanup() throws IOException { keyManager.getSnapshotSstFilteringService(); sstFilteringService.pause(); - writeClient.createSnapshot(volumeName, bucketNames.get(0), "snap1"); - writeClient.createSnapshot(volumeName, bucketNames.get(0), "snap2"); + createSnapshot(volumeName, bucketNames.get(0), "snap1"); + createSnapshot(volumeName, bucketNames.get(0), "snap2"); SnapshotInfo snapshot1Info = om.getMetadataManager().getSnapshotInfoTable() .get(SnapshotInfo.getTableKey(volumeName, bucketNames.get(0), "snap1")); @@ -297,17 +306,17 @@ public void testActiveAndDeletedSnapshotCleanup() throws IOException { .await().until(() -> snap1Current.exists() && snap2Current.exists()); long snap1SstFileCountBeforeFilter = Arrays.stream(snapshot1Dir.listFiles()) - .filter(f -> f.getName().endsWith(".sst")).count(); + .filter(f -> f.getName().endsWith(SST_FILE_EXTENSION)).count(); long snap2SstFileCountBeforeFilter = Arrays.stream(snapshot2Dir.listFiles()) - .filter(f -> f.getName().endsWith(".sst")).count(); + .filter(f -> f.getName().endsWith(SST_FILE_EXTENSION)).count(); // delete snap1 - writeClient.deleteSnapshot(volumeName, bucketNames.get(0), "snap1"); + deleteSnapshot(volumeName, bucketNames.get(0), "snap1"); sstFilteringService.resume(); // Filtering service will only act on snap2 as it is an active snaphot with().atMost(Duration.ofSeconds(10)).pollInterval(Duration.ofSeconds(1)) .await() - .until(() -> sstFilteringService.getSnapshotFilteredCount().get() >= 2); + .until(() -> sstFilteringService.getSnapshotFilteredCount().get() >= countTotalSnapshots); long snap1SstFileCountAfterFilter = Arrays.stream(snapshot1Dir.listFiles()) .filter(f -> f.getName().endsWith(SST_FILE_EXTENSION)).count(); long snap2SstFileCountAfterFilter = Arrays.stream(snapshot2Dir.listFiles()) @@ -315,10 +324,12 @@ public void testActiveAndDeletedSnapshotCleanup() throws IOException { // one sst will be filtered in both active but not in deleted snapshot // as sstFiltering svc won't run on already deleted snapshots but will mark // it as filtered. - assertEquals(2, sstFilteringService.getSnapshotFilteredCount().get()); + assertEquals(countTotalSnapshots, sstFilteringService.getSnapshotFilteredCount().get()); assertEquals(snap1SstFileCountBeforeFilter, snap1SstFileCountAfterFilter); - assertEquals(snap2SstFileCountBeforeFilter - 1, - snap2SstFileCountAfterFilter); + // If method with order 1 is run .sst file from /vol1/buck1 and /vol1/buck2 will be deleted. + // As part of this method .sst file from /volume1/bucket2/ will be deleted. + // sstFiltering won't run on deleted snapshots in /volume1/bucket1. + assertThat(snap2SstFileCountBeforeFilter).isGreaterThan(snap2SstFileCountAfterFilter); } private void createKeys(String volumeName, @@ -331,8 +342,7 @@ private void createKeys(String volumeName, } } - private void createVolumeAndBucket(String volumeName, - String bucketName) + private void createVolume(String volumeName) throws IOException { OMRequestTestUtils.addVolumeToOM(keyManager.getMetadataManager(), OmVolumeArgs.newBuilder() @@ -340,7 +350,10 @@ private void createVolumeAndBucket(String volumeName, .setAdminName("a") .setVolume(volumeName) .build()); + } + private void addBucketToVolume(String volumeName, String bucketName) + throws IOException { OMRequestTestUtils.addBucketToOM(keyManager.getMetadataManager(), OmBucketInfo.newBuilder().setVolumeName(volumeName) .setBucketName(bucketName) @@ -381,13 +394,15 @@ private void createKey(OzoneManagerProtocol managerProtocol, * snapshot bucket. */ @Test + @Order(3) public void testSstFilteringService() throws IOException { RDBStore activeDbStore = (RDBStore) om.getMetadataManager().getStore(); String volumeName = "volume"; List bucketNames = Arrays.asList("bucket", "bucket1", "bucket2"); + createVolume(volumeName); for (String bucketName : bucketNames) { - createVolumeAndBucket(volumeName, bucketName); + addBucketToVolume(volumeName, bucketName); } int keyCount = 150; @@ -422,8 +437,7 @@ public void testSstFilteringService() throws IOException { List snapshotNames = Arrays.asList("snap", "snap-1", "snap-2"); for (int i = 0; i < 3; i++) { - writeClient.createSnapshot(volumeName, bucketNames.get(i), - snapshotNames.get(i)); + createSnapshot(volumeName, bucketNames.get(i), snapshotNames.get(i)); } SstFilteringService sstFilteringService = @@ -432,8 +446,8 @@ public void testSstFilteringService() throws IOException { with().atMost(Duration.ofSeconds(10)) .pollInterval(Duration.ofSeconds(1)) .await() - .until(() -> sstFilteringService.getSnapshotFilteredCount().get() >= 3); - assertEquals(3, sstFilteringService.getSnapshotFilteredCount().get()); + .until(() -> sstFilteringService.getSnapshotFilteredCount().get() >= countTotalSnapshots); + assertEquals(countTotalSnapshots, sstFilteringService.getSnapshotFilteredCount().get()); Set keyInBucketAfterFilteringRun = getKeysFromSnapshot(volumeName, bucketNames.get(0), @@ -483,4 +497,14 @@ private Set getKeysFromSnapshot(String volume, return getKeysFromDb(omSnapshot.getMetadataManager(), volume, bucket); } } + + private void createSnapshot(String volumeName, String bucketName, String snapshotName) throws IOException { + writeClient.createSnapshot(volumeName, bucketName, snapshotName); + countTotalSnapshots++; + } + + private void deleteSnapshot(String volumeName, String bucketName, String snapshotName) throws IOException { + writeClient.deleteSnapshot(volumeName, bucketName, snapshotName); + countTotalSnapshots--; + } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java index 49ea3213886..26852e1642a 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java @@ -209,17 +209,8 @@ public void testKeyPurgeInSnapshot() throws Exception { deletedKey)); } - SnapshotInfo fromSnapshotInfo = new SnapshotInfo.Builder() - .setVolumeName(volumeName) - .setBucketName(bucketName) - .setName("snap1") - .build(); - - ReferenceCounted rcOmSnapshot = - ozoneManager.getOmSnapshotManager().getSnapshot( - fromSnapshotInfo.getVolumeName(), - fromSnapshotInfo.getBucketName(), - fromSnapshotInfo.getName()); + ReferenceCounted rcOmSnapshot = ozoneManager.getOmSnapshotManager() + .getSnapshot(snapInfo.getVolumeName(), snapInfo.getBucketName(), snapInfo.getName()); OmSnapshot omSnapshot = rcOmSnapshot.get(); // The keys should be present in the snapshot's deletedTable @@ -247,8 +238,7 @@ public void testKeyPurgeInSnapshot() throws Exception { try (BatchOperation batchOperation = omMetadataManager.getStore().initBatchOperation()) { - OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse( - omResponse, deletedKeyNames, fromSnapshotInfo, null); + OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse(omResponse, deletedKeyNames, snapInfo, null); omKeyPurgeResponse.addToDBBatch(omMetadataManager, batchOperation); // Do manual commit and see whether addToBatch is successful or not. 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 79a06a78bca..d3d61f9c36a 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 @@ -158,7 +158,7 @@ public void checkIfDeleteServiceIsDeletingKeys() GenericTestUtils.waitFor( () -> keyDeletingService.getDeletedKeyCount().get() >= keyCount, 1000, 10000); - Assertions.assertTrue(keyDeletingService.getRunCount().get() > 1); + Assertions.assertTrue(keyDeletingService.getRunCount().get() >= 1); Assertions.assertEquals(0, keyManager.getPendingDeletionKeys( Integer.MAX_VALUE).getKeyBlocksList().size()); } @@ -360,7 +360,7 @@ public void checkDeletionForKeysWithMultipleVersions() GenericTestUtils.waitFor( () -> keyDeletingService.getDeletedKeyCount().get() >= 1, 1000, 10000); - Assertions.assertTrue(keyDeletingService.getRunCount().get() > 1); + Assertions.assertTrue(keyDeletingService.getRunCount().get() >= 1); Assertions.assertEquals(0, keyManager.getPendingDeletionKeys( Integer.MAX_VALUE).getKeyBlocksList().size()); @@ -439,7 +439,7 @@ public void checkDeletedTableCleanUpForSnapshot() GenericTestUtils.waitFor( () -> keyDeletingService.getDeletedKeyCount().get() >= 1, 1000, 10000); - Assertions.assertTrue(keyDeletingService.getRunCount().get() > 1); + Assertions.assertTrue(keyDeletingService.getRunCount().get() >= 1); Assertions.assertEquals(0, keyManager .getPendingDeletionKeys(Integer.MAX_VALUE).getKeyBlocksList().size()); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java new file mode 100644 index 00000000000..3948f4fab80 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java @@ -0,0 +1,97 @@ +/* + * 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.service; + + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; +import org.apache.hadoop.ozone.om.KeyManagerImpl; +import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; +import java.time.Duration; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Class to unit test SnapshotDeletingService. + */ +@ExtendWith(MockitoExtension.class) +public class TestSnapshotDeletingService { + @Mock + private OzoneManager ozoneManager; + @Mock + private KeyManagerImpl keyManager; + @Mock + private OmSnapshotManager omSnapshotManager; + @Mock + private SnapshotChainManager chainManager; + @Mock + private OmMetadataManagerImpl omMetadataManager; + @Mock + private ScmBlockLocationProtocol scmClient; + private final OzoneConfiguration conf = new OzoneConfiguration();; + private final long sdsRunInterval = Duration.ofMillis(1000).toMillis(); + private final long sdsServiceTimeout = Duration.ofSeconds(10).toMillis(); + + + private static Stream testCasesForIgnoreSnapshotGc() { + SnapshotInfo filteredSnapshot = SnapshotInfo.newBuilder().setSstFiltered(true).setName("snap1").build(); + SnapshotInfo unFilteredSnapshot = SnapshotInfo.newBuilder().setSstFiltered(false).setName("snap1").build(); + return Stream.of( + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false), + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true), + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true), + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true)); + } + + @ParameterizedTest + @MethodSource("testCasesForIgnoreSnapshotGc") + public void testProcessSnapshotLogicInSDS(SnapshotInfo snapshotInfo, + SnapshotInfo.SnapshotStatus status, + boolean expectedOutcome) + throws IOException { + Mockito.when(omMetadataManager.getSnapshotChainManager()).thenReturn(chainManager); + Mockito.when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager); + Mockito.when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); + Mockito.when(ozoneManager.getConfiguration()).thenReturn(conf); + + SnapshotDeletingService snapshotDeletingService = + new SnapshotDeletingService(sdsRunInterval, sdsServiceTimeout, ozoneManager, scmClient); + + snapshotInfo.setSnapshotStatus(status); + assertEquals(expectedOutcome, snapshotDeletingService.shouldIgnoreSnapshot(snapshotInfo)); + } +} diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java index da182730bc8..7e2483e574b 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java @@ -18,21 +18,15 @@ package org.apache.hadoop.ozone.om.snapshot; -import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; -import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.nio.file.Files; import java.nio.file.Path; import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.Stream; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.getINode; @@ -84,42 +78,4 @@ public void testLinkFiles(@TempDir File tempDir) throws Exception { assertEquals(tree1Files, tree2Files); GenericTestUtils.deleteDirectory(tempDir); } - - - private static Stream testCasesForIgnoreSnapshotGc() { - SnapshotInfo filteredSnapshot = - SnapshotInfo.newBuilder().setSstFiltered(true).setName("snap1").build(); - SnapshotInfo unFilteredSnapshot = - SnapshotInfo.newBuilder().setSstFiltered(false).setName("snap1") - .build(); - // {IsSnapshotFiltered,isSnapshotDeleted,IsSstServiceEnabled = ShouldIgnore} - return Stream.of(Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, true, false), - Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true, true), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, true, true), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true, true), - Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false, false), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false, false), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, false, true), - Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, false, true)); - } - - @ParameterizedTest - @MethodSource("testCasesForIgnoreSnapshotGc") - public void testProcessSnapshotLogicInSDS(SnapshotInfo snapshotInfo, - SnapshotInfo.SnapshotStatus status, boolean isSstFilteringSvcEnabled, - boolean expectedOutcome) { - snapshotInfo.setSnapshotStatus(status); - assertEquals(expectedOutcome, - SnapshotDeletingService.shouldIgnoreSnapshot(snapshotInfo, - isSstFilteringSvcEnabled)); - } - } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java index b70704ce2c6..ffc5f66316b 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java @@ -361,7 +361,6 @@ public void init() throws RocksDBException, IOException, ExecutionException { omSnapshotManager = mock(OmSnapshotManager.class); when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager); - when(omSnapshotManager.isSnapshotStatus(any(), any())).thenReturn(true); SnapshotCache snapshotCache = new SnapshotCache(mockCacheLoader(), 10, omMetrics, 0); when(omSnapshotManager.getActiveSnapshot(anyString(), anyString(), anyString())) 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 53dc2110465..cb557bd53c7 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 @@ -175,18 +175,24 @@ public BasicOzoneClientAdapterImpl(String omHost, int omPort, OzoneClientFactory.getRpcClient(conf); } objectStore = ozoneClient.getObjectStore(); - this.volume = objectStore.getVolume(volumeStr); - this.bucket = volume.getBucket(bucketStr); - bucketReplicationConfig = this.bucket.getReplicationConfig(); - nextReplicationConfigRefreshTime = - clock.millis() + bucketRepConfigRefreshPeriodMS; - - // resolve the bucket layout in case of Link Bucket - BucketLayout resolvedBucketLayout = - OzoneClientUtils.resolveLinkBucketLayout(bucket, objectStore, - new HashSet<>()); - - OzoneFSUtils.validateBucketLayout(bucket.getName(), resolvedBucketLayout); + try { + this.volume = objectStore.getVolume(volumeStr); + this.bucket = volume.getBucket(bucketStr); + bucketReplicationConfig = this.bucket.getReplicationConfig(); + nextReplicationConfigRefreshTime = clock.millis() + bucketRepConfigRefreshPeriodMS; + + // resolve the bucket layout in case of Link Bucket + BucketLayout resolvedBucketLayout = + OzoneClientUtils.resolveLinkBucketLayout(bucket, objectStore, new HashSet<>()); + + OzoneFSUtils.validateBucketLayout(bucket.getName(), resolvedBucketLayout); + } catch (IOException | RuntimeException exception) { + // in case of exception, the adapter object will not be + // initialised making the client object unreachable, close the client + // to release resources in this case and rethrow. + ozoneClient.close(); + throw exception; + } this.configuredDnPort = conf.getInt( OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT, 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 b650310ef1d..4d989571714 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 @@ -200,7 +200,15 @@ public BasicRootedOzoneClientAdapterImpl(String omHost, int omPort, OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT); // Fetches the bucket layout to be used by OFS. - initDefaultFsBucketLayout(conf); + try { + initDefaultFsBucketLayout(conf); + } catch (IOException | RuntimeException exception) { + // in case of exception, the adapter object will not be + // initialised making the client object unreachable, close the client + // to release resources in this case and rethrow. + ozoneClient.close(); + throw exception; + } config = conf; } finally { diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml index bb107b08da0..b71156b2204 100644 --- a/hadoop-ozone/pom.xml +++ b/hadoop-ozone/pom.xml @@ -382,6 +382,29 @@ + + org.apache.maven.plugins + maven-remote-resources-plugin + + + org.apache.ozone:ozone-dev-support:${ozone.version} + + + + + org.apache.ozone + ozone-dev-support + ${ozone.version} + + + + + + process + + + + diff --git a/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java b/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java index 43e2d728b76..7c293ff1861 100644 --- a/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java +++ b/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java @@ -51,7 +51,8 @@ public enum UnHealthyContainerStates { UNDER_REPLICATED, OVER_REPLICATED, MIS_REPLICATED, - ALL_REPLICAS_UNHEALTHY + ALL_REPLICAS_BAD, + NEGATIVE_SIZE // Added new state to track containers with negative sizes } private static final String CONTAINER_ID = "container_id"; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java index ef40329c80c..b22e0b870ac 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java @@ -50,6 +50,12 @@ public class UnhealthyContainersResponse { @JsonProperty("misReplicatedCount") private long misReplicatedCount = 0; + /** + * Total count of containers with negative size. + */ + @JsonProperty("negativeSizeCount") + private long negativeSizeCount = 0; + /** * A collection of unhealthy containers. */ @@ -73,6 +79,9 @@ public void setSummaryCount(String state, long count) { } else if (state.equals( UnHealthyContainerStates.MIS_REPLICATED.toString())) { this.misReplicatedCount = count; + } else if (state.equals( + UnHealthyContainerStates.NEGATIVE_SIZE.toString())) { + this.negativeSizeCount = count; } } @@ -92,6 +101,10 @@ public long getMisReplicatedCount() { return misReplicatedCount; } + public long getNegativeSizeCount() { + return negativeSizeCount; + } + public Collection getContainers() { return containers; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java index bb93923cfd1..f9fb3d7998d 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java @@ -53,6 +53,7 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_COUNT; +import static org.apache.hadoop.ozone.recon.ReconConstants.DEFAULT_FETCH_COUNT; import static org.apache.hadoop.ozone.recon.ReconConstants.TOTAL_KEYS; import static org.apache.hadoop.ozone.recon.ReconConstants.TOTAL_USED_BYTES; @@ -65,6 +66,7 @@ public class ContainerHealthTask extends ReconScmTask { private static final Logger LOG = LoggerFactory.getLogger(ContainerHealthTask.class); + public static final int FETCH_COUNT = Integer.parseInt(DEFAULT_FETCH_COUNT); private ReadWriteLock lock = new ReentrantReadWriteLock(true); @@ -131,8 +133,24 @@ public void triggerContainerHealthCheck() { LOG.info("Container Health task thread took {} milliseconds to" + " process {} existing database records.", Time.monotonicNow() - start, existingCount); + + checkAndProcessContainers(unhealthyContainerStateStatsMap, currentTime); + processedContainers.clear(); + } finally { + lock.writeLock().unlock(); + } + } + + private void checkAndProcessContainers( + Map> + unhealthyContainerStateStatsMap, long currentTime) { + ContainerID startID = ContainerID.valueOf(1); + List containers = containerManager.getContainers(startID, + FETCH_COUNT); + long start; + long iterationCount = 0; + while (!containers.isEmpty()) { start = Time.monotonicNow(); - final List containers = containerManager.getContainers(); containers.stream() .filter(c -> !processedContainers.contains(c)) .forEach(c -> processContainer(c, currentTime, @@ -142,10 +160,19 @@ public void triggerContainerHealthCheck() { " processing {} containers.", Time.monotonicNow() - start, containers.size()); logUnhealthyContainerStats(unhealthyContainerStateStatsMap); - processedContainers.clear(); - } finally { - lock.writeLock().unlock(); + if (containers.size() >= FETCH_COUNT) { + startID = ContainerID.valueOf( + containers.get(containers.size() - 1).getContainerID() + 1); + containers = containerManager.getContainers(startID, FETCH_COUNT); + } else { + containers.clear(); + } + iterationCount++; } + LOG.info( + "Container Health task thread took {} iterations to fetch all " + + "containers using batched approach with batch size of {}", + iterationCount, FETCH_COUNT); } private void logUnhealthyContainerStats( @@ -184,6 +211,8 @@ private void initializeUnhealthyContainerStateStatsMap( UnHealthyContainerStates.OVER_REPLICATED, new HashMap<>()); unhealthyContainerStateStatsMap.put( UnHealthyContainerStates.MIS_REPLICATED, new HashMap<>()); + unhealthyContainerStateStatsMap.put( + UnHealthyContainerStates.NEGATIVE_SIZE, new HashMap<>()); } private ContainerHealthStatus setCurrentContainer(long recordId) @@ -280,12 +309,20 @@ private long processExistingDBRecords(long currentTime, private void processContainer(ContainerInfo container, long currentTime, Map> - unhealthyContainerStateStatsMap) { + unhealthyContainerStateStatsMap) { try { Set containerReplicas = containerManager.getContainerReplicas(container.containerID()); ContainerHealthStatus h = new ContainerHealthStatus(container, containerReplicas, placementPolicy, reconContainerMetadataManager); + + // Handle negative sized containers separately + if (h.getContainer().getUsedBytes() < 0) { + handleNegativeSizedContainers(h, currentTime, + unhealthyContainerStateStatsMap); + return; + } + if (h.isHealthy() || h.isDeleted()) { return; } @@ -331,6 +368,32 @@ private boolean containerDeletedInSCM(ContainerInfo containerInfo) { return false; } + /** + * This method is used to handle containers with negative sizes. It logs an + * error message and inserts a record into the UNHEALTHY_CONTAINERS table. + * @param containerHealthStatus + * @param currentTime + * @param unhealthyContainerStateStatsMap + */ + private void handleNegativeSizedContainers( + ContainerHealthStatus containerHealthStatus, long currentTime, + Map> + unhealthyContainerStateStatsMap) { + ContainerInfo container = containerHealthStatus.getContainer(); + LOG.error( + "Container {} has negative size. Please visit Recon's unhealthy " + + "container endpoint for more details.", + container.getContainerID()); + UnhealthyContainers record = + ContainerHealthRecords.recordForState(containerHealthStatus, + UnHealthyContainerStates.NEGATIVE_SIZE, currentTime); + List records = Collections.singletonList(record); + populateContainerStats(containerHealthStatus, + UnHealthyContainerStates.NEGATIVE_SIZE, + unhealthyContainerStateStatsMap); + containerHealthSchemaManager.insertUnhealthyContainerRecords(records); + } + /** * Helper methods to generate and update the required database records for * unhealthy containers. diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/ContainerHealthSchemaManager.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/ContainerHealthSchemaManager.java index 364aff103a5..4e7765956f4 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/ContainerHealthSchemaManager.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/persistence/ContainerHealthSchemaManager.java @@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.recon.persistence; import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.UNDER_REPLICATED; -import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.ALL_REPLICAS_UNHEALTHY; +import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.ALL_REPLICAS_BAD; import static org.hadoop.ozone.recon.schema.tables.UnhealthyContainersTable.UNHEALTHY_CONTAINERS; import static org.jooq.impl.DSL.count; @@ -71,7 +71,7 @@ public List getUnhealthyContainers( SelectQuery query = dslContext.selectQuery(); query.addFrom(UNHEALTHY_CONTAINERS); if (state != null) { - if (state.equals(ALL_REPLICAS_UNHEALTHY)) { + if (state.equals(ALL_REPLICAS_BAD)) { query.addConditions(UNHEALTHY_CONTAINERS.CONTAINER_STATE .eq(UNDER_REPLICATED.toString())); query.addConditions(UNHEALTHY_CONTAINERS.ACTUAL_REPLICA_COUNT.eq(0)); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java index fb387861f0e..105406f2bdf 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java @@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.recon.ReconUtils; import org.apache.hadoop.ozone.recon.scm.ReconScmTask; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; +import org.hadoop.ozone.recon.schema.ContainerSchemaDefinition; import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition; import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao; import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao; @@ -34,13 +35,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.ArrayList; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED; import static org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE; @@ -60,6 +62,8 @@ public class ContainerSizeCountTask extends ReconScmTask { private ContainerCountBySizeDao containerCountBySizeDao; private DSLContext dslContext; private HashMap processedContainers = new HashMap<>(); + private Map> + unhealthyContainerStateStatsMap; private ReadWriteLock lock = new ReentrantReadWriteLock(true); public ContainerSizeCountTask( @@ -121,7 +125,17 @@ protected synchronized void run() { private void process(ContainerInfo container, Map map) { final ContainerID id = container.containerID(); - final long currentSize = container.getUsedBytes(); + final long usedBytes = container.getUsedBytes(); + final long currentSize; + + if (usedBytes < 0) { + LOG.warn("Negative usedBytes ({}) for container {}, treating it as 0", + usedBytes, id); + currentSize = 0; + } else { + currentSize = usedBytes; + } + final Long previousSize = processedContainers.put(id, currentSize); if (previousSize != null) { decrementContainerSizeCount(previousSize, map); @@ -132,24 +146,27 @@ private void process(ContainerInfo container, /** * The process() function is responsible for updating the counts of * containers being tracked in a containerSizeCountMap based on the - * ContainerInfo objects in the list containers.It then iterates through + * ContainerInfo objects in the list containers. It then iterates through * the list of containers and does the following for each container: * - * 1) If the container is not present in processedContainers, - * it is a new container, so it is added to the processedContainers map - * and the count for its size in the containerSizeCountMap is incremented - * by 1 using the handlePutKeyEvent() function. - * 2) If the container is present in processedContainers but its size has - * been updated to the new size then the count for the old size in the - * containerSizeCountMap is decremented by 1 using the - * handleDeleteKeyEvent() function. The count for the new size is then - * incremented by 1 using the handlePutKeyEvent() function. - * 3) If the container is not present in containers list, it means the - * container has been deleted. - * The remaining containers inside the deletedContainers map are the ones - * that are not in the cluster and need to be deleted. Finally, the counts in - * the containerSizeCountMap are written to the database using the - * writeCountsToDB() function. + * 1) If the container's state is not "deleted," it will be processed: + * - If the container is not present in processedContainers, it is a new + * container. Therefore, it is added to the processedContainers map, and + * the count for its size in the containerSizeCountMap is incremented by + * 1 using the handlePutKeyEvent() function. + * - If the container is present in processedContainers but its size has + * been updated to a new size, the count for the old size in the + * containerSizeCountMap is decremented by 1 using the + * handleDeleteKeyEvent() function. Subsequently, the count for the new + * size is incremented by 1 using the handlePutKeyEvent() function. + * + * 2) If the container's state is "deleted," it is skipped, as deleted + * containers are not processed. + * + * After processing, the remaining containers inside the deletedContainers map + * are those that are not in the cluster and need to be deleted from the total + * size counts. Finally, the counts in the containerSizeCountMap are written + * to the database using the writeCountsToDB() function. */ public void process(List containers) { lock.writeLock().lock(); @@ -161,7 +178,9 @@ public void process(List containers) { // Loop to handle container create and size-update operations for (ContainerInfo container : containers) { - // The containers present in the cache hence it is not yet deleted + if (container.getState().equals(DELETED)) { + continue; // Skip deleted containers + } deletedContainers.remove(container.containerID()); // For New Container being created try { @@ -246,10 +265,10 @@ public String getTaskName() { /** * - * The handleContainerDeleteOperations() function loops through the entries - * in the deletedContainers map and calls the handleDeleteKeyEvent() function - * for each one. This will decrement the size counts of those containers by - * one which are no longer present in the cluster + * Handles the deletion of containers by updating the tracking of processed containers + * and adjusting the count of containers based on their sizes. When a container is deleted, + * it is removed from the tracking of processed containers, and the count of containers + * corresponding to its size is decremented in the container size count map. * * Used by process() * @@ -261,6 +280,9 @@ private void handleContainerDeleteOperations( Map containerSizeCountMap) { for (Map.Entry containerId : deletedContainers.entrySet()) { + // processedContainers will only keep a track of all containers that have + // been processed except DELETED containers. + processedContainers.remove(containerId.getKey()); long containerSize = deletedContainers.get(containerId.getKey()); decrementContainerSizeCount(containerSize, containerSizeCountMap); } @@ -316,19 +338,26 @@ private static void updateContainerSizeCount(long containerSize, int delta, } /** - * * The purpose of this function is to categorize containers into different * size ranges, or "bins," based on their size. * The ContainerSizeCountKey object is used to store the upper bound value * for each size range, and is later used to lookup the count of containers * in that size range within a Map. * - * Used by decrementContainerSizeCount() and incrementContainerSizeCount() + * If the container size is 0, the method sets the size of + * ContainerSizeCountKey as zero without calculating the upper bound. Used by + * decrementContainerSizeCount() and incrementContainerSizeCount() * * @param containerSize to calculate the upperSizeBound */ private static ContainerSizeCountKey getContainerSizeCountKey( long containerSize) { + // If containerSize is 0, return a ContainerSizeCountKey with size 0 + if (containerSize == 0) { + return new ContainerSizeCountKey(0L); + } + + // Otherwise, calculate the upperSizeBound return new ContainerSizeCountKey( ReconUtils.getContainerSizeUpperBound(containerSize)); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java index 7ac29e5d0f0..15245bc7773 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java @@ -855,10 +855,12 @@ public void testGetContainerCounts() throws Exception { ContainerInfo omContainerInfo1 = mock(ContainerInfo.class); given(omContainerInfo1.containerID()).willReturn(new ContainerID(1)); given(omContainerInfo1.getUsedBytes()).willReturn(1500000000L); // 1.5GB + given(omContainerInfo1.getState()).willReturn(LifeCycleState.OPEN); ContainerInfo omContainerInfo2 = mock(ContainerInfo.class); given(omContainerInfo2.containerID()).willReturn(new ContainerID(2)); given(omContainerInfo2.getUsedBytes()).willReturn(2500000000L); // 2.5GB + given(omContainerInfo2.getState()).willReturn(LifeCycleState.OPEN); // Create a list of container info objects List containers = new ArrayList<>(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java index 847b5d98c7e..4d391feaae2 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java @@ -18,9 +18,15 @@ package org.apache.hadoop.ozone.recon.fsck; -import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.ALL_REPLICAS_UNHEALTHY; +import static org.assertj.core.api.Assertions.assertThat; +import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.ALL_REPLICAS_BAD; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -96,7 +102,8 @@ public void testRun() throws Exception { List mockContainers = getMockContainers(7); when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock); when(scmMock.getContainerManager()).thenReturn(containerManagerMock); - when(containerManagerMock.getContainers()).thenReturn(mockContainers); + when(containerManagerMock.getContainers(any(ContainerID.class), + anyInt())).thenReturn(mockContainers); for (ContainerInfo c : mockContainers) { when(containerManagerMock.getContainer(c.containerID())).thenReturn(c); when(scmClientMock.getContainerWithPipeline(c.getContainerID())) @@ -151,7 +158,7 @@ public void testRun() throws Exception { reconTaskStatusDao, containerHealthSchemaManager, placementMock, reconTaskConfig, reconContainerMetadataManager); containerHealthTask.start(); - LambdaTestUtils.await(6000, 1000, () -> + LambdaTestUtils.await(60000, 1000, () -> (unHealthyContainersTableHandle.count() == 6)); UnhealthyContainers rec = unHealthyContainersTableHandle.fetchByContainerId(1L).get(0); @@ -164,7 +171,7 @@ public void testRun() throws Exception { List unhealthyContainers = containerHealthSchemaManager.getUnhealthyContainers( - ALL_REPLICAS_UNHEALTHY, 0, Integer.MAX_VALUE); + ALL_REPLICAS_BAD, 0, Integer.MAX_VALUE); assertEquals(1, unhealthyContainers.size()); assertEquals(2L, unhealthyContainers.get(0).getContainerId().longValue()); @@ -268,7 +275,8 @@ public void testDeletedContainer() throws Exception { List mockContainers = getMockContainers(3); when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock); when(scmMock.getContainerManager()).thenReturn(containerManagerMock); - when(containerManagerMock.getContainers()).thenReturn(mockContainers); + when(containerManagerMock.getContainers(any(ContainerID.class), + anyInt())).thenReturn(mockContainers); for (ContainerInfo c : mockContainers) { when(containerManagerMock.getContainer(c.containerID())).thenReturn(c); when(scmClientMock.getContainerWithPipeline(c.getContainerID())) @@ -332,6 +340,149 @@ public void testDeletedContainer() throws Exception { currentTime); } + @Test + public void testAllContainerStateInsertions() { + UnhealthyContainersDao unHealthyContainersTableHandle = + getDao(UnhealthyContainersDao.class); + + ContainerHealthSchemaManager containerHealthSchemaManager = + new ContainerHealthSchemaManager( + getSchemaDefinition(ContainerSchemaDefinition.class), + unHealthyContainersTableHandle); + + // Iterate through each state in the UnHealthyContainerStates enum + for (ContainerSchemaDefinition.UnHealthyContainerStates state : + ContainerSchemaDefinition.UnHealthyContainerStates.values()) { + + // Create a dummy UnhealthyContainer record with the current state + UnhealthyContainers unhealthyContainer = new UnhealthyContainers(); + unhealthyContainer.setContainerId(state.ordinal() + 1L); + + // Set replica counts based on the state + switch (state) { + case MISSING: + case EMPTY_MISSING: + unhealthyContainer.setExpectedReplicaCount(3); + unhealthyContainer.setActualReplicaCount(0); + unhealthyContainer.setReplicaDelta(3); + break; + + case UNDER_REPLICATED: + unhealthyContainer.setExpectedReplicaCount(3); + unhealthyContainer.setActualReplicaCount(1); + unhealthyContainer.setReplicaDelta(2); + break; + + case OVER_REPLICATED: + unhealthyContainer.setExpectedReplicaCount(3); + unhealthyContainer.setActualReplicaCount(4); + unhealthyContainer.setReplicaDelta(-1); + break; + + case MIS_REPLICATED: + case NEGATIVE_SIZE: + unhealthyContainer.setExpectedReplicaCount(3); + unhealthyContainer.setActualReplicaCount(3); + unhealthyContainer.setReplicaDelta(0); + break; + + case ALL_REPLICAS_BAD: + unhealthyContainer.setExpectedReplicaCount(3); + unhealthyContainer.setActualReplicaCount(0); + unhealthyContainer.setReplicaDelta(3); + break; + + default: + fail("Unhandled state: " + state.name() + ". Please add this state to the switch case."); + } + + unhealthyContainer.setContainerState(state.name()); + unhealthyContainer.setInStateSince(System.currentTimeMillis()); + + // Try inserting the record and catch any exception that occurs + Exception exception = null; + try { + containerHealthSchemaManager.insertUnhealthyContainerRecords( + Collections.singletonList(unhealthyContainer)); + } catch (Exception e) { + exception = e; + } + + // Assert no exception should be thrown for each state + assertNull(exception, + "Exception was thrown during insertion for state " + state.name() + + ": " + exception); + + // Optionally, verify the record was inserted correctly + List insertedRecords = + unHealthyContainersTableHandle.fetchByContainerId( + state.ordinal() + 1L); + assertFalse(insertedRecords.isEmpty(), + "Record was not inserted for state " + state.name() + "."); + assertEquals(insertedRecords.get(0).getContainerState(), state.name(), + "The inserted container state does not match for state " + + state.name() + "."); + } + } + + @Test + public void testNegativeSizeContainers() throws Exception { + // Setup mock objects and test environment + UnhealthyContainersDao unhealthyContainersDao = + getDao(UnhealthyContainersDao.class); + ContainerHealthSchemaManager containerHealthSchemaManager = + new ContainerHealthSchemaManager( + getSchemaDefinition(ContainerSchemaDefinition.class), + unhealthyContainersDao); + ReconStorageContainerManagerFacade scmMock = + mock(ReconStorageContainerManagerFacade.class); + ContainerManager containerManagerMock = mock(ContainerManager.class); + StorageContainerServiceProvider scmClientMock = + mock(StorageContainerServiceProvider.class); + ReconContainerMetadataManager reconContainerMetadataManager = + mock(ReconContainerMetadataManager.class); + MockPlacementPolicy placementMock = new MockPlacementPolicy(); + + // Mock container info setup + List mockContainers = getMockContainers(3); + when(scmMock.getContainerManager()).thenReturn(containerManagerMock); + when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock); + when(containerManagerMock.getContainers(any(ContainerID.class), + anyInt())).thenReturn(mockContainers); + for (ContainerInfo c : mockContainers) { + when(containerManagerMock.getContainer( + c.containerID())).thenReturn(c); + when(scmClientMock.getContainerWithPipeline( + c.getContainerID())).thenReturn(new ContainerWithPipeline(c, null)); + when(containerManagerMock.getContainer(c.containerID()) + .getUsedBytes()).thenReturn(Long.valueOf(-10)); + } + + // Verify the table is initially empty + assertThat(unhealthyContainersDao.findAll()).isEmpty(); + + // Setup and start the container health task + ReconTaskStatusDao reconTaskStatusDao = getDao(ReconTaskStatusDao.class); + ReconTaskConfig reconTaskConfig = new ReconTaskConfig(); + reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(2)); + ContainerHealthTask containerHealthTask = new ContainerHealthTask( + scmMock.getContainerManager(), scmMock.getScmServiceProvider(), + reconTaskStatusDao, + containerHealthSchemaManager, placementMock, reconTaskConfig, + reconContainerMetadataManager); + containerHealthTask.start(); + + // Wait for the task to identify unhealthy containers + LambdaTestUtils.await(6000, 1000, + () -> unhealthyContainersDao.count() == 3); + + // Assert that all unhealthy containers have been identified as NEGATIVE_SIZE states + List negativeSizeContainers = + unhealthyContainersDao.fetchByContainerState("NEGATIVE_SIZE"); + assertThat(negativeSizeContainers).hasSize(3); + } + + private Set getMockReplicas( long containerId, State...states) { Set replicas = new HashSet<>(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java index 9b5dc3bc9bc..573947bda3b 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java @@ -18,6 +18,11 @@ package org.apache.hadoop.ozone.recon.tasks; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.OPEN; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.QUASI_CLOSED; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSING; import static org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.BDDMockito.given; @@ -80,18 +85,21 @@ public void setUp() { @Test public void testProcess() { // mock a container with invalid used bytes - final ContainerInfo omContainerInfo0 = mock(ContainerInfo.class); + ContainerInfo omContainerInfo0 = mock(ContainerInfo.class); given(omContainerInfo0.containerID()).willReturn(new ContainerID(0)); given(omContainerInfo0.getUsedBytes()).willReturn(-1L); + given(omContainerInfo0.getState()).willReturn(OPEN); // Write 2 keys ContainerInfo omContainerInfo1 = mock(ContainerInfo.class); given(omContainerInfo1.containerID()).willReturn(new ContainerID(1)); given(omContainerInfo1.getUsedBytes()).willReturn(1500000000L); // 1.5GB + given(omContainerInfo1.getState()).willReturn(CLOSED); ContainerInfo omContainerInfo2 = mock(ContainerInfo.class); given(omContainerInfo2.containerID()).willReturn(new ContainerID(2)); given(omContainerInfo2.getUsedBytes()).willReturn(2500000000L); // 2.5GB + given(omContainerInfo2.getState()).willReturn(CLOSING); // mock getContainers method to return a list of containers List containers = new ArrayList<>(); @@ -101,8 +109,8 @@ public void testProcess() { task.process(containers); - // Verify 2 containers are in correct bins. - assertEquals(2, containerCountBySizeDao.count()); + // Verify 3 containers are in correct bins. + assertEquals(3, containerCountBySizeDao.count()); // container size upper bound for // 1500000000L (1.5GB) is 2147483648L = 2^31 = 2GB (next highest power of 2) @@ -120,10 +128,11 @@ public void testProcess() { containerCountBySizeDao.findById(recordToFind.value1()).getCount() .longValue()); - // Add a new key + // Add a new container ContainerInfo omContainerInfo3 = mock(ContainerInfo.class); given(omContainerInfo3.containerID()).willReturn(new ContainerID(3)); given(omContainerInfo3.getUsedBytes()).willReturn(1000000000L); // 1GB + given(omContainerInfo3.getState()).willReturn(QUASI_CLOSED); containers.add(omContainerInfo3); // Update existing key. @@ -133,7 +142,7 @@ public void testProcess() { task.process(containers); // Total size groups added to the database - assertEquals(4, containerCountBySizeDao.count()); + assertEquals(5, containerCountBySizeDao.count()); // Check whether container size upper bound for // 50000L is 536870912L = 2^29 = 512MB (next highest power of 2) @@ -160,4 +169,59 @@ public void testProcess() { .getCount() .longValue()); } + + @Test + public void testProcessDeletedAndNegativeSizedContainers() { + // Create a list of containers, including one that is deleted + ContainerInfo omContainerInfo1 = mock(ContainerInfo.class); + given(omContainerInfo1.containerID()).willReturn(new ContainerID(1)); + given(omContainerInfo1.getUsedBytes()).willReturn(1500000000L); // 1.5GB + given(omContainerInfo1.getState()).willReturn(OPEN); + + ContainerInfo omContainerInfo2 = mock(ContainerInfo.class); + given(omContainerInfo2.containerID()).willReturn(new ContainerID(2)); + given(omContainerInfo2.getUsedBytes()).willReturn(2500000000L); // 2.5GB + given(omContainerInfo2.getState()).willReturn(CLOSED); + + ContainerInfo omContainerInfoDeleted = mock(ContainerInfo.class); + given(omContainerInfoDeleted.containerID()).willReturn(new ContainerID(3)); + given(omContainerInfoDeleted.getUsedBytes()).willReturn(1000000000L); + given(omContainerInfoDeleted.getState()).willReturn(DELETED); // 1GB + + // Create a mock container with negative size + final ContainerInfo negativeSizeContainer = mock(ContainerInfo.class); + given(negativeSizeContainer.containerID()).willReturn(new ContainerID(0)); + given(negativeSizeContainer.getUsedBytes()).willReturn(-1L); + given(negativeSizeContainer.getState()).willReturn(OPEN); + + // Create a mock container with negative size and DELETE state + final ContainerInfo negativeSizeDeletedContainer = + mock(ContainerInfo.class); + given(negativeSizeDeletedContainer.containerID()).willReturn( + new ContainerID(0)); + given(negativeSizeDeletedContainer.getUsedBytes()).willReturn(-1L); + given(negativeSizeDeletedContainer.getState()).willReturn(DELETED); + + // Create a mock container with id 1 and updated size of 1GB from 1.5GB + final ContainerInfo validSizeContainer = mock(ContainerInfo.class); + given(validSizeContainer.containerID()).willReturn(new ContainerID(1)); + given(validSizeContainer.getUsedBytes()).willReturn(1000000000L); // 1GB + given(validSizeContainer.getState()).willReturn(CLOSED); + + // Mock getContainers method to return a list of containers including + // both valid and invalid ones + List containers = new ArrayList<>(); + containers.add(omContainerInfo1); + containers.add(omContainerInfo2); + containers.add(omContainerInfoDeleted); + containers.add(negativeSizeContainer); + containers.add(negativeSizeDeletedContainer); + containers.add(validSizeContainer); + + task.process(containers); + + // Verify that only the valid containers are counted + assertEquals(3, containerCountBySizeDao.count()); + } + } diff --git a/pom.xml b/pom.xml index 70eba595f42..3622f13be78 100644 --- a/pom.xml +++ b/pom.xml @@ -24,6 +24,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs pom + dev-support hadoop-hdds hadoop-ozone @@ -75,7 +76,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs ${hdds.version} - 3.1.0 + 3.1.1 1.0.6 @@ -207,7 +208,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 4.12.0 4.2.2 2.6.1 - 1.3.5 + 2.1.1 2.12.5 0.19 @@ -1944,25 +1945,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs org.apache.maven.plugins maven-remote-resources-plugin ${maven-remote-resources-plugin.version} - - - org.apache.hadoop:hadoop-build-tools:${hadoop.version} - - - - - org.apache.hadoop - hadoop-build-tools - ${hadoop.version} - - - - - - process - - - org.apache.maven.plugins