|
@@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
|
|
-import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -40,6 +39,8 @@ import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
import java.util.NavigableSet;
|
|
|
import java.util.TreeSet;
|
|
|
+import java.util.concurrent.locks.ReadWriteLock;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
|
.CONTAINER_EXISTS;
|
|
@@ -108,7 +109,7 @@ public class ContainerStateMap {
|
|
|
// Container State Map lock should be held before calling into
|
|
|
// Update ContainerAttributes. The consistency of ContainerAttributes is
|
|
|
// protected by this lock.
|
|
|
- private final AutoCloseableLock autoLock;
|
|
|
+ private final ReadWriteLock lock;
|
|
|
|
|
|
/**
|
|
|
* Create a ContainerStateMap.
|
|
@@ -120,7 +121,7 @@ public class ContainerStateMap {
|
|
|
typeMap = new ContainerAttribute<>();
|
|
|
openPipelineMap = new ContainerAttribute<>();
|
|
|
containerMap = new HashMap<>();
|
|
|
- autoLock = new AutoCloseableLock();
|
|
|
+ lock = new ReentrantReadWriteLock();
|
|
|
contReplicaMap = new HashMap<>();
|
|
|
// new InstrumentedLock(getClass().getName(), LOG,
|
|
|
// new ReentrantLock(),
|
|
@@ -140,7 +141,8 @@ public class ContainerStateMap {
|
|
|
Preconditions.checkArgument(info.getReplicationFactor().getNumber() > 0,
|
|
|
"ExpectedReplicaCount should be greater than 0");
|
|
|
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.writeLock().lock();
|
|
|
+ try {
|
|
|
ContainerID id = ContainerID.valueof(info.getContainerID());
|
|
|
if (containerMap.putIfAbsent(id, info) != null) {
|
|
|
LOG.debug("Duplicate container ID detected. {}", id);
|
|
@@ -157,6 +159,8 @@ public class ContainerStateMap {
|
|
|
openPipelineMap.insert(info.getPipelineID(), id);
|
|
|
}
|
|
|
LOG.trace("Created container with {} successfully.", id);
|
|
|
+ } finally {
|
|
|
+ lock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -177,8 +181,13 @@ public class ContainerStateMap {
|
|
|
* @return container info, if found.
|
|
|
*/
|
|
|
public ContainerInfo getContainerInfo(long containerID) {
|
|
|
- ContainerID id = new ContainerID(containerID);
|
|
|
- return containerMap.get(id);
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
+ ContainerID id = new ContainerID(containerID);
|
|
|
+ return containerMap.get(id);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -191,11 +200,14 @@ public class ContainerStateMap {
|
|
|
public Set<DatanodeDetails> getContainerReplicas(ContainerID containerID)
|
|
|
throws SCMException {
|
|
|
Preconditions.checkNotNull(containerID);
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
if (contReplicaMap.containsKey(containerID)) {
|
|
|
return Collections
|
|
|
.unmodifiableSet(contReplicaMap.get(containerID));
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
throw new SCMException(
|
|
|
"No entry exist for containerId: " + containerID + " in replica map.",
|
|
@@ -213,8 +225,8 @@ public class ContainerStateMap {
|
|
|
public void addContainerReplica(ContainerID containerID,
|
|
|
DatanodeDetails... dnList) {
|
|
|
Preconditions.checkNotNull(containerID);
|
|
|
- // Take lock to avoid race condition around insertion.
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.writeLock().lock();
|
|
|
+ try {
|
|
|
for (DatanodeDetails dn : dnList) {
|
|
|
Preconditions.checkNotNull(dn);
|
|
|
if (contReplicaMap.containsKey(containerID)) {
|
|
@@ -228,6 +240,8 @@ public class ContainerStateMap {
|
|
|
contReplicaMap.put(containerID, dnSet);
|
|
|
}
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ lock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -243,11 +257,13 @@ public class ContainerStateMap {
|
|
|
Preconditions.checkNotNull(containerID);
|
|
|
Preconditions.checkNotNull(dn);
|
|
|
|
|
|
- // Take lock to avoid race condition.
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.writeLock().lock();
|
|
|
+ try {
|
|
|
if (contReplicaMap.containsKey(containerID)) {
|
|
|
return contReplicaMap.get(containerID).remove(dn);
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ lock.writeLock().unlock();
|
|
|
}
|
|
|
throw new SCMException(
|
|
|
"No entry exist for containerId: " + containerID + " in replica map.",
|
|
@@ -265,8 +281,11 @@ public class ContainerStateMap {
|
|
|
* @return - Map
|
|
|
*/
|
|
|
public Map<ContainerID, ContainerInfo> getContainerMap() {
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
return Collections.unmodifiableMap(containerMap);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -277,7 +296,8 @@ public class ContainerStateMap {
|
|
|
public void updateContainerInfo(ContainerInfo info) throws SCMException {
|
|
|
Preconditions.checkNotNull(info);
|
|
|
ContainerInfo currentInfo = null;
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.writeLock().lock();
|
|
|
+ try {
|
|
|
currentInfo = containerMap.get(
|
|
|
ContainerID.valueof(info.getContainerID()));
|
|
|
|
|
@@ -285,6 +305,8 @@ public class ContainerStateMap {
|
|
|
throw new SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
|
|
|
}
|
|
|
containerMap.put(info.containerID(), info);
|
|
|
+ } finally {
|
|
|
+ lock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -304,51 +326,56 @@ public class ContainerStateMap {
|
|
|
ContainerID id = new ContainerID(info.getContainerID());
|
|
|
ContainerInfo currentInfo = null;
|
|
|
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
- currentInfo = containerMap.get(id);
|
|
|
+ lock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ try {
|
|
|
+ currentInfo = containerMap.get(id);
|
|
|
|
|
|
- if (currentInfo == null) {
|
|
|
- throw new
|
|
|
- SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
|
|
|
+ if (currentInfo == null) {
|
|
|
+ throw new
|
|
|
+ SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
|
|
|
+ }
|
|
|
+ // We are updating two places before this update is done, these can
|
|
|
+ // fail independently, since the code needs to handle it.
|
|
|
+
|
|
|
+ // We update the attribute map, if that fails it will throw an
|
|
|
+ // exception, so no issues, if we are successful, we keep track of the
|
|
|
+ // fact that we have updated the lifecycle state in the map, and update
|
|
|
+ // the container state. If this second update fails, we will attempt to
|
|
|
+ // roll back the earlier change we did. If the rollback fails, we can
|
|
|
+ // be in an inconsistent state,
|
|
|
+
|
|
|
+ info.setState(newState);
|
|
|
+ containerMap.put(id, info);
|
|
|
+ lifeCycleStateMap.update(currentState, newState, id);
|
|
|
+ LOG.trace("Updated the container {} to new state. Old = {}, new = " +
|
|
|
+ "{}", id, currentState, newState);
|
|
|
+ } catch (SCMException ex) {
|
|
|
+ LOG.error("Unable to update the container state. {}", ex);
|
|
|
+ // we need to revert the change in this attribute since we are not
|
|
|
+ // able to update the hash table.
|
|
|
+ LOG.info("Reverting the update to lifecycle state. Moving back to " +
|
|
|
+ "old state. Old = {}, Attempted state = {}", currentState,
|
|
|
+ newState);
|
|
|
+
|
|
|
+ containerMap.put(id, currentInfo);
|
|
|
+
|
|
|
+ // if this line throws, the state map can be in an inconsistent
|
|
|
+ // state, since we will have modified the attribute by the
|
|
|
+ // container state will not in sync since we were not able to put
|
|
|
+ // that into the hash table.
|
|
|
+ lifeCycleStateMap.update(newState, currentState, id);
|
|
|
+
|
|
|
+ throw new SCMException("Updating the container map failed.", ex,
|
|
|
+ FAILED_TO_CHANGE_CONTAINER_STATE);
|
|
|
}
|
|
|
- // We are updating two places before this update is done, these can
|
|
|
- // fail independently, since the code needs to handle it.
|
|
|
-
|
|
|
- // We update the attribute map, if that fails it will throw an exception,
|
|
|
- // so no issues, if we are successful, we keep track of the fact that we
|
|
|
- // have updated the lifecycle state in the map, and update the container
|
|
|
- // state. If this second update fails, we will attempt to roll back the
|
|
|
- // earlier change we did. If the rollback fails, we can be in an
|
|
|
- // inconsistent state,
|
|
|
-
|
|
|
- info.setState(newState);
|
|
|
- containerMap.put(id, info);
|
|
|
- lifeCycleStateMap.update(currentState, newState, id);
|
|
|
- LOG.trace("Updated the container {} to new state. Old = {}, new = " +
|
|
|
- "{}", id, currentState, newState);
|
|
|
- } catch (SCMException ex) {
|
|
|
- LOG.error("Unable to update the container state. {}", ex);
|
|
|
- // we need to revert the change in this attribute since we are not
|
|
|
- // able to update the hash table.
|
|
|
- LOG.info("Reverting the update to lifecycle state. Moving back to " +
|
|
|
- "old state. Old = {}, Attempted state = {}", currentState,
|
|
|
- newState);
|
|
|
-
|
|
|
- containerMap.put(id, currentInfo);
|
|
|
-
|
|
|
- // if this line throws, the state map can be in an inconsistent
|
|
|
- // state, since we will have modified the attribute by the
|
|
|
- // container state will not in sync since we were not able to put
|
|
|
- // that into the hash table.
|
|
|
- lifeCycleStateMap.update(newState, currentState, id);
|
|
|
-
|
|
|
- throw new SCMException("Updating the container map failed.", ex,
|
|
|
- FAILED_TO_CHANGE_CONTAINER_STATE);
|
|
|
- }
|
|
|
- // In case the container is set to closed state, it needs to be removed from
|
|
|
- // the pipeline Map.
|
|
|
- if (!info.isContainerOpen()) {
|
|
|
- openPipelineMap.remove(info.getPipelineID(), id);
|
|
|
+ // In case the container is set to closed state, it needs to be removed
|
|
|
+ // from the pipeline Map.
|
|
|
+ if (!info.isContainerOpen()) {
|
|
|
+ openPipelineMap.remove(info.getPipelineID(), id);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ lock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -360,9 +387,11 @@ public class ContainerStateMap {
|
|
|
*/
|
|
|
NavigableSet<ContainerID> getContainerIDsByOwner(String ownerName) {
|
|
|
Preconditions.checkNotNull(ownerName);
|
|
|
-
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
return ownerMap.getCollection(ownerName);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -374,9 +403,11 @@ public class ContainerStateMap {
|
|
|
*/
|
|
|
NavigableSet<ContainerID> getContainerIDsByType(ReplicationType type) {
|
|
|
Preconditions.checkNotNull(type);
|
|
|
-
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
return typeMap.getCollection(type);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -389,9 +420,11 @@ public class ContainerStateMap {
|
|
|
public NavigableSet<ContainerID> getOpenContainerIDsByPipeline(
|
|
|
PipelineID pipelineID) {
|
|
|
Preconditions.checkNotNull(pipelineID);
|
|
|
-
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
return openPipelineMap.getCollection(pipelineID);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -403,9 +436,11 @@ public class ContainerStateMap {
|
|
|
*/
|
|
|
NavigableSet<ContainerID> getContainerIDsByFactor(ReplicationFactor factor) {
|
|
|
Preconditions.checkNotNull(factor);
|
|
|
-
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
return factorMap.getCollection(factor);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -415,11 +450,14 @@ public class ContainerStateMap {
|
|
|
* @param state - State - Open, Closed etc.
|
|
|
* @return List of containers by state.
|
|
|
*/
|
|
|
- NavigableSet<ContainerID> getContainerIDsByState(LifeCycleState state) {
|
|
|
+ public NavigableSet<ContainerID> getContainerIDsByState(
|
|
|
+ LifeCycleState state) {
|
|
|
Preconditions.checkNotNull(state);
|
|
|
-
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
return lifeCycleStateMap.getCollection(state);
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -441,7 +479,8 @@ public class ContainerStateMap {
|
|
|
Preconditions.checkNotNull(factor, "Factor cannot be null");
|
|
|
Preconditions.checkNotNull(type, "Type cannot be null");
|
|
|
|
|
|
- try (AutoCloseableLock lock = autoLock.acquire()) {
|
|
|
+ lock.readLock().lock();
|
|
|
+ try {
|
|
|
|
|
|
// If we cannot meet any one condition we return EMPTY_SET immediately.
|
|
|
// Since when we intersect these sets, the result will be empty if any
|
|
@@ -479,6 +518,8 @@ public class ContainerStateMap {
|
|
|
currentSet = intersectSets(currentSet, sets[x]);
|
|
|
}
|
|
|
return currentSet;
|
|
|
+ } finally {
|
|
|
+ lock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|