HDDS-475. Block Allocation returns same BlockID on different keys creation.
Contributed by Nanda Kumar.
This commit is contained in:
parent
26d0c63a1e
commit
23a6137a40
|
@ -48,8 +48,6 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.concurrent.TimeUnit;
|
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
|
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
||||||
.CHILL_MODE_EXCEPTION;
|
.CHILL_MODE_EXCEPTION;
|
||||||
|
@ -76,7 +74,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
private final NodeManager nodeManager;
|
private final NodeManager nodeManager;
|
||||||
private final Mapping containerManager;
|
private final Mapping containerManager;
|
||||||
|
|
||||||
private final ReadWriteLock lock;
|
|
||||||
private final long containerSize;
|
private final long containerSize;
|
||||||
|
|
||||||
private final DeletedBlockLog deletedBlockLog;
|
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,
|
||||||
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
|
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
|
||||||
rand = new Random();
|
rand = new Random();
|
||||||
this.lock = new ReentrantReadWriteLock();
|
|
||||||
|
|
||||||
mxBean = MBeans.register("BlockManager", "BlockManagerImpl", this);
|
mxBean = MBeans.register("BlockManager", "BlockManagerImpl", this);
|
||||||
|
|
||||||
|
@ -223,74 +219,29 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
|
|
||||||
ContainerWithPipeline containerWithPipeline;
|
ContainerWithPipeline containerWithPipeline;
|
||||||
|
|
||||||
lock.readLock().lock();
|
// This is to optimize performance, if the below condition is evaluated
|
||||||
try {
|
// to false, then we can be sure that there are no containers in
|
||||||
// This is to optimize performance, if the below condition is evaluated
|
// ALLOCATED state.
|
||||||
// to false, then we can be sure that there are no containers in
|
// This can result in false positive, but it will never be false negative.
|
||||||
// ALLOCATED state.
|
// How can this result in false positive? We check if there are any
|
||||||
// This can result in false positive, but it will never be false negative.
|
// containers in ALLOCATED state, this check doesn't care about the
|
||||||
// How can this result in false positive? We check if there are any
|
// USER of the containers. So there might be cases where a different
|
||||||
// containers in ALLOCATED state, this check doesn't care about the
|
// USER has few containers in ALLOCATED state, which will result in
|
||||||
// USER of the containers. So there might be cases where a different
|
// false positive.
|
||||||
// USER has few containers in ALLOCATED state, which will result in
|
if (!containerManager.getStateManager().getContainerStateMap()
|
||||||
// false positive.
|
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
||||||
if (!containerManager.getStateManager().getContainerStateMap()
|
.isEmpty()) {
|
||||||
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
// Since the above check can result in false positive, we have to do
|
||||||
.isEmpty()) {
|
// the actual check and find out if there are containers in ALLOCATED
|
||||||
// Since the above check can result in false positive, we have to do
|
// state matching our criteria.
|
||||||
// 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.
|
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
if (!containerManager.getStateManager().getContainerStateMap()
|
// Using containers from ALLOCATED state should be done within
|
||||||
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
|
// synchronized block (or) write lock. Since we already hold a
|
||||||
.isEmpty()) {
|
// read lock, we will end up in deadlock situation if we take
|
||||||
containerWithPipeline = containerManager
|
// write lock here.
|
||||||
.getMatchingContainerWithPipeline(size, owner, type, factor,
|
containerWithPipeline = containerManager
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
.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);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (containerWithPipeline != null) {
|
if (containerWithPipeline != null) {
|
||||||
containerManager.updateContainerState(
|
containerManager.updateContainerState(
|
||||||
containerWithPipeline.getContainerInfo().getContainerID(),
|
containerWithPipeline.getContainerInfo().getContainerID(),
|
||||||
|
@ -299,19 +250,55 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
HddsProtos.LifeCycleState.ALLOCATED);
|
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;
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
lock.readLock().unlock();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// 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);
|
||||||
|
}
|
||||||
|
|
||||||
|
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 : Revisit this local ID allocation when HA is added.
|
||||||
// TODO: this does not work well if multiple allocation kicks in a tight
|
long localID = UniqueId.next();
|
||||||
// loop.
|
|
||||||
long localID = Time.getUtcTime();
|
|
||||||
long containerID = containerInfo.getContainerID();
|
long containerID = containerInfo.getContainerID();
|
||||||
|
|
||||||
boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED);
|
boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED);
|
||||||
|
@ -463,4 +448,51 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
public static Logger getLogger() {
|
public static Logger getLogger() {
|
||||||
return LOG;
|
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);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue