|
@@ -1,4 +1,4 @@
|
|
|
-/**
|
|
|
+/*
|
|
|
* 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
|
|
@@ -17,16 +17,19 @@
|
|
|
|
|
|
package org.apache.hadoop.ozone.scm.container;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
-import org.apache.commons.lang3.builder.EqualsBuilder;
|
|
|
-import org.apache.commons.lang3.builder.HashCodeBuilder;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
|
|
|
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleEvent;
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState;
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
|
|
|
+import org.apache.hadoop.ozone.scm.container.ContainerStates.ContainerID;
|
|
|
+import org.apache.hadoop.ozone.scm.container.ContainerStates.ContainerState;
|
|
|
+import org.apache.hadoop.ozone.scm.container.ContainerStates.ContainerStateMap;
|
|
|
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
|
|
|
import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
|
|
|
import org.apache.hadoop.scm.ScmConfigKeys;
|
|
@@ -39,26 +42,15 @@ import org.slf4j.LoggerFactory;
|
|
|
import java.io.Closeable;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
-import java.util.Iterator;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.Queue;
|
|
|
-import java.util.Set;
|
|
|
-import java.util.PriorityQueue;
|
|
|
import java.util.List;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.concurrent.ConcurrentLinkedQueue;
|
|
|
-import java.util.concurrent.locks.ReadWriteLock;
|
|
|
-import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
-
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleEvent;
|
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
|
|
|
+import java.util.NavigableSet;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
-import static org.apache.hadoop.ozone.scm.exceptions
|
|
|
- .SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE;
|
|
|
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
|
|
|
+ .FAILED_TO_CHANGE_CONTAINER_STATE;
|
|
|
|
|
|
/**
|
|
|
* A container state manager keeps track of container states and returns
|
|
@@ -86,7 +78,7 @@ import static org.apache.hadoop.ozone.scm.exceptions
|
|
|
* this container.
|
|
|
* <p>
|
|
|
* 4. Once the creation of the container is complete, the client will make
|
|
|
- * another call to the SCM, this time specifing the containerName and the
|
|
|
+ * another call to the SCM, this time specifying the containerName and the
|
|
|
* COMPLETE_CREATE as the Event.
|
|
|
* <p>
|
|
|
* 5. With COMPLETE_CREATE event, the container moves to an Open State. This is
|
|
@@ -125,14 +117,9 @@ public class ContainerStateManager implements Closeable {
|
|
|
OzoneProtos.LifeCycleEvent> stateMachine;
|
|
|
|
|
|
private final long containerSize;
|
|
|
- private final long cacheSize;
|
|
|
- private final long blockSize;
|
|
|
-
|
|
|
- // A map that maintains the ContainerKey to Containers of that type ordered
|
|
|
- // by last access time.
|
|
|
- private final ReadWriteLock lock;
|
|
|
- private final Queue<ContainerInfo> containerCloseQueue;
|
|
|
- private Map<ContainerKey, PriorityQueue<ContainerInfo>> containers;
|
|
|
+ private final ConcurrentHashMap<ContainerState, ContainerID> lastUsedMap;
|
|
|
+ private final ContainerStateMap containers;
|
|
|
+ private final AtomicLong containerCount;
|
|
|
|
|
|
/**
|
|
|
* Constructs a Container State Manager that tracks all containers owned by
|
|
@@ -140,9 +127,9 @@ public class ContainerStateManager implements Closeable {
|
|
|
* <p>
|
|
|
* TODO : Add Container Tags so we know which containers are owned by SCM.
|
|
|
*/
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
public ContainerStateManager(Configuration configuration,
|
|
|
- Mapping containerMapping, final long cacheSize) throws IOException {
|
|
|
- this.cacheSize = cacheSize;
|
|
|
+ Mapping containerMapping) {
|
|
|
|
|
|
// Initialize the container state machine.
|
|
|
Set<OzoneProtos.LifeCycleState> finalStates = new HashSet();
|
|
@@ -160,68 +147,46 @@ public class ContainerStateManager implements Closeable {
|
|
|
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
|
|
|
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
|
|
|
|
|
|
- this.blockSize = OzoneConsts.MB * configuration.getLong(
|
|
|
- OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB,
|
|
|
- OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
|
|
|
-
|
|
|
- lock = new ReentrantReadWriteLock();
|
|
|
- containers = new HashMap<>();
|
|
|
+ lastUsedMap = new ConcurrentHashMap<>();
|
|
|
+ containerCount = new AtomicLong(0);
|
|
|
+ containers = new ContainerStateMap();
|
|
|
loadExistingContainers(containerMapping);
|
|
|
- containerCloseQueue = new ConcurrentLinkedQueue<>();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Creates containers maps of following types.
|
|
|
- * <p>
|
|
|
- * OZONE of type {Ratis, StandAlone, Chained} for each of these {ALLOCATED,
|
|
|
- * CREATING, OPEN, CLOSED, DELETING, DELETED} container states
|
|
|
- * <p>
|
|
|
- * CBLOCK of type {Ratis, StandAlone, Chained} for each of these {ALLOCATED,
|
|
|
- * CREATING, OPEN, CLOSED, DELETING, DELETED} container states
|
|
|
- * <p>
|
|
|
- * Commented out for now: HDFS of type {Ratis, StandAlone, Chained} for each
|
|
|
- * of these {ALLOCATED, CREATING, OPEN, CLOSED, DELETING, DELETED} container
|
|
|
- * states
|
|
|
- */
|
|
|
- private void initializeContainerMaps(String owner) {
|
|
|
- // Called only from Ctor path, hence no lock is held.
|
|
|
- Preconditions.checkNotNull(containers);
|
|
|
- for (ReplicationType type : ReplicationType.values()) {
|
|
|
- for (ReplicationFactor factor : ReplicationFactor.values()) {
|
|
|
- for (LifeCycleState state : LifeCycleState.values()) {
|
|
|
- ContainerKey key = new ContainerKey(owner, type, factor, state);
|
|
|
- PriorityQueue<ContainerInfo> queue = new PriorityQueue<>();
|
|
|
- containers.put(key, queue);
|
|
|
- }
|
|
|
+ private void loadExistingContainers(Mapping containerMapping) {
|
|
|
+
|
|
|
+ List<ContainerInfo> containerList;
|
|
|
+ try {
|
|
|
+ containerList = containerMapping.listContainer(null,
|
|
|
+ null, Integer.MAX_VALUE);
|
|
|
+
|
|
|
+ // if there are no container to load, let us return.
|
|
|
+ if (containerList == null || containerList.size() == 0) {
|
|
|
+ LOG.info("No containers to load for this cluster.");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ if (!e.getMessage().equals("No container exists in current db")) {
|
|
|
+ LOG.error("Could not list the containers", e);
|
|
|
}
|
|
|
+ return;
|
|
|
}
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Load containers from the container store into the containerMaps.
|
|
|
- *
|
|
|
- * @param containerMapping -- Mapping object containing container store.
|
|
|
- */
|
|
|
- private void loadExistingContainers(Mapping containerMapping) {
|
|
|
try {
|
|
|
- List<String> ownerList = new ArrayList<>();
|
|
|
- List<ContainerInfo> containerList =
|
|
|
- containerMapping.listContainer(null, null, Integer.MAX_VALUE);
|
|
|
+ long maxID = 0;
|
|
|
for (ContainerInfo container : containerList) {
|
|
|
- String owner = container.getOwner();
|
|
|
- if (ownerList.isEmpty() || !ownerList.contains(owner)) {
|
|
|
- ownerList.add(owner);
|
|
|
- initializeContainerMaps(owner);
|
|
|
+ containers.addContainer(container);
|
|
|
+
|
|
|
+ if (maxID < container.getContainerID()) {
|
|
|
+ maxID = container.getContainerID();
|
|
|
}
|
|
|
- ContainerKey key =
|
|
|
- new ContainerKey(owner, container.getPipeline().getType(),
|
|
|
- container.getPipeline().getFactor(), container.getState());
|
|
|
- containers.get(key).add(container);
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- if (!e.getMessage().equals("No container exists in current db")) {
|
|
|
- LOG.info("Could not list the containers", e);
|
|
|
+
|
|
|
+ containerCount.set(maxID);
|
|
|
}
|
|
|
+ } catch (SCMException ex) {
|
|
|
+ LOG.error("Unable to create a container information. ", ex);
|
|
|
+ // Fix me, what is the proper shutdown procedure for SCM ??
|
|
|
+ // System.exit(1) // Should we exit here?
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -230,9 +195,11 @@ public class ContainerStateManager implements Closeable {
|
|
|
*
|
|
|
* @return the list of all container info.
|
|
|
*/
|
|
|
- List<ContainerInfo> getAllContainers() {
|
|
|
+ public List<ContainerInfo> getAllContainers() {
|
|
|
List<ContainerInfo> list = new ArrayList<>();
|
|
|
- containers.forEach((key, value) -> list.addAll(value));
|
|
|
+
|
|
|
+ //No Locking needed since the return value is an immutable map.
|
|
|
+ containers.getContainerMap().forEach((key, value) -> list.add(value));
|
|
|
return list;
|
|
|
}
|
|
|
|
|
@@ -315,7 +282,7 @@ public class ContainerStateManager implements Closeable {
|
|
|
* @param replicationFactor - Replication replicationFactor.
|
|
|
* @param containerName - Container Name.
|
|
|
* @return Container Info.
|
|
|
- * @throws IOException
|
|
|
+ * @throws IOException on Failure.
|
|
|
*/
|
|
|
public ContainerInfo allocateContainer(PipelineSelector selector, OzoneProtos
|
|
|
.ReplicationType type, OzoneProtos.ReplicationFactor replicationFactor,
|
|
@@ -335,22 +302,11 @@ public class ContainerStateManager implements Closeable {
|
|
|
.setNumberOfKeys(0)
|
|
|
.setStateEnterTime(Time.monotonicNow())
|
|
|
.setOwner(owner)
|
|
|
+ .setContainerID(containerCount.incrementAndGet())
|
|
|
.build();
|
|
|
Preconditions.checkNotNull(containerInfo);
|
|
|
- lock.writeLock().lock();
|
|
|
- try {
|
|
|
- ContainerKey key = new ContainerKey(owner, type, replicationFactor,
|
|
|
- containerInfo.getState());
|
|
|
- PriorityQueue<ContainerInfo> queue = containers.get(key);
|
|
|
- if (queue == null) {
|
|
|
- initializeContainerMaps(owner);
|
|
|
- queue = containers.get(key);
|
|
|
- }
|
|
|
- queue.add(containerInfo);
|
|
|
- LOG.trace("New container allocated: {}", containerInfo);
|
|
|
- } finally {
|
|
|
- lock.writeLock().unlock();
|
|
|
- }
|
|
|
+ containers.addContainer(containerInfo);
|
|
|
+ LOG.trace("New container allocated: {}", containerInfo);
|
|
|
return containerInfo;
|
|
|
}
|
|
|
|
|
@@ -360,7 +316,7 @@ public class ContainerStateManager implements Closeable {
|
|
|
* @param info - ContainerInfo
|
|
|
* @param event - LifeCycle Event
|
|
|
* @return Updated ContainerInfo.
|
|
|
- * @throws SCMException
|
|
|
+ * @throws SCMException on Failure.
|
|
|
*/
|
|
|
public ContainerInfo updateContainerState(ContainerInfo
|
|
|
info, OzoneProtos.LifeCycleEvent event) throws SCMException {
|
|
@@ -369,7 +325,8 @@ public class ContainerStateManager implements Closeable {
|
|
|
newState = this.stateMachine.getNextState(info.getState(), event);
|
|
|
} catch (InvalidStateTransitionException ex) {
|
|
|
String error = String.format("Failed to update container state %s, " +
|
|
|
- "reason: invalid state transition from state: %s upon event: %s.",
|
|
|
+ "reason: invalid state transition from state: %s upon " +
|
|
|
+ "event: %s.",
|
|
|
info.getPipeline().getContainerName(), info.getState(), event);
|
|
|
LOG.error(error);
|
|
|
throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
|
|
@@ -377,191 +334,119 @@ public class ContainerStateManager implements Closeable {
|
|
|
|
|
|
// This is a post condition after executing getNextState.
|
|
|
Preconditions.checkNotNull(newState);
|
|
|
- Pipeline pipeline = info.getPipeline();
|
|
|
-
|
|
|
- ContainerKey oldKey = new ContainerKey(info.getOwner(), pipeline.getType(),
|
|
|
- pipeline.getFactor(), info.getState());
|
|
|
-
|
|
|
- ContainerKey newKey = new ContainerKey(info.getOwner(), pipeline.getType(),
|
|
|
- pipeline.getFactor(), newState);
|
|
|
- lock.writeLock().lock();
|
|
|
- try {
|
|
|
-
|
|
|
- PriorityQueue<ContainerInfo> currentQueue = containers.get(oldKey);
|
|
|
- // This should never happen, since we have initialized the map and
|
|
|
- // queues to all possible states. No harm in asserting that info.
|
|
|
- Preconditions.checkNotNull(currentQueue);
|
|
|
-
|
|
|
- // TODO : Should we read this container info from the database if this
|
|
|
- // is missing in the queue?. Right now we just add it into the queue.
|
|
|
- // We also need a background thread that will remove unused containers
|
|
|
- // from memory after 24 hours. This is really a low priority work item
|
|
|
- // since typical clusters will have less than 10's of millions of open
|
|
|
- // containers at a given time, which we can easily keep in memory.
|
|
|
-
|
|
|
- if (currentQueue.contains(info)) {
|
|
|
- currentQueue.remove(info);
|
|
|
- }
|
|
|
+ containers.updateState(info, info.getState(), newState);
|
|
|
+ return containers.getContainerInfo(info);
|
|
|
+ }
|
|
|
|
|
|
- PriorityQueue<ContainerInfo> nextQueue = containers.get(newKey);
|
|
|
- Preconditions.checkNotNull(nextQueue);
|
|
|
-
|
|
|
- ContainerInfo containerInfo = new ContainerInfo.Builder()
|
|
|
- .setContainerName(info.getContainerName())
|
|
|
- .setState(newState)
|
|
|
- .setPipeline(info.getPipeline())
|
|
|
- .setAllocatedBytes(info.getAllocatedBytes())
|
|
|
- .setUsedBytes(info.getUsedBytes())
|
|
|
- .setNumberOfKeys(info.getNumberOfKeys())
|
|
|
- .setStateEnterTime(Time.monotonicNow())
|
|
|
- .setOwner(info.getOwner())
|
|
|
- .build();
|
|
|
- Preconditions.checkNotNull(containerInfo);
|
|
|
- nextQueue.add(containerInfo);
|
|
|
-
|
|
|
- return containerInfo;
|
|
|
- } finally {
|
|
|
- lock.writeLock().unlock();
|
|
|
- }
|
|
|
+ /**
|
|
|
+ * Update the container State.
|
|
|
+ * @param info - Container Info
|
|
|
+ * @return ContainerInfo
|
|
|
+ * @throws SCMException - on Error.
|
|
|
+ */
|
|
|
+ public ContainerInfo updateContainerInfo(ContainerInfo info)
|
|
|
+ throws SCMException {
|
|
|
+ containers.updateContainerInfo(info);
|
|
|
+ return containers.getContainerInfo(info);
|
|
|
}
|
|
|
|
|
|
+
|
|
|
/**
|
|
|
* Return a container matching the attributes specified.
|
|
|
*
|
|
|
* @param size - Space needed in the Container.
|
|
|
- * @param owner - Owner of the container {OZONE, CBLOCK}
|
|
|
+ * @param owner - Owner of the container - A specific nameservice.
|
|
|
* @param type - Replication Type {StandAlone, Ratis}
|
|
|
* @param factor - Replication Factor {ONE, THREE}
|
|
|
* @param state - State of the Container-- {Open, Allocated etc.}
|
|
|
- * @return ContainerInfo
|
|
|
+ * @return ContainerInfo, null if there is no match found.
|
|
|
*/
|
|
|
public ContainerInfo getMatchingContainer(final long size,
|
|
|
String owner, ReplicationType type, ReplicationFactor factor,
|
|
|
LifeCycleState state) {
|
|
|
- ContainerKey key = new ContainerKey(owner, type, factor, state);
|
|
|
- lock.writeLock().lock();
|
|
|
- try {
|
|
|
- PriorityQueue<ContainerInfo> queue = containers.get(key);
|
|
|
- if (queue == null) {
|
|
|
- initializeContainerMaps(owner);
|
|
|
- queue = containers.get(key);
|
|
|
- }
|
|
|
- if (queue.size() == 0) {
|
|
|
- // We don't have any Containers of this type.
|
|
|
- return null;
|
|
|
- }
|
|
|
- Iterator<ContainerInfo> iter = queue.iterator();
|
|
|
- // Two assumptions here.
|
|
|
- // 1. The Iteration on the heap is in ordered by the last used time.
|
|
|
- // 2. We remove and add the node back to push the node to the end of
|
|
|
- // the queue.
|
|
|
-
|
|
|
- while (iter.hasNext()) {
|
|
|
- ContainerInfo info = iter.next();
|
|
|
- if (info.getAllocatedBytes() + size <= this.containerSize) {
|
|
|
- queue.remove(info);
|
|
|
- info.allocate(size);
|
|
|
- info.updateLastUsedTime();
|
|
|
- queue.add(info);
|
|
|
-
|
|
|
- return info;
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- } finally {
|
|
|
- lock.writeLock().unlock();
|
|
|
+ // Find containers that match the query spec, if no match return null.
|
|
|
+ NavigableSet<ContainerID> matchingSet =
|
|
|
+ containers.getMatchingContainerIDs(state, owner, factor, type);
|
|
|
+ if (matchingSet == null || matchingSet.size() == 0) {
|
|
|
+ return null;
|
|
|
}
|
|
|
- return null;
|
|
|
- }
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public List<ContainerInfo> getMatchingContainers(String owner,
|
|
|
- ReplicationType type, ReplicationFactor factor, LifeCycleState state) {
|
|
|
- ContainerKey key = new ContainerKey(owner, type, factor, state);
|
|
|
- lock.readLock().lock();
|
|
|
- try {
|
|
|
- if (containers.get(key) == null) {
|
|
|
- return null;
|
|
|
- } else {
|
|
|
- return Arrays.asList((ContainerInfo[]) containers.get(key)
|
|
|
- .toArray(new ContainerInfo[0]));
|
|
|
- }
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Could not get matching containers", e);
|
|
|
- } finally {
|
|
|
- lock.readLock().unlock();
|
|
|
+ // Get the last used container and find container above the last used
|
|
|
+ // container ID.
|
|
|
+ ContainerState key = new ContainerState(owner, type, factor);
|
|
|
+ ContainerID lastID = lastUsedMap.get(key);
|
|
|
+ if(lastID == null) {
|
|
|
+ lastID = matchingSet.first();
|
|
|
}
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void close() throws IOException {
|
|
|
- //TODO: update container metadata db with actual allocated bytes values.
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Class that acts as the container Key.
|
|
|
- */
|
|
|
- private static class ContainerKey {
|
|
|
- private final LifeCycleState state;
|
|
|
- private final ReplicationType type;
|
|
|
- private final String owner;
|
|
|
- private final ReplicationFactor replicationFactor;
|
|
|
-
|
|
|
- /**
|
|
|
- * Constructs a Container Key.
|
|
|
- *
|
|
|
- * @param owner - Container Owners
|
|
|
- * @param type - Replication Type.
|
|
|
- * @param factor - Replication Factors
|
|
|
- * @param state - LifeCycle State
|
|
|
- */
|
|
|
- ContainerKey(String owner, ReplicationType type,
|
|
|
- ReplicationFactor factor, LifeCycleState state) {
|
|
|
- this.state = state;
|
|
|
- this.type = type;
|
|
|
- this.owner = owner;
|
|
|
- this.replicationFactor = factor;
|
|
|
+ // There is a small issue here. The first time, we will skip the first
|
|
|
+ // container. But in most cases it will not matter.
|
|
|
+ NavigableSet<ContainerID> resultSet = matchingSet.tailSet(lastID, false);
|
|
|
+ if (resultSet.size() == 0) {
|
|
|
+ resultSet = matchingSet;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public boolean equals(Object o) {
|
|
|
- if (this == o) {
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- if (o == null || getClass() != o.getClass()) {
|
|
|
- return false;
|
|
|
- }
|
|
|
+ ContainerInfo selectedContainer =
|
|
|
+ findContainerWithSpace(size, resultSet, owner);
|
|
|
+ if (selectedContainer == null) {
|
|
|
|
|
|
- ContainerKey that = (ContainerKey) o;
|
|
|
+ // If we did not find any space in the tailSet, we need to look for
|
|
|
+ // space in the headset, we need to pass true to deal with the
|
|
|
+ // situation that we have a lone container that has space. That is we
|
|
|
+ // ignored the last used container under the assumption we can find
|
|
|
+ // other containers with space, but if have a single container that is
|
|
|
+ // not true. Hence we need to include the last used container as the
|
|
|
+ // last element in the sorted set.
|
|
|
|
|
|
- return new EqualsBuilder()
|
|
|
- .append(state, that.state)
|
|
|
- .append(type, that.type)
|
|
|
- .append(owner, that.owner)
|
|
|
- .append(replicationFactor, that.replicationFactor)
|
|
|
- .isEquals();
|
|
|
+ resultSet = matchingSet.headSet(lastID, true);
|
|
|
+ selectedContainer = findContainerWithSpace(size, resultSet, owner);
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public int hashCode() {
|
|
|
- return new HashCodeBuilder(137, 757)
|
|
|
- .append(state)
|
|
|
- .append(type)
|
|
|
- .append(owner)
|
|
|
- .append(replicationFactor)
|
|
|
- .toHashCode();
|
|
|
+ // Update the allocated Bytes on this container.
|
|
|
+ if(selectedContainer != null) {
|
|
|
+ selectedContainer.updateAllocatedBytes(size);
|
|
|
}
|
|
|
+ return selectedContainer;
|
|
|
|
|
|
- @Override
|
|
|
- public String toString() {
|
|
|
- return "ContainerKey{" +
|
|
|
- "state=" + state +
|
|
|
- ", type=" + type +
|
|
|
- ", owner=" + owner +
|
|
|
- ", replicationFactor=" + replicationFactor +
|
|
|
- '}';
|
|
|
+ }
|
|
|
+
|
|
|
+ private ContainerInfo findContainerWithSpace(long size,
|
|
|
+ NavigableSet<ContainerID> searchSet, String owner) {
|
|
|
+ // Get the container with space to meet our request.
|
|
|
+ for (ContainerID id : searchSet) {
|
|
|
+ ContainerInfo containerInfo = containers.getContainerInfo(id.getId());
|
|
|
+ if ((containerInfo.getAllocatedBytes() <= this.containerSize) &&
|
|
|
+ (containerInfo.getAllocatedBytes() <= size)) {
|
|
|
+ containerInfo.updateLastUsedTime();
|
|
|
+
|
|
|
+ ContainerState key = new ContainerState(owner,
|
|
|
+ containerInfo.getPipeline().getType(),
|
|
|
+ containerInfo.getPipeline().getFactor());
|
|
|
+ lastUsedMap.put(key, containerInfo.containerID());
|
|
|
+ return containerInfo;
|
|
|
+ }
|
|
|
}
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns a set of ContainerIDs that match the Container.
|
|
|
+ *
|
|
|
+ * @param owner Owner of the Containers.
|
|
|
+ * @param type - Replication Type of the containers
|
|
|
+ * @param factor - Replication factor of the containers.
|
|
|
+ * @param state - Current State, like Open, Close etc.
|
|
|
+ * @return Set of containers that match the specific query parameters.
|
|
|
+ */
|
|
|
+ public NavigableSet<ContainerID> getMatchingContainerIDs(
|
|
|
+ String owner, ReplicationType type, ReplicationFactor factor,
|
|
|
+ LifeCycleState state) {
|
|
|
+ return containers.getMatchingContainerIDs(state, owner,
|
|
|
+ factor, type);
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void close() throws IOException {
|
|
|
+ }
|
|
|
+
|
|
|
}
|