|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.hdds.scm.pipeline;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+import org.apache.commons.lang3.builder.HashCodeBuilder;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
@@ -27,6 +28,7 @@ import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.*;
|
|
|
+import java.util.concurrent.CopyOnWriteArrayList;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
/**
|
|
@@ -42,15 +44,27 @@ class PipelineStateMap {
|
|
|
|
|
|
private final Map<PipelineID, Pipeline> pipelineMap;
|
|
|
private final Map<PipelineID, NavigableSet<ContainerID>> pipeline2container;
|
|
|
+ private final Map<PipelineQuery, List<Pipeline>> query2OpenPipelines;
|
|
|
|
|
|
PipelineStateMap() {
|
|
|
|
|
|
// TODO: Use TreeMap for range operations?
|
|
|
- this.pipelineMap = new HashMap<>();
|
|
|
- this.pipeline2container = new HashMap<>();
|
|
|
+ pipelineMap = new HashMap<>();
|
|
|
+ pipeline2container = new HashMap<>();
|
|
|
+ query2OpenPipelines = new HashMap<>();
|
|
|
+ initializeQueryMap();
|
|
|
|
|
|
}
|
|
|
|
|
|
+ private void initializeQueryMap() {
|
|
|
+ for (ReplicationType type : ReplicationType.values()) {
|
|
|
+ for (ReplicationFactor factor : ReplicationFactor.values()) {
|
|
|
+ query2OpenPipelines
|
|
|
+ .put(new PipelineQuery(type, factor), new CopyOnWriteArrayList<>());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Adds provided pipeline in the data structures.
|
|
|
*
|
|
@@ -70,6 +84,9 @@ class PipelineStateMap {
|
|
|
.format("Duplicate pipeline ID %s detected.", pipeline.getId()));
|
|
|
}
|
|
|
pipeline2container.put(pipeline.getId(), new TreeSet<>());
|
|
|
+ if (pipeline.getPipelineState() == PipelineState.OPEN) {
|
|
|
+ query2OpenPipelines.get(new PipelineQuery(pipeline)).add(pipeline);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -188,6 +205,10 @@ class PipelineStateMap {
|
|
|
Preconditions.checkNotNull(factor, "Replication factor cannot be null");
|
|
|
Preconditions.checkNotNull(state, "Pipeline state cannot be null");
|
|
|
|
|
|
+ if (state == PipelineState.OPEN) {
|
|
|
+ return Collections.unmodifiableList(
|
|
|
+ query2OpenPipelines.get(new PipelineQuery(type, factor)));
|
|
|
+ }
|
|
|
return pipelineMap.values().stream().filter(
|
|
|
pipeline -> pipeline.getType() == type
|
|
|
&& pipeline.getPipelineState() == state
|
|
@@ -293,7 +314,52 @@ class PipelineStateMap {
|
|
|
Preconditions.checkNotNull(state, "Pipeline LifeCycleState cannot be null");
|
|
|
|
|
|
final Pipeline pipeline = getPipeline(pipelineID);
|
|
|
- return pipelineMap.compute(pipelineID,
|
|
|
+ Pipeline updatedPipeline = pipelineMap.compute(pipelineID,
|
|
|
(id, p) -> Pipeline.newBuilder(pipeline).setState(state).build());
|
|
|
+ PipelineQuery query = new PipelineQuery(pipeline);
|
|
|
+ if (updatedPipeline.getPipelineState() == PipelineState.OPEN) {
|
|
|
+ // for transition to OPEN state add pipeline to query2OpenPipelines
|
|
|
+ query2OpenPipelines.get(query).add(updatedPipeline);
|
|
|
+ } else if (updatedPipeline.getPipelineState() == PipelineState.CLOSED) {
|
|
|
+ // for transition from OPEN to CLOSED state remove pipeline from
|
|
|
+ // query2OpenPipelines
|
|
|
+ query2OpenPipelines.get(query).remove(pipeline);
|
|
|
+ }
|
|
|
+ return updatedPipeline;
|
|
|
+ }
|
|
|
+
|
|
|
+ private class PipelineQuery {
|
|
|
+ private ReplicationType type;
|
|
|
+ private ReplicationFactor factor;
|
|
|
+
|
|
|
+ PipelineQuery(ReplicationType type, ReplicationFactor factor) {
|
|
|
+ this.type = type;
|
|
|
+ this.factor = factor;
|
|
|
+ }
|
|
|
+
|
|
|
+ PipelineQuery(Pipeline pipeline) {
|
|
|
+ type = pipeline.getType();
|
|
|
+ factor = pipeline.getFactor();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean equals(Object other) {
|
|
|
+ if (this == other) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ if (!this.getClass().equals(other.getClass())) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ PipelineQuery otherQuery = (PipelineQuery) other;
|
|
|
+ return type == otherQuery.type && factor == otherQuery.factor;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int hashCode() {
|
|
|
+ return new HashCodeBuilder()
|
|
|
+ .append(type)
|
|
|
+ .append(factor)
|
|
|
+ .toHashCode();
|
|
|
+ }
|
|
|
}
|
|
|
}
|