|
@@ -48,8 +48,6 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
-import java.util.concurrent.locks.ReadWriteLock;
|
|
|
-import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
|
.CHILL_MODE_EXCEPTION;
|
|
@@ -76,7 +74,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
private final NodeManager nodeManager;
|
|
|
private final Mapping containerManager;
|
|
|
|
|
|
- private final ReadWriteLock lock;
|
|
|
private final long containerSize;
|
|
|
|
|
|
private final DeletedBlockLog deletedBlockLog;
|
|
@@ -113,7 +110,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
|
|
|
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
|
|
|
rand = new Random();
|
|
|
- this.lock = new ReentrantReadWriteLock();
|
|
|
|
|
|
mxBean = MBeans.register("BlockManager", "BlockManagerImpl", this);
|
|
|
|
|
@@ -223,74 +219,29 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
|
|
|
ContainerWithPipeline containerWithPipeline;
|
|
|
|
|
|
- lock.readLock().lock();
|
|
|
- try {
|
|
|
- // This is to optimize performance, if the below condition is evaluated
|
|
|
- // to false, then we can be sure that there are no containers in
|
|
|
- // ALLOCATED state.
|
|
|
- // This can result in false positive, but it will never be false negative.
|
|
|
- // How can this result in false positive? We check if there are any
|
|
|
- // containers in ALLOCATED state, this check doesn't care about the
|
|
|
- // USER of the containers. So there might be cases where a different
|
|
|
- // USER has few containers in ALLOCATED state, which will result in
|
|
|
- // false positive.
|
|
|
- if (!containerManager.getStateManager().getContainerStateMap()
|
|
|
- .getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
|
|
- .isEmpty()) {
|
|
|
- // Since the above check can result in false positive, we have to do
|
|
|
- // the actual check and find out if there are containers in ALLOCATED
|
|
|
- // state matching our criteria.
|
|
|
- synchronized (this) {
|
|
|
- // Using containers from ALLOCATED state should be done within
|
|
|
- // synchronized block (or) write lock. Since we already hold a
|
|
|
- // read lock, we will end up in deadlock situation if we take
|
|
|
- // write lock here.
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
- HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
- if (containerWithPipeline != null) {
|
|
|
- containerManager.updateContainerState(
|
|
|
- containerWithPipeline.getContainerInfo().getContainerID(),
|
|
|
- HddsProtos.LifeCycleEvent.CREATE);
|
|
|
- return newBlock(containerWithPipeline,
|
|
|
- HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Since we found no allocated containers that match our criteria, let us
|
|
|
- // look for OPEN containers that match the criteria.
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
- HddsProtos.LifeCycleState.OPEN);
|
|
|
- if (containerWithPipeline != null) {
|
|
|
- return newBlock(containerWithPipeline, HddsProtos.LifeCycleState.OPEN);
|
|
|
- }
|
|
|
-
|
|
|
- // We found neither ALLOCATED or OPEN Containers. This generally means
|
|
|
- // that most of our containers are full or we have not allocated
|
|
|
- // containers of the type and replication factor. So let us go and
|
|
|
- // allocate some.
|
|
|
-
|
|
|
- // Even though we have already checked the containers in ALLOCATED
|
|
|
- // state, we have to check again as we only hold a read lock.
|
|
|
- // Some other thread might have pre-allocated container in meantime.
|
|
|
+ // This is to optimize performance, if the below condition is evaluated
|
|
|
+ // to false, then we can be sure that there are no containers in
|
|
|
+ // ALLOCATED state.
|
|
|
+ // This can result in false positive, but it will never be false negative.
|
|
|
+ // How can this result in false positive? We check if there are any
|
|
|
+ // containers in ALLOCATED state, this check doesn't care about the
|
|
|
+ // USER of the containers. So there might be cases where a different
|
|
|
+ // USER has few containers in ALLOCATED state, which will result in
|
|
|
+ // false positive.
|
|
|
+ if (!containerManager.getStateManager().getContainerStateMap()
|
|
|
+ .getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
|
|
+ .isEmpty()) {
|
|
|
+ // Since the above check can result in false positive, we have to do
|
|
|
+ // the actual check and find out if there are containers in ALLOCATED
|
|
|
+ // state matching our criteria.
|
|
|
synchronized (this) {
|
|
|
- if (!containerManager.getStateManager().getContainerStateMap()
|
|
|
- .getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
|
|
- .isEmpty()) {
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
- HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
- }
|
|
|
- if (containerWithPipeline == null) {
|
|
|
- preAllocateContainers(containerProvisionBatchSize,
|
|
|
- type, factor, owner);
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
- HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
- }
|
|
|
-
|
|
|
+ // Using containers from ALLOCATED state should be done within
|
|
|
+ // synchronized block (or) write lock. Since we already hold a
|
|
|
+ // read lock, we will end up in deadlock situation if we take
|
|
|
+ // write lock here.
|
|
|
+ containerWithPipeline = containerManager
|
|
|
+ .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
if (containerWithPipeline != null) {
|
|
|
containerManager.updateContainerState(
|
|
|
containerWithPipeline.getContainerInfo().getContainerID(),
|
|
@@ -299,19 +250,55 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
}
|
|
|
}
|
|
|
- // we have tried all strategies we know and but somehow we are not able
|
|
|
- // to get a container for this block. Log that info and return a null.
|
|
|
- LOG.error(
|
|
|
- "Unable to allocate a block for the size: {}, type: {}, " +
|
|
|
- "factor: {}",
|
|
|
- size,
|
|
|
- type,
|
|
|
- factor);
|
|
|
- return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Since we found no allocated containers that match our criteria, let us
|
|
|
+ // look for OPEN containers that match the criteria.
|
|
|
+ containerWithPipeline = containerManager
|
|
|
+ .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ HddsProtos.LifeCycleState.OPEN);
|
|
|
+ if (containerWithPipeline != null) {
|
|
|
+ return newBlock(containerWithPipeline, HddsProtos.LifeCycleState.OPEN);
|
|
|
+ }
|
|
|
+
|
|
|
+ // We found neither ALLOCATED or OPEN Containers. This generally means
|
|
|
+ // that most of our containers are full or we have not allocated
|
|
|
+ // containers of the type and replication factor. So let us go and
|
|
|
+ // allocate some.
|
|
|
+
|
|
|
+ // Even though we have already checked the containers in ALLOCATED
|
|
|
+ // state, we have to check again as we only hold a read lock.
|
|
|
+ // Some other thread might have pre-allocated container in meantime.
|
|
|
+ synchronized (this) {
|
|
|
+ if (!containerManager.getStateManager().getContainerStateMap()
|
|
|
+ .getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
|
|
+ .isEmpty()) {
|
|
|
+ containerWithPipeline = containerManager
|
|
|
+ .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
+ }
|
|
|
+ if (containerWithPipeline == null) {
|
|
|
+ preAllocateContainers(containerProvisionBatchSize,
|
|
|
+ type, factor, owner);
|
|
|
+ containerWithPipeline = containerManager
|
|
|
+ .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
+ }
|
|
|
|
|
|
- } finally {
|
|
|
- lock.readLock().unlock();
|
|
|
+ if (containerWithPipeline != null) {
|
|
|
+ containerManager.updateContainerState(
|
|
|
+ containerWithPipeline.getContainerInfo().getContainerID(),
|
|
|
+ HddsProtos.LifeCycleEvent.CREATE);
|
|
|
+ return newBlock(containerWithPipeline,
|
|
|
+ HddsProtos.LifeCycleState.ALLOCATED);
|
|
|
+ }
|
|
|
}
|
|
|
+ // we have tried all strategies we know and but somehow we are not able
|
|
|
+ // to get a container for this block. Log that info and return a null.
|
|
|
+ LOG.error(
|
|
|
+ "Unable to allocate a block for the size: {}, type: {}, factor: {}",
|
|
|
+ size, type, factor);
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -330,9 +317,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
}
|
|
|
|
|
|
// TODO : Revisit this local ID allocation when HA is added.
|
|
|
- // TODO: this does not work well if multiple allocation kicks in a tight
|
|
|
- // loop.
|
|
|
- long localID = Time.getUtcTime();
|
|
|
+ long localID = UniqueId.next();
|
|
|
long containerID = containerInfo.getContainerID();
|
|
|
|
|
|
boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED);
|
|
@@ -463,4 +448,51 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
public static Logger getLogger() {
|
|
|
return LOG;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This class uses system current time milliseconds to generate unique id.
|
|
|
+ */
|
|
|
+ public static final class UniqueId {
|
|
|
+ /*
|
|
|
+ * When we represent time in milliseconds using 'long' data type,
|
|
|
+ * the LSB bits are used. Currently we are only using 44 bits (LSB),
|
|
|
+ * 20 bits (MSB) are not used.
|
|
|
+ * We will exhaust this 44 bits only when we are in year 2525,
|
|
|
+ * until then we can safely use this 20 bits (MSB) for offset to generate
|
|
|
+ * unique id within millisecond.
|
|
|
+ *
|
|
|
+ * Year : Mon Dec 31 18:49:04 IST 2525
|
|
|
+ * TimeInMillis: 17545641544247
|
|
|
+ * Binary Representation:
|
|
|
+ * MSB (20 bits): 0000 0000 0000 0000 0000
|
|
|
+ * LSB (44 bits): 1111 1111 0101 0010 1001 1011 1011 0100 1010 0011 0111
|
|
|
+ *
|
|
|
+ * We have 20 bits to run counter, we should exclude the first bit (MSB)
|
|
|
+ * as we don't want to deal with negative values.
|
|
|
+ * To be on safer side we will use 'short' data type which is of length
|
|
|
+ * 16 bits and will give us 65,536 values for offset.
|
|
|
+ *
|
|
|
+ */
|
|
|
+
|
|
|
+ private static volatile short offset = 0;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Private constructor so that no one can instantiate this class.
|
|
|
+ */
|
|
|
+ private UniqueId() {}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Calculate and returns next unique id based on System#currentTimeMillis.
|
|
|
+ *
|
|
|
+ * @return unique long value
|
|
|
+ */
|
|
|
+ public static synchronized long next() {
|
|
|
+ long utcTime = Time.getUtcTime();
|
|
|
+ if ((utcTime & 0xFFFF000000000000L) == 0) {
|
|
|
+ return utcTime << Short.SIZE | (offset++ & 0x0000FFFF);
|
|
|
+ }
|
|
|
+ throw new RuntimeException("Got invalid UTC time," +
|
|
|
+ " cannot generate unique Id. UTC Time: " + utcTime);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|