Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -163,15 +164,11 @@ public XMLConfiguration() {
}

public XMLConfiguration(List<Property> properties) {
this.properties = properties;
this.properties = new ArrayList<>(properties);
}

public List<Property> getProperties() {
return properties;
}

public void setProperties(List<Property> properties) {
this.properties = properties;
return Collections.unmodifiableList(properties);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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<DatanodeDetailsProto> datanodes = new ArrayList<>();
private final List<DatanodeDetailsProto> datanodes;

private List<Pipeline> pipelines = new ArrayList<>();
private final List<Pipeline> pipelines;

private Random random = new Random();
private final Random random = new Random();

public FakeClusterTopology() {
private static FakeClusterTopology newFakeClusterTopology() {
final int nodeCount = 9;
final List<DatanodeDetailsProto> datanodes = new ArrayList<>(nodeCount);
final List<Pipeline> 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<DatanodeDetailsProto> datanodes, List<Pipeline> pipelines) {
this.datanodes = Collections.unmodifiableList(datanodes);
this.pipelines = Collections.unmodifiableList(pipelines);
}

private static DatanodeDetailsProto createDatanode() {
return DatanodeDetailsProto.newBuilder()
.setUuid(UUID.randomUUID().toString())
.setHostName("localhost")
Expand All @@ -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<DatanodeDetailsProto> getAllDatanodes() {
public Iterable<DatanodeDetailsProto> getAllDatanodes() {
return datanodes;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -242,7 +242,16 @@ public synchronized void setPort(Name name, int port) {
* @return DataNode Ports
*/
public synchronized List<Port> getPorts() {
return ports;
return new ArrayList<>(ports);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should also change the callers in order to reduce copying:

  • NodeDecommissionManager
//NodeDecommissionManager
  private boolean validateDNPortMatch(int port, DatanodeDetails dn) {
    return dn.containPort(port);
  }
//DatanodeDetails
  public synchronized boolean containPort(int port) {
    for (Port p : ports) {
      if (p.getValue() == port) {
        return true;
      }
    }
    return false;
  }
  • DatanodeIdYaml
@@ -238,8 +239,9 @@ private static DatanodeDetailsYaml getDatanodeDetailsYaml(
         = new DatanodeLayoutStorage(conf, datanodeDetails.getUuidString());
 
     Map<String, Integer> portDetails = new LinkedHashMap<>();
-    if (!CollectionUtils.isEmpty(datanodeDetails.getPorts())) {
-      for (DatanodeDetails.Port port : datanodeDetails.getPorts()) {
+    final List<DatanodeDetails.Port> datanodePorts = datanodeDetails.getPorts();
+    if (!CollectionUtils.isEmpty(datanodePorts)) {
+      for (DatanodeDetails.Port port : datanodePorts) {
         Field f = null;
         try {
           f = DatanodeDetails.Port.Name.class

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the suggestion, updated.

}

public synchronized boolean hasPort(int port) {
for (Port p : ports) {
if (p.getValue() == port) {
return true;
}
}
return false;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,24 +19,25 @@
package org.apache.hadoop.hdds.scm;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;

/**
* ScmInfo wraps the result returned from SCM#getScmInfo which
* contains clusterId and the SCM Id.
*/
public final class ScmInfo {
private String clusterId;
private String scmId;
private List<String> peerRoles;
private final String clusterId;
private final String scmId;
private final List<String> peerRoles;

/**
* Builder for ScmInfo.
*/
public static class Builder {
private String clusterId;
private String scmId;
private List<String> peerRoles;
private final List<String> peerRoles;

public Builder() {
peerRoles = new ArrayList<>();
Expand Down Expand Up @@ -80,7 +81,7 @@ public ScmInfo build() {
private ScmInfo(String clusterId, String scmId, List<String> peerRoles) {
this.clusterId = clusterId;
this.scmId = scmId;
this.peerRoles = peerRoles;
this.peerRoles = Collections.unmodifiableList(peerRoles);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -60,7 +61,7 @@ public void setLogIndex(long logIndex) {
}

public List<DatanodeDetails> getDatanodes() {
return datanodes;
return Collections.unmodifiableList(datanodes);
}

public void addDatanode(DatanodeDetails dn) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -38,28 +39,24 @@
*/
public class ExcludeList {

private final Map<DatanodeDetails, Long> datanodes;
private final Set<ContainerID> containerIds;
private final Set<PipelineID> pipelineIds;
private final Map<DatanodeDetails, Long> datanodes = new ConcurrentHashMap<>();
private final Set<ContainerID> containerIds = new HashSet<>();
private final Set<PipelineID> 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<ContainerID> getContainerIds() {
return containerIds;
return Collections.unmodifiableSet(containerIds);
}

public Set<DatanodeDetails> getDatanodes() {
Expand Down Expand Up @@ -99,7 +96,7 @@ public void addPipeline(PipelineID pipelineId) {
}

public Set<PipelineID> getPipelineIds() {
return pipelineIds;
return Collections.unmodifiableSet(pipelineIds);
}

public HddsProtos.ExcludeListProto getProtoBuf() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ public final class SCMHAUtils {
public static final Logger LOG =
LoggerFactory.getLogger(SCMHAUtils.class);

private static final List<Class<? extends Exception>>
private static final ImmutableList<Class<? extends Exception>>
RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST =
ImmutableList.<Class<? extends Exception>>builder()
.add(LeaderNotReadyException.class)
Expand All @@ -74,7 +74,7 @@ public final class SCMHAUtils {
.add(ResourceUnavailableException.class)
.build();

private static final List<Class<? extends Exception>>
private static final ImmutableList<Class<? extends Exception>>
NON_RETRIABLE_EXCEPTION_LIST =
ImmutableList.<Class<? extends Exception>>builder()
.add(SCMException.class)
Expand Down Expand Up @@ -316,7 +316,7 @@ public static Throwable getExceptionForClass(Exception e,
return null;
}

public static List<Class<? extends
private static List<Class<? extends
Exception>> getRetriableWithNoFailoverExceptionList() {
return RETRIABLE_WITH_NO_FAILOVER_EXCEPTION_LIST;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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();
}
Expand Down Expand Up @@ -541,7 +540,7 @@ public static class Builder {
private UUID leaderId = null;
private Instant creationTimestamp = null;
private UUID suggestedLeaderId = null;
private Map<DatanodeDetails, Integer> replicaIndexes;
private Map<DatanodeDetails, Integer> replicaIndexes = ImmutableMap.of();

public Builder() { }

Expand All @@ -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<DatanodeDetails, Integer> 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();
}
}

Expand Down Expand Up @@ -598,7 +598,7 @@ public Builder setNodes(List<DatanodeDetails> nodes) {

public Builder setNodeOrder(List<Integer> orders) {
// for build from ProtoBuf
this.nodeOrder = orders;
this.nodeOrder = Collections.unmodifiableList(orders);
return this;
}

Expand All @@ -624,7 +624,7 @@ public Builder setSuggestedLeaderId(UUID uuid) {


public Builder setReplicaIndexes(Map<DatanodeDetails, Integer> indexes) {
this.replicaIndexes = indexes;
this.replicaIndexes = indexes == null ? ImmutableMap.of() : ImmutableMap.copyOf(indexes);
return this;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -746,7 +746,7 @@ public static void validateContainerResponse(
response.getMessage(), response.getResult());
}

public static List<Validator> getValidatorList() {
private static List<Validator> getValidatorList() {
return VALIDATORS;
}

Expand Down
Loading