HDDS-211. Add a create container Lock. Contributed by Bharat Viswanadham
This commit is contained in:
parent
7dcf5878a4
commit
e899c4cc01
@ -119,8 +119,10 @@ public ContainerCommandResponseProto dispatch(
|
||||
responseProto = handler.handle(msg, container);
|
||||
if (responseProto != null) {
|
||||
metrics.incContainerOpsLatencies(cmdType, System.nanoTime() - startTime);
|
||||
return responseProto;
|
||||
} else {
|
||||
return ContainerUtils.unsupportedRequest(msg);
|
||||
}
|
||||
return responseProto;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -36,7 +36,7 @@
|
||||
* Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
|
||||
* should have an implementation for Handler.
|
||||
*/
|
||||
public class Handler {
|
||||
public abstract class Handler {
|
||||
|
||||
protected final Configuration conf;
|
||||
protected final ContainerSet containerSet;
|
||||
@ -64,10 +64,8 @@ public static Handler getHandlerForContainerType(ContainerType containerType,
|
||||
}
|
||||
}
|
||||
|
||||
public ContainerCommandResponseProto handle(
|
||||
ContainerCommandRequestProto msg, Container container) {
|
||||
return null;
|
||||
}
|
||||
public abstract ContainerCommandResponseProto handle(
|
||||
ContainerCommandRequestProto msg, Container container);
|
||||
|
||||
public void setScmID(String scmId) {
|
||||
this.scmID = scmId;
|
||||
|
@ -62,6 +62,7 @@
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
|
||||
import org.apache.hadoop.util.AutoCloseableLock;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -70,6 +71,7 @@
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.Result.CONTAINER_INTERNAL_ERROR;
|
||||
@ -102,6 +104,7 @@ public class KeyValueHandler extends Handler {
|
||||
private final ChunkManager chunkManager;
|
||||
private VolumeChoosingPolicy volumeChoosingPolicy;
|
||||
private final int maxContainerSizeGB;
|
||||
private final AutoCloseableLock handlerLock;
|
||||
|
||||
|
||||
public KeyValueHandler(Configuration config, ContainerSet contSet,
|
||||
@ -115,6 +118,9 @@ public KeyValueHandler(Configuration config, ContainerSet contSet,
|
||||
maxContainerSizeGB = config.getInt(ScmConfigKeys
|
||||
.OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
|
||||
.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
|
||||
// this handler lock is used for synchronizing createContainer Requests,
|
||||
// so using a fair lock here.
|
||||
handlerLock = new AutoCloseableLock(new ReentrantLock(true));
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -159,7 +165,6 @@ public ContainerCommandResponseProto handle(
|
||||
case GetSmallFile:
|
||||
return handleGetSmallFile(request, kvContainer);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@ -204,10 +209,19 @@ ContainerCommandResponseProto handleCreateContainer(
|
||||
newContainerData, conf);
|
||||
|
||||
try {
|
||||
newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
|
||||
containerSet.addContainer(newContainer);
|
||||
handlerLock.acquire();
|
||||
if (containerSet.getContainer(containerID) == null) {
|
||||
newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
|
||||
containerSet.addContainer(newContainer);
|
||||
} else {
|
||||
throw new StorageContainerException("Container already exists with " +
|
||||
"container Id " + containerID, ContainerProtos.Result
|
||||
.CONTAINER_EXISTS);
|
||||
}
|
||||
} catch (StorageContainerException ex) {
|
||||
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
||||
} finally {
|
||||
handlerLock.release();
|
||||
}
|
||||
|
||||
return ContainerUtils.getSuccessResponse(request);
|
||||
|
@ -28,7 +28,6 @@
|
||||
import org.apache.commons.lang3.RandomUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.client.BlockID;
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
|
||||
import org.apache.hadoop.util.Time;
|
||||
@ -42,13 +41,11 @@
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
|
||||
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
@ -106,10 +103,6 @@ public void initialize() throws IOException {
|
||||
// data directory
|
||||
conf.set("dfs.datanode.data.dir", baseDir + File.separator + "data");
|
||||
|
||||
// metadata directory
|
||||
StorageLocation metadataDir = StorageLocation.parse(
|
||||
baseDir + File.separator + CONTAINER_ROOT_PREFIX);
|
||||
|
||||
ContainerSet containerSet = new ContainerSet();
|
||||
VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user