HDDS-211. Add a create container Lock. Contributed by Bharat Viswanadham
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e899c4cc Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e899c4cc Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e899c4cc Branch: refs/heads/trunk Commit: e899c4cc01df3b657893314adcdbb411dfbf3764 Parents: 7dcf587 Author: Bharat Viswanadham <bha...@apache.org> Authored: Fri Jul 6 15:02:40 2018 -0700 Committer: Bharat Viswanadham <bha...@apache.org> Committed: Fri Jul 6 15:02:40 2018 -0700 ---------------------------------------------------------------------- .../container/common/impl/HddsDispatcher.java | 4 +++- .../container/common/interfaces/Handler.java | 8 +++----- .../container/keyvalue/KeyValueHandler.java | 20 +++++++++++++++++--- .../genesis/BenchMarkDatanodeDispatcher.java | 7 ------- 4 files changed, 23 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java ---------------------------------------------------------------------- diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java index 25700f9..f0c2aa9 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java @@ -119,8 +119,10 @@ public class HddsDispatcher implements ContainerDispatcher { responseProto = handler.handle(msg, container); if (responseProto != null) { metrics.incContainerOpsLatencies(cmdType, System.nanoTime() - startTime); + return responseProto; + } else { + return ContainerUtils.unsupportedRequest(msg); } - return responseProto; } @Override http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java ---------------------------------------------------------------------- diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java index 2725f59..15eed4f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java @@ -36,7 +36,7 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; * 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 class Handler { } } - 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; http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java ---------------------------------------------------------------------- diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index 110d0b4..3806ed6 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -62,6 +62,7 @@ import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl; 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.HashMap; 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 class KeyValueHandler extends Handler { 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 class KeyValueHandler extends Handler { case GetSmallFile: return handleGetSmallFile(request, kvContainer); } - return null; } @@ -204,10 +209,19 @@ public class KeyValueHandler extends Handler { 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); http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java ---------------------------------------------------------------------- diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java index 331e3ed..93e7ef1 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java @@ -28,7 +28,6 @@ import org.apache.commons.lang3.RandomStringUtils; 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 org.openjdk.jmh.annotations.TearDown; 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 class BenchMarkDatanodeDispatcher { // 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); --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org