diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java index e324a63d3ba..b8742c6ba92 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Enumeration; import java.util.HashMap; import java.util.List; @@ -163,15 +164,11 @@ public XMLConfiguration() { } public XMLConfiguration(List properties) { - this.properties = properties; + this.properties = new ArrayList<>(properties); } public List getProperties() { - return properties; - } - - public void setProperties(List properties) { - this.properties = properties; + return Collections.unmodifiableList(properties); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java index 2d29dc8565c..ba203f9c8e2 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/freon/FakeClusterTopology.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.freon; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Random; import java.util.UUID; @@ -36,40 +37,49 @@ * Class to store pre-generated topology information for load-tests. */ @SuppressWarnings("java:S2245") // no need for secure random -public class FakeClusterTopology { +public final class FakeClusterTopology { private static final Logger LOGGER = LoggerFactory.getLogger(FakeClusterTopology.class); - public static final FakeClusterTopology INSTANCE = new FakeClusterTopology(); + public static final FakeClusterTopology INSTANCE = newFakeClusterTopology(); - private List datanodes = new ArrayList<>(); + private final List datanodes; - private List pipelines = new ArrayList<>(); + private final List pipelines; - private Random random = new Random(); + private final Random random = new Random(); - public FakeClusterTopology() { + private static FakeClusterTopology newFakeClusterTopology() { + final int nodeCount = 9; + final List datanodes = new ArrayList<>(nodeCount); + final List pipelines = new ArrayList<>(nodeCount / 3); try { - for (int i = 0; i < 9; i++) { + for (int i = 0; i < nodeCount; i++) { datanodes.add(createDatanode()); if ((i + 1) % 3 == 0) { pipelines.add(Pipeline.newBuilder() .setId(PipelineID.randomId().getProtobuf()) .setFactor(ReplicationFactor.THREE) .setType(ReplicationType.RATIS) - .addMembers(getDatanode(i - 2)) - .addMembers(getDatanode(i - 1)) - .addMembers(getDatanode(i)) + .addMembers(datanodes.get(i - 2)) + .addMembers(datanodes.get(i - 1)) + .addMembers(datanodes.get(i)) .build()); } } } catch (Exception ex) { LOGGER.error("Can't initialize FakeClusterTopology", ex); } + return new FakeClusterTopology(datanodes, pipelines); } - private DatanodeDetailsProto createDatanode() { + private FakeClusterTopology(List datanodes, List pipelines) { + this.datanodes = Collections.unmodifiableList(datanodes); + this.pipelines = Collections.unmodifiableList(pipelines); + } + + private static DatanodeDetailsProto createDatanode() { return DatanodeDetailsProto.newBuilder() .setUuid(UUID.randomUUID().toString()) .setHostName("localhost") @@ -79,15 +89,11 @@ private DatanodeDetailsProto createDatanode() { .build(); } - public DatanodeDetailsProto getDatanode(int i) { - return datanodes.get(i); - } - public Pipeline getRandomPipeline() { return pipelines.get(random.nextInt(pipelines.size())); } - public List getAllDatanodes() { + public Iterable getAllDatanodes() { return datanodes; } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java index 224f961158f..f54b96eb049 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java @@ -242,7 +242,16 @@ public synchronized void setPort(Name name, int port) { * @return DataNode Ports */ public synchronized List getPorts() { - return ports; + return new ArrayList<>(ports); + } + + public synchronized boolean hasPort(int port) { + for (Port p : ports) { + if (p.getValue() == port) { + return true; + } + } + return false; } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java index b9d823e8d81..19c39698dec 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmInfo.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm; import java.util.ArrayList; +import java.util.Collections; import java.util.List; /** @@ -26,9 +27,9 @@ * contains clusterId and the SCM Id. */ public final class ScmInfo { - private String clusterId; - private String scmId; - private List peerRoles; + private final String clusterId; + private final String scmId; + private final List peerRoles; /** * Builder for ScmInfo. @@ -36,7 +37,7 @@ public final class ScmInfo { public static class Builder { private String clusterId; private String scmId; - private List peerRoles; + private final List peerRoles; public Builder() { peerRoles = new ArrayList<>(); @@ -80,7 +81,7 @@ public ScmInfo build() { private ScmInfo(String clusterId, String scmId, List peerRoles) { this.clusterId = clusterId; this.scmId = scmId; - this.peerRoles = peerRoles; + this.peerRoles = Collections.unmodifiableList(peerRoles); } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java index b6834aba1eb..61fd0d8f033 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientReply.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerCommandResponseProto; +import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -60,7 +61,7 @@ public void setLogIndex(long logIndex) { } public List getDatanodes() { - return datanodes; + return Collections.unmodifiableList(datanodes); } public void addDatanode(DatanodeDetails dn) { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java index 2577a1e5ea2..258c0be8961 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java @@ -25,6 +25,7 @@ import java.time.Clock; import java.time.ZoneOffset; import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.Map; @@ -38,28 +39,24 @@ */ public class ExcludeList { - private final Map datanodes; - private final Set containerIds; - private final Set pipelineIds; + private final Map datanodes = new ConcurrentHashMap<>(); + private final Set containerIds = new HashSet<>(); + private final Set pipelineIds = new HashSet<>(); private long expiryTime = 0; - private java.time.Clock clock; + private final Clock clock; public ExcludeList() { - datanodes = new ConcurrentHashMap<>(); - containerIds = new HashSet<>(); - pipelineIds = new HashSet<>(); clock = Clock.system(ZoneOffset.UTC); } - public ExcludeList(long autoExpiryTime, java.time.Clock clock) { - this(); + public ExcludeList(long autoExpiryTime, Clock clock) { this.expiryTime = autoExpiryTime; this.clock = clock; } public Set getContainerIds() { - return containerIds; + return Collections.unmodifiableSet(containerIds); } public Set getDatanodes() { @@ -99,7 +96,7 @@ public void addPipeline(PipelineID pipelineId) { } public Set getPipelineIds() { - return pipelineIds; + return Collections.unmodifiableSet(pipelineIds); } public HddsProtos.ExcludeListProto getProtoBuf() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java index 80e09af172b..af4e7299383 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAUtils.java @@ -65,7 +65,7 @@ public final class SCMHAUtils { public static final Logger LOG = LoggerFactory.getLogger(SCMHAUtils.class); - private static final List> + private static final ImmutableList> RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST = ImmutableList.>builder() .add(LeaderNotReadyException.class) @@ -74,7 +74,7 @@ public final class SCMHAUtils { .add(ResourceUnavailableException.class) .build(); - private static final List> + private static final ImmutableList> NON_RETRIABLE_EXCEPTION_LIST = ImmutableList.>builder() .add(SCMException.class) @@ -316,7 +316,7 @@ public static Throwable getExceptionForClass(Exception e, return null; } - public static List> getRetriableWithNoFailoverExceptionList() { return RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java index 05d83a8b8b5..6ea92f74c19 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java @@ -23,7 +23,6 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedList; @@ -113,7 +112,7 @@ private Pipeline(Builder b) { suggestedLeaderId = b.suggestedLeaderId; nodeStatus = b.nodeStatus; nodesInOrder = b.nodesInOrder != null ? ImmutableList.copyOf(b.nodesInOrder) : ImmutableList.of(); - replicaIndexes = b.replicaIndexes != null ? ImmutableMap.copyOf(b.replicaIndexes) : ImmutableMap.of(); + replicaIndexes = b.replicaIndexes; creationTimestamp = b.creationTimestamp != null ? b.creationTimestamp : Instant.now(); stateEnterTime = Instant.now(); } @@ -541,7 +540,7 @@ public static class Builder { private UUID leaderId = null; private Instant creationTimestamp = null; private UUID suggestedLeaderId = null; - private Map replicaIndexes; + private Map replicaIndexes = ImmutableMap.of(); public Builder() { } @@ -555,13 +554,14 @@ public Builder(Pipeline pipeline) { this.creationTimestamp = pipeline.getCreationTimestamp(); this.suggestedLeaderId = pipeline.getSuggestedLeaderId(); if (nodeStatus != null) { - replicaIndexes = new HashMap<>(); + final ImmutableMap.Builder b = ImmutableMap.builder(); for (DatanodeDetails dn : nodeStatus.keySet()) { int index = pipeline.getReplicaIndex(dn); if (index > 0) { - replicaIndexes.put(dn, index); + b.put(dn, index); } } + replicaIndexes = b.build(); } } @@ -598,7 +598,7 @@ public Builder setNodes(List nodes) { public Builder setNodeOrder(List orders) { // for build from ProtoBuf - this.nodeOrder = orders; + this.nodeOrder = Collections.unmodifiableList(orders); return this; } @@ -624,7 +624,7 @@ public Builder setSuggestedLeaderId(UUID uuid) { public Builder setReplicaIndexes(Map indexes) { - this.replicaIndexes = indexes; + this.replicaIndexes = indexes == null ? ImmutableMap.of() : ImmutableMap.copyOf(indexes); return this; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 6bb87654c0e..66c8459a01a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -746,7 +746,7 @@ public static void validateContainerResponse( response.getMessage(), response.getResult()); } - public static List getValidatorList() { + private static List getValidatorList() { return VALIDATORS; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BatchOperation.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BatchOperation.java deleted file mode 100644 index c5640cb1542..00000000000 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BatchOperation.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdds.utils; - -import com.google.common.collect.Lists; - -import java.util.List; -import java.util.Objects; - -/** - * An utility class to store a batch of DB write operations. - */ -public class BatchOperation { - - /** - * Enum for write operations. - */ - public enum Operation { - DELETE, PUT - } - - private List operations = - Lists.newArrayList(); - - /** - * Add a PUT operation into the batch. - */ - public void put(byte[] key, byte[] value) { - operations.add(new SingleOperation(Operation.PUT, key, value)); - } - - /** - * Add a DELETE operation into the batch. - */ - public void delete(byte[] key) { - operations.add(new SingleOperation(Operation.DELETE, key, null)); - - } - - public List getOperations() { - return operations; - } - - /** - * A SingleOperation represents a PUT or DELETE operation - * and the data the operation needs to manipulates. - */ - static class SingleOperation { - - private final Operation opt; - private final byte[] key; - private final byte[] value; - - SingleOperation(Operation opt, byte[] key, byte[] value) { - this.opt = opt; - this.key = Objects.requireNonNull(key, "key cannot be null"); - this.value = value; - } - - public Operation getOpt() { - return opt; - } - - public byte[] getKey() { - return key; - } - - public byte[] getValue() { - return value; - } - } -} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumData.java index aea5c510661..4f6bfa450b0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumData.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChecksumData.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.List; + import org.apache.commons.lang3.builder.HashCodeBuilder; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos @@ -33,7 +34,7 @@ */ public class ChecksumData { - private ChecksumType type; + private final ChecksumType type; // Checksum will be computed for every bytesPerChecksum number of bytes and // stored sequentially in checksumList private final int bytesPerChecksum; @@ -47,7 +48,7 @@ public ChecksumData(ChecksumType checksumType, int bytesPerChecksum, List checksums) { this.type = checksumType; this.bytesPerChecksum = bytesPerChecksum; - this.checksums = checksums; + this.checksums = Collections.unmodifiableList(checksums); } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java index 97122e68987..4c78ca777f2 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java @@ -18,10 +18,10 @@ package org.apache.hadoop.ozone.common.statemachine; -import com.google.common.base.Supplier; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableSet; import java.util.HashMap; import java.util.Map; @@ -33,16 +33,16 @@ * @param events allowed */ public class StateMachine, EVENT extends Enum> { - private STATE initialState; - private Set finalStates; + private final STATE initialState; + private final ImmutableSet finalStates; private final LoadingCache> transitions = CacheBuilder.newBuilder().build( - CacheLoader.from((Supplier>) () -> new HashMap())); + CacheLoader.from(() -> new HashMap<>())); public StateMachine(STATE initState, Set finalStates) { this.initialState = initState; - this.finalStates = finalStates; + this.finalStates = finalStates == null ? ImmutableSet.of() : ImmutableSet.copyOf(finalStates); } public STATE getInitialState() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java index cbbbb70278a..75408d65a66 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java @@ -64,7 +64,6 @@ public ChunkInfo(String chunkName, long offset, long len) { * * @param key - Key Name. * @param value - Value. - * @throws IOException */ public void addMetadata(String key, String value) throws IOException { synchronized (this.metadata) { @@ -80,7 +79,6 @@ public void addMetadata(String key, String value) throws IOException { * * @param info - Protobuf class * @return ChunkInfo - * @throws IOException */ public static ChunkInfo getFromProtoBuf(ContainerProtos.ChunkInfo info) throws IOException { @@ -182,14 +180,9 @@ public ByteString getStripeChecksum() { public void setStripeChecksum(ByteString stripeChecksum) { this.stripeChecksum = stripeChecksum; } - - /** - * Returns Metadata associated with this Chunk. - * - * @return - Map of Key,values. - */ - public Map getMetadata() { - return metadata; + + public String getMetadata(String key) { + return metadata.get(key); } @Override diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java index b3ffe59f1d7..b3142c579f6 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.util; -import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; @@ -122,9 +121,7 @@ public void run() { * This is exposed purely for testing: do not invoke it. * @return the number of shutdown hooks which timed out. */ - @InterfaceAudience.Private - @VisibleForTesting - int executeShutdown() { + private int executeShutdown() { int timeouts = 0; for (HookEntry entry: getShutdownHooksInOrder()) { Future future = EXECUTOR.submit(entry.getHook()); @@ -190,9 +187,7 @@ public static ShutdownHookManager get() { * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig * #OZONE_SHUTDOWN_TIMEOUT_MINIMUM} */ - @InterfaceAudience.Private - @VisibleForTesting - static long getShutdownTimeout(ConfigurationSource conf) { + private static long getShutdownTimeout(ConfigurationSource conf) { long duration = HddsUtils.getShutDownTimeOut(conf); if (duration < OZONE_SHUTDOWN_TIMEOUT_MINIMUM) { duration = OZONE_SHUTDOWN_TIMEOUT_MINIMUM; @@ -204,9 +199,7 @@ static long getShutdownTimeout(ConfigurationSource conf) { * Private structure to store ShutdownHook, its priority and timeout * settings. */ - @InterfaceAudience.Private - @VisibleForTesting - static class HookEntry { + private static class HookEntry { private final Runnable hook; private final int priority; private final long timeout; @@ -260,12 +253,9 @@ TimeUnit getTimeUnit() { private final Set hooks = Collections.synchronizedSet(new HashSet<>()); - private AtomicBoolean shutdownInProgress = new AtomicBoolean(false); + private final AtomicBoolean shutdownInProgress = new AtomicBoolean(false); - //private to constructor to ensure singularity - @VisibleForTesting - @InterfaceAudience.Private - ShutdownHookManager() { + private ShutdownHookManager() { } /** @@ -274,21 +264,13 @@ TimeUnit getTimeUnit() { * * @return the list of shutdownHooks in order of execution. */ - @InterfaceAudience.Private - @VisibleForTesting - List getShutdownHooksInOrder() { - List list; + private List getShutdownHooksInOrder() { + List list; synchronized (hooks) { - list = new ArrayList(hooks); + list = new ArrayList<>(hooks); } - Collections.sort(list, new Comparator< HookEntry >() { - - //reversing comparison so highest priority hooks are first - @Override - public int compare(HookEntry o1, HookEntry o2) { - return o2.priority - o1.priority; - } - }); + //reversing comparison so highest priority hooks are first + list.sort(Comparator.comparing(HookEntry::getPriority).reversed()); return list; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java index f8acbc7e2d6..85de8c08cc1 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java @@ -26,6 +26,7 @@ import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.UUID; @@ -238,8 +239,9 @@ private static DatanodeDetailsYaml getDatanodeDetailsYaml( = new DatanodeLayoutStorage(conf, datanodeDetails.getUuidString()); Map portDetails = new LinkedHashMap<>(); - if (!CollectionUtils.isEmpty(datanodeDetails.getPorts())) { - for (DatanodeDetails.Port port : datanodeDetails.getPorts()) { + final List ports = datanodeDetails.getPorts(); + if (!CollectionUtils.isEmpty(ports)) { + for (DatanodeDetails.Port port : ports) { Field f = null; try { f = DatanodeDetails.Port.Name.class diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java index 7266904139d..c3f6ac92120 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java @@ -377,8 +377,8 @@ public static boolean isOverWriteRequested(FileChannel channel, ChunkInfo * @param chunkInfo - Chunk info * @return true if the user asks for it. */ - public static boolean isOverWritePermitted(ChunkInfo chunkInfo) { - String overWrite = chunkInfo.getMetadata().get(OzoneConsts.CHUNK_OVERWRITE); + private static boolean isOverWritePermitted(ChunkInfo chunkInfo) { + String overWrite = chunkInfo.getMetadata(OzoneConsts.CHUNK_OVERWRITE); return Boolean.parseBoolean(overWrite); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java index df224d1d449..a593062bcdb 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java @@ -254,12 +254,7 @@ private DatanodeDetails findDnWithMostRecentHeartbeat( * @return True if port is used by the datanode. False otherwise. */ private boolean validateDNPortMatch(int port, DatanodeDetails dn) { - for (DatanodeDetails.Port p : dn.getPorts()) { - if (p.getValue() == port) { - return true; - } - } - return false; + return dn.hasPort(port); } public NodeDecommissionManager(OzoneConfiguration config, NodeManager nm, ContainerManager cm, diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java index cfdc924486a..927e9186ff5 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java @@ -159,8 +159,7 @@ public static void generateConfigurations(String path, } OzoneConfiguration.XMLConfiguration generatedConfig = - new OzoneConfiguration.XMLConfiguration(); - generatedConfig.setProperties(requiredProperties); + new OzoneConfiguration.XMLConfiguration(requiredProperties); File output = new File(path, "ozone-site.xml"); if (output.createNewFile()) {