diff options
author | Anu Engineer <aengineer@apache.org> | 2018-05-07 14:42:18 -0700 |
---|---|---|
committer | Anu Engineer <aengineer@apache.org> | 2018-05-07 14:58:52 -0700 |
commit | 3a43ac2851f5dea4deb8a1dfebf9bf65fc57bd76 (patch) | |
tree | 90f57883dda458ccd87c2c9e1f9c8530aa6c5316 | |
parent | a3a1552c33d5650fbd0a702369fccd21b8c9d3e2 (diff) |
HDDS-1. Remove SCM Block DB. Contributed by Xiaoyu Yao.
127 files changed, 2059 insertions, 2402 deletions
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java index db5a56730d..42005f0b09 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java @@ -18,6 +18,8 @@ package org.apache.hadoop.util; import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.TimeZone; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -34,6 +36,8 @@ public final class Time { */ private static final long NANOSECONDS_PER_MILLISECOND = 1000000; + private static final TimeZone UTC_ZONE = TimeZone.getTimeZone("UTC"); + private static final ThreadLocal<SimpleDateFormat> DATE_FORMAT = new ThreadLocal<SimpleDateFormat>() { @Override @@ -82,4 +86,12 @@ public final class Time { public static String formatTime(long millis) { return DATE_FORMAT.get().format(millis); } + + /** + * Get the current UTC time in milliseconds. + * @return the current UTC time in milliseconds. + */ + public static long getUtcTime() { + return Calendar.getInstance(UTC_ZONE).getTimeInMillis(); + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java index 75851042c2..dcaa57621d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java @@ -60,7 +60,7 @@ public class XceiverClientManager implements Closeable { //TODO : change this to SCM configuration class private final Configuration conf; - private final Cache<String, XceiverClientSpi> clientCache; + private final Cache<Long, XceiverClientSpi> clientCache; private final boolean useRatis; private static XceiverClientMetrics metrics; @@ -84,10 +84,10 @@ public class XceiverClientManager implements Closeable { .expireAfterAccess(staleThresholdMs, TimeUnit.MILLISECONDS) .maximumSize(maxSize) .removalListener( - new RemovalListener<String, XceiverClientSpi>() { + new RemovalListener<Long, XceiverClientSpi>() { @Override public void onRemoval( - RemovalNotification<String, XceiverClientSpi> + RemovalNotification<Long, XceiverClientSpi> removalNotification) { synchronized (clientCache) { // Mark the entry as evicted @@ -99,7 +99,7 @@ public class XceiverClientManager implements Closeable { } @VisibleForTesting - public Cache<String, XceiverClientSpi> getClientCache() { + public Cache<Long, XceiverClientSpi> getClientCache() { return clientCache; } @@ -114,14 +114,14 @@ public class XceiverClientManager implements Closeable { * @return XceiverClientSpi connected to a container * @throws IOException if a XceiverClientSpi cannot be acquired */ - public XceiverClientSpi acquireClient(Pipeline pipeline) + public XceiverClientSpi acquireClient(Pipeline pipeline, long containerID) throws IOException { Preconditions.checkNotNull(pipeline); Preconditions.checkArgument(pipeline.getMachines() != null); Preconditions.checkArgument(!pipeline.getMachines().isEmpty()); synchronized (clientCache) { - XceiverClientSpi info = getClient(pipeline); + XceiverClientSpi info = getClient(pipeline, containerID); info.incrementReference(); return info; } @@ -139,11 +139,10 @@ public class XceiverClientManager implements Closeable { } } - private XceiverClientSpi getClient(Pipeline pipeline) + private XceiverClientSpi getClient(Pipeline pipeline, long containerID) throws IOException { - String containerName = pipeline.getContainerName(); try { - return clientCache.get(containerName, + return clientCache.get(containerID, new Callable<XceiverClientSpi>() { @Override public XceiverClientSpi call() throws Exception { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java index 8f30a7fad1..15d197c405 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java @@ -86,15 +86,16 @@ public class ContainerOperationClient implements ScmClient { * @inheritDoc */ @Override - public Pipeline createContainer(String containerId, String owner) + public ContainerInfo createContainer(String owner) throws IOException { XceiverClientSpi client = null; try { - Pipeline pipeline = + ContainerInfo container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerId, owner); - client = xceiverClientManager.acquireClient(pipeline); + xceiverClientManager.getFactor(), owner); + Pipeline pipeline = container.getPipeline(); + client = xceiverClientManager.acquireClient(pipeline, container.getContainerID()); // Allocated State means that SCM has allocated this pipeline in its // namespace. The client needs to create the pipeline on the machines @@ -104,10 +105,8 @@ public class ContainerOperationClient implements ScmClient { if (pipeline.getLifeCycleState() == ALLOCATED) { createPipeline(client, pipeline); } - // TODO : Container Client State needs to be updated. - // TODO : Return ContainerInfo instead of Pipeline - createContainer(containerId, client, pipeline); - return pipeline; + createContainer(client, container.getContainerID()); + return container; } finally { if (client != null) { xceiverClientManager.releaseClient(client); @@ -118,20 +117,19 @@ public class ContainerOperationClient implements ScmClient { /** * Create a container over pipeline specified by the SCM. * - * @param containerId - Container ID - * @param client - Client to communicate with Datanodes - * @param pipeline - A pipeline that is already created. + * @param client - Client to communicate with Datanodes. + * @param containerId - Container ID. * @throws IOException */ - public void createContainer(String containerId, XceiverClientSpi client, - Pipeline pipeline) throws IOException { + public void createContainer(XceiverClientSpi client, + long containerId) throws IOException { String traceID = UUID.randomUUID().toString(); storageContainerLocationClient.notifyObjectStageChange( ObjectStageChangeRequestProto.Type.container, containerId, ObjectStageChangeRequestProto.Op.create, ObjectStageChangeRequestProto.Stage.begin); - ContainerProtocolCalls.createContainer(client, traceID); + ContainerProtocolCalls.createContainer(client, containerId, traceID); storageContainerLocationClient.notifyObjectStageChange( ObjectStageChangeRequestProto.Type.container, containerId, @@ -142,8 +140,8 @@ public class ContainerOperationClient implements ScmClient { // creation state. if (LOG.isDebugEnabled()) { LOG.debug("Created container " + containerId - + " leader:" + pipeline.getLeader() - + " machines:" + pipeline.getMachines()); + + " leader:" + client.getPipeline().getLeader() + + " machines:" + client.getPipeline().getMachines()); } } @@ -168,20 +166,25 @@ public class ContainerOperationClient implements ScmClient { // 2. Talk to Datanodes to create the pipeline. // // 3. update SCM that pipeline creation was successful. - storageContainerLocationClient.notifyObjectStageChange( - ObjectStageChangeRequestProto.Type.pipeline, - pipeline.getPipelineName(), - ObjectStageChangeRequestProto.Op.create, - ObjectStageChangeRequestProto.Stage.begin); + + // TODO: this has not been fully implemented on server side + // SCMClientProtocolServer#notifyObjectStageChange + // TODO: when implement the pipeline state machine, change + // the pipeline name (string) to pipeline id (long) + //storageContainerLocationClient.notifyObjectStageChange( + // ObjectStageChangeRequestProto.Type.pipeline, + // pipeline.getPipelineName(), + // ObjectStageChangeRequestProto.Op.create, + // ObjectStageChangeRequestProto.Stage.begin); client.createPipeline(pipeline.getPipelineName(), pipeline.getMachines()); - storageContainerLocationClient.notifyObjectStageChange( - ObjectStageChangeRequestProto.Type.pipeline, - pipeline.getPipelineName(), - ObjectStageChangeRequestProto.Op.create, - ObjectStageChangeRequestProto.Stage.complete); + //storageContainerLocationClient.notifyObjectStageChange( + // ObjectStageChangeRequestProto.Type.pipeline, + // pipeline.getPipelineName(), + // ObjectStageChangeRequestProto.Op.create, + // ObjectStageChangeRequestProto.Stage.complete); // TODO : Should we change the state on the client side ?? // That makes sense, but it is not needed for the client to work. @@ -193,16 +196,17 @@ public class ContainerOperationClient implements ScmClient { * @inheritDoc */ @Override - public Pipeline createContainer(HddsProtos.ReplicationType type, - HddsProtos.ReplicationFactor factor, - String containerId, String owner) throws IOException { + public ContainerInfo createContainer(HddsProtos.ReplicationType type, + HddsProtos.ReplicationFactor factor, String owner) throws IOException { XceiverClientSpi client = null; try { // allocate container on SCM. - Pipeline pipeline = + ContainerInfo container = storageContainerLocationClient.allocateContainer(type, factor, - containerId, owner); - client = xceiverClientManager.acquireClient(pipeline); + owner); + Pipeline pipeline = container.getPipeline(); + client = xceiverClientManager.acquireClient(pipeline, + container.getContainerID()); // Allocated State means that SCM has allocated this pipeline in its // namespace. The client needs to create the pipeline on the machines @@ -210,12 +214,11 @@ public class ContainerOperationClient implements ScmClient { if (pipeline.getLifeCycleState() == ALLOCATED) { createPipeline(client, pipeline); } - - // TODO : Return ContainerInfo instead of Pipeline // connect to pipeline leader and allocate container on leader datanode. - client = xceiverClientManager.acquireClient(pipeline); - createContainer(containerId, client, pipeline); - return pipeline; + client = xceiverClientManager.acquireClient(pipeline, + container.getContainerID()); + createContainer(client, container.getContainerID()); + return container; } finally { if (client != null) { xceiverClientManager.releaseClient(client); @@ -258,18 +261,18 @@ public class ContainerOperationClient implements ScmClient { * @throws IOException */ @Override - public void deleteContainer(Pipeline pipeline, boolean force) + public void deleteContainer(long containerID, Pipeline pipeline, boolean force) throws IOException { XceiverClientSpi client = null; try { - client = xceiverClientManager.acquireClient(pipeline); + client = xceiverClientManager.acquireClient(pipeline, containerID); String traceID = UUID.randomUUID().toString(); - ContainerProtocolCalls.deleteContainer(client, force, traceID); + ContainerProtocolCalls.deleteContainer(client, containerID, force, traceID); storageContainerLocationClient - .deleteContainer(pipeline.getContainerName()); + .deleteContainer(containerID); if (LOG.isDebugEnabled()) { LOG.debug("Deleted container {}, leader: {}, machines: {} ", - pipeline.getContainerName(), + containerID, pipeline.getLeader(), pipeline.getMachines()); } @@ -284,11 +287,10 @@ public class ContainerOperationClient implements ScmClient { * {@inheritDoc} */ @Override - public List<ContainerInfo> listContainer(String startName, - String prefixName, int count) - throws IOException { + public List<ContainerInfo> listContainer(long startContainerID, + int count) throws IOException { return storageContainerLocationClient.listContainer( - startName, prefixName, count); + startContainerID, count); } /** @@ -300,17 +302,17 @@ public class ContainerOperationClient implements ScmClient { * @throws IOException */ @Override - public ContainerData readContainer(Pipeline pipeline) throws IOException { + public ContainerData readContainer(long containerID, + Pipeline pipeline) throws IOException { XceiverClientSpi client = null; try { - client = xceiverClientManager.acquireClient(pipeline); + client = xceiverClientManager.acquireClient(pipeline, containerID); String traceID = UUID.randomUUID().toString(); ReadContainerResponseProto response = - ContainerProtocolCalls.readContainer(client, - pipeline.getContainerName(), traceID); + ContainerProtocolCalls.readContainer(client, containerID, traceID); if (LOG.isDebugEnabled()) { LOG.debug("Read container {}, leader: {}, machines: {} ", - pipeline.getContainerName(), + containerID, pipeline.getLeader(), pipeline.getMachines()); } @@ -329,7 +331,7 @@ public class ContainerOperationClient implements ScmClient { * @throws IOException */ @Override - public Pipeline getContainer(String containerId) throws + public ContainerInfo getContainer(long containerId) throws IOException { return storageContainerLocationClient.getContainer(containerId); } @@ -341,7 +343,8 @@ public class ContainerOperationClient implements ScmClient { * @throws IOException */ @Override - public void closeContainer(Pipeline pipeline) throws IOException { + public void closeContainer(long containerId, Pipeline pipeline) + throws IOException { XceiverClientSpi client = null; try { LOG.debug("Close container {}", pipeline); @@ -364,18 +367,16 @@ public class ContainerOperationClient implements ScmClient { For now, take the #2 way. */ // Actually close the container on Datanode - client = xceiverClientManager.acquireClient(pipeline); + client = xceiverClientManager.acquireClient(pipeline, containerId); String traceID = UUID.randomUUID().toString(); - String containerId = pipeline.getContainerName(); - storageContainerLocationClient.notifyObjectStageChange( ObjectStageChangeRequestProto.Type.container, containerId, ObjectStageChangeRequestProto.Op.close, ObjectStageChangeRequestProto.Stage.begin); - ContainerProtocolCalls.closeContainer(client, traceID); + ContainerProtocolCalls.closeContainer(client, containerId, traceID); // Notify SCM to close the container storageContainerLocationClient.notifyObjectStageChange( ObjectStageChangeRequestProto.Type.container, @@ -391,13 +392,13 @@ public class ContainerOperationClient implements ScmClient { /** * Get the the current usage information. - * @param pipeline - Pipeline + * @param containerID - ID of the container. * @return the size of the given container. * @throws IOException */ @Override - public long getContainerSize(Pipeline pipeline) throws IOException { - // TODO : Pipeline can be null, handle it correctly. + public long getContainerSize(long containerID) throws IOException { + // TODO : Fix this, it currently returns the capacity but not the current usage. long size = getContainerSizeB(); if (size == -1) { throw new IOException("Container size unknown!"); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index 9b8eaa9661..c4c336221c 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos .ReadChunkResponseProto; +import org.apache.hadoop.hdds.client.BlockID; import java.io.EOFException; import java.io.IOException; @@ -45,7 +46,7 @@ public class ChunkInputStream extends InputStream implements Seekable { private static final int EOF = -1; - private final String key; + private final BlockID blockID; private final String traceID; private XceiverClientManager xceiverClientManager; private XceiverClientSpi xceiverClient; @@ -58,15 +59,15 @@ public class ChunkInputStream extends InputStream implements Seekable { /** * Creates a new ChunkInputStream. * - * @param key chunk key + * @param blockID block ID of the chunk * @param xceiverClientManager client manager that controls client * @param xceiverClient client to perform container calls * @param chunks list of chunks to read * @param traceID container protocol call traceID */ - public ChunkInputStream(String key, XceiverClientManager xceiverClientManager, + public ChunkInputStream(BlockID blockID, XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient, List<ChunkInfo> chunks, String traceID) { - this.key = key; + this.blockID = blockID; this.traceID = traceID; this.xceiverClientManager = xceiverClientManager; this.xceiverClient = xceiverClient; @@ -196,7 +197,7 @@ public class ChunkInputStream extends InputStream implements Seekable { final ReadChunkResponseProto readChunkResponse; try { readChunkResponse = ContainerProtocolCalls.readChunk(xceiverClient, - chunks.get(chunkIndex), key, traceID); + chunks.get(chunkIndex), blockID, traceID); } catch (IOException e) { throw new IOException("Unexpected OzoneException: " + e.toString(), e); } @@ -211,7 +212,7 @@ public class ChunkInputStream extends InputStream implements Seekable { || pos >= chunkOffset[chunks.size() - 1] + chunks.get(chunks.size() - 1) .getLen()) { throw new EOFException( - "EOF encountered pos: " + pos + " container key: " + key); + "EOF encountered pos: " + pos + " container key: " + blockID.getLocalID()); } if (chunkIndex == -1) { chunkIndex = Arrays.binarySearch(chunkOffset, pos); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java index b65df9f89b..325f110512 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.KeyData; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue; +import org.apache.hadoop.hdds.client.BlockID; import java.io.IOException; import java.io.OutputStream; @@ -53,7 +54,7 @@ import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls */ public class ChunkOutputStream extends OutputStream { - private final String containerKey; + private final BlockID blockID; private final String key; private final String traceID; private final KeyData.Builder containerKeyData; @@ -67,25 +68,24 @@ public class ChunkOutputStream extends OutputStream { /** * Creates a new ChunkOutputStream. * - * @param containerKey container key + * @param blockID block ID * @param key chunk key * @param xceiverClientManager client manager that controls client * @param xceiverClient client to perform container calls * @param traceID container protocol call args * @param chunkSize chunk size */ - public ChunkOutputStream(String containerKey, String key, - XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient, - String traceID, int chunkSize) { - this.containerKey = containerKey; + public ChunkOutputStream(BlockID blockID, String key, + XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient, + String traceID, int chunkSize) { + this.blockID = blockID; this.key = key; this.traceID = traceID; this.chunkSize = chunkSize; KeyValue keyValue = KeyValue.newBuilder() .setKey("TYPE").setValue("KEY").build(); this.containerKeyData = KeyData.newBuilder() - .setContainerName(xceiverClient.getPipeline().getContainerName()) - .setName(containerKey) + .setBlockID(blockID.getProtobuf()) .addMetadata(keyValue); this.xceiverClientManager = xceiverClientManager; this.xceiverClient = xceiverClient; @@ -217,7 +217,7 @@ public class ChunkOutputStream extends OutputStream { .setLen(data.size()) .build(); try { - writeChunk(xceiverClient, chunk, key, data, traceID); + writeChunk(xceiverClient, chunk, blockID, data, traceID); } catch (IOException e) { throw new IOException( "Unexpected Storage Container Exception: " + e.toString(), e); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java new file mode 100644 index 0000000000..7236af7e30 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -0,0 +1,59 @@ +/** + * 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 + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * <p> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p> + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.hadoop.hdds.client; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; + +/** + * BlockID of ozone (containerID + localID) + */ +public class BlockID { + private long containerID; + private long localID; + + public BlockID(long containerID, long localID) { + this.containerID = containerID; + this.localID = localID; + } + + public long getContainerID() { + return containerID; + } + + public long getLocalID() { + return localID; + } + + @Override + public String toString() { + return new ToStringBuilder(this). + append("containerID", containerID). + append("localID", localID). + toString(); + } + + public HddsProtos.BlockID getProtobuf() { + return HddsProtos.BlockID.newBuilder(). + setContainerID(containerID).setLocalID(localID).build(); + } + + public static BlockID getFromProtobuf(HddsProtos.BlockID blockID) { + return new BlockID(blockID.getContainerID(), + blockID.getLocalID()); + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java index 0d4a2990b6..dcf9fed800 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java @@ -41,78 +41,76 @@ import java.util.List; public interface ScmClient { /** * Creates a Container on SCM and returns the pipeline. - * @param containerId - String container ID - * @return Pipeline + * @return ContainerInfo * @throws IOException */ - Pipeline createContainer(String containerId, String owner) throws IOException; + ContainerInfo createContainer(String owner) throws IOException; /** * Gets a container by Name -- Throws if the container does not exist. - * @param containerId - String Container ID + * @param containerId - Container ID * @return Pipeline * @throws IOException */ - Pipeline getContainer(String containerId) throws IOException; + ContainerInfo getContainer(long containerId) throws IOException; /** - * Close a container by name. + * Close a container. * - * @param pipeline the container to be closed. + * @param containerId - ID of the container. + * @param pipeline - Pipeline where the container is located. * @throws IOException */ - void closeContainer(Pipeline pipeline) throws IOException; + void closeContainer(long containerId, Pipeline pipeline) throws IOException; /** * Deletes an existing container. + * @param containerId - ID of the container. * @param pipeline - Pipeline that represents the container. * @param force - true to forcibly delete the container. * @throws IOException */ - void deleteContainer(Pipeline pipeline, boolean force) throws IOException; + void deleteContainer(long containerId, Pipeline pipeline, boolean force) throws IOException; /** * Lists a range of containers and get their info. * - * @param startName start name, if null, start searching at the head. - * @param prefixName prefix name, if null, then filter is disabled. - * @param count count, if count < 0, the max size is unlimited.( - * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) + * @param startContainerID start containerID. + * @param count count must be > 0. * * @return a list of pipeline. * @throws IOException */ - List<ContainerInfo> listContainer(String startName, String prefixName, + List<ContainerInfo> listContainer(long startContainerID, int count) throws IOException; /** * Read meta data from an existing container. - * @param pipeline - Pipeline that represents the container. + * @param containerID - ID of the container. + * @param pipeline - Pipeline where the container is located. * @return ContainerInfo * @throws IOException */ - ContainerData readContainer(Pipeline pipeline) throws IOException; - + ContainerData readContainer(long containerID, Pipeline pipeline) + throws IOException; /** * Gets the container size -- Computed by SCM from Container Reports. - * @param pipeline - Pipeline + * @param containerID - ID of the container. * @return number of bytes used by this container. * @throws IOException */ - long getContainerSize(Pipeline pipeline) throws IOException; + long getContainerSize(long containerID) throws IOException; /** * Creates a Container on SCM and returns the pipeline. * @param type - Replication Type. * @param replicationFactor - Replication Factor - * @param containerId - Container ID - * @return Pipeline + * @return ContainerInfo * @throws IOException - in case of error. */ - Pipeline createContainer(HddsProtos.ReplicationType type, - HddsProtos.ReplicationFactor replicationFactor, String containerId, + ContainerInfo createContainer(HddsProtos.ReplicationType type, + HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException; /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java index d253b15cd2..9b8946978c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java @@ -18,13 +18,15 @@ package org.apache.hadoop.hdds.scm.container.common.helpers; +import org.apache.hadoop.hdds.client.BlockID; + /** * Allocated block wraps the result returned from SCM#allocateBlock which * contains a Pipeline and the key. */ public final class AllocatedBlock { private Pipeline pipeline; - private String key; + private BlockID blockID; // Indicates whether the client should create container before writing block. private boolean shouldCreateContainer; @@ -33,7 +35,7 @@ public final class AllocatedBlock { */ public static class Builder { private Pipeline pipeline; - private String key; + private BlockID blockID; private boolean shouldCreateContainer; public Builder setPipeline(Pipeline p) { @@ -41,8 +43,8 @@ public final class AllocatedBlock { return this; } - public Builder setKey(String k) { - this.key = k; + public Builder setBlockID(BlockID blockID) { + this.blockID = blockID; return this; } @@ -52,14 +54,14 @@ public final class AllocatedBlock { } public AllocatedBlock build() { - return new AllocatedBlock(pipeline, key, shouldCreateContainer); + return new AllocatedBlock(pipeline, blockID, shouldCreateContainer); } } - private AllocatedBlock(Pipeline pipeline, String key, + private AllocatedBlock(Pipeline pipeline, BlockID blockID, boolean shouldCreateContainer) { this.pipeline = pipeline; - this.key = key; + this.blockID = blockID; this.shouldCreateContainer = shouldCreateContainer; } @@ -67,8 +69,8 @@ public final class AllocatedBlock { return pipeline; } - public String getKey() { - return key; + public BlockID getBlockID() { + return blockID; } public boolean getCreateContainer() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java index 823a7fbc05..0bd4c26d42 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java @@ -43,11 +43,9 @@ public class ContainerInfo // The wall-clock ms since the epoch at which the current state enters. private long stateEnterTime; private String owner; - private String containerName; private long containerID; ContainerInfo( long containerID, - final String containerName, HddsProtos.LifeCycleState state, Pipeline pipeline, long allocatedBytes, @@ -56,7 +54,6 @@ public class ContainerInfo long stateEnterTime, String owner) { this.containerID = containerID; - this.containerName = containerName; this.pipeline = pipeline; this.allocatedBytes = allocatedBytes; this.usedBytes = usedBytes; @@ -82,7 +79,6 @@ public class ContainerInfo builder.setState(info.getState()); builder.setStateEnterTime(info.getStateEnterTime()); builder.setOwner(info.getOwner()); - builder.setContainerName(info.getContainerName()); builder.setContainerID(info.getContainerID()); return builder.build(); } @@ -91,10 +87,6 @@ public class ContainerInfo return containerID; } - public String getContainerName() { - return containerName; - } - public HddsProtos.LifeCycleState getState() { return state; } @@ -170,7 +162,6 @@ public class ContainerInfo if (getOwner() != null) { builder.setOwner(getOwner()); } - builder.setContainerName(getContainerName()); return builder.build(); } @@ -189,7 +180,6 @@ public class ContainerInfo + ", pipeline=" + pipeline + ", stateEnterTime=" + stateEnterTime + ", owner=" + owner - + ", containerName='" + containerName + '}'; } @@ -206,7 +196,7 @@ public class ContainerInfo ContainerInfo that = (ContainerInfo) o; return new EqualsBuilder() - .append(pipeline.getContainerName(), that.pipeline.getContainerName()) + .append(getContainerID(), that.getContainerID()) // TODO : Fix this later. If we add these factors some tests fail. // So Commenting this to continue and will enforce this with @@ -221,7 +211,7 @@ public class ContainerInfo @Override public int hashCode() { return new HashCodeBuilder(11, 811) - .append(pipeline.getContainerName()) + .append(getContainerID()) .append(pipeline.getFactor()) .append(pipeline.getType()) .append(owner) @@ -275,7 +265,6 @@ public class ContainerInfo private long keys; private long stateEnterTime; private String owner; - private String containerName; private long containerID; public Builder setContainerID(long id) { @@ -319,14 +308,9 @@ public class ContainerInfo return this; } - public Builder setContainerName(String container) { - this.containerName = container; - return this; - } - public ContainerInfo build() { return new - ContainerInfo(containerID, containerName, state, pipeline, + ContainerInfo(containerID, state, pipeline, allocated, used, keys, stateEnterTime, owner); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/DeleteBlockResult.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/DeleteBlockResult.java index fd97eae3b7..5f5aaceb16 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/DeleteBlockResult.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/DeleteBlockResult.java @@ -17,6 +17,8 @@ package org.apache.hadoop.hdds.scm.container.common.helpers; +import org.apache.hadoop.hdds.client.BlockID; + import static org.apache.hadoop.hdds.protocol.proto .ScmBlockLocationProtocolProtos.DeleteScmBlockResult; @@ -24,21 +26,21 @@ import static org.apache.hadoop.hdds.protocol.proto * Class wraps storage container manager block deletion results. */ public class DeleteBlockResult { - private String key; + private BlockID blockID; private DeleteScmBlockResult.Result result; - public DeleteBlockResult(final String key, + public DeleteBlockResult(final BlockID blockID, final DeleteScmBlockResult.Result result) { - this.key = key; + this.blockID = blockID; this.result = result; } /** - * Get key deleted. - * @return key name. + * Get block id deleted. + * @return block id. */ - public String getKey() { - return key; + public BlockID getBlockID() { + return blockID; } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java index 32d0a2d85a..87408385ec 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java @@ -57,7 +57,6 @@ public class Pipeline { WRITER = mapper.writer(filters); } - private String containerName; private PipelineChannel pipelineChannel; /** * Allows you to maintain private data on pipelines. This is not serialized @@ -68,11 +67,9 @@ public class Pipeline { /** * Constructs a new pipeline data structure. * - * @param containerName - Container * @param pipelineChannel - transport information for this container */ - public Pipeline(String containerName, PipelineChannel pipelineChannel) { - this.containerName = containerName; + public Pipeline(PipelineChannel pipelineChannel) { this.pipelineChannel = pipelineChannel; data = null; } @@ -87,7 +84,7 @@ public class Pipeline { Preconditions.checkNotNull(pipeline); PipelineChannel pipelineChannel = PipelineChannel.getFromProtoBuf(pipeline.getPipelineChannel()); - return new Pipeline(pipeline.getContainerName(), pipelineChannel); + return new Pipeline(pipelineChannel); } public HddsProtos.ReplicationFactor getFactor() { @@ -146,21 +143,11 @@ public class Pipeline { public HddsProtos.Pipeline getProtobufMessage() { HddsProtos.Pipeline.Builder builder = HddsProtos.Pipeline.newBuilder(); - builder.setContainerName(this.containerName); builder.setPipelineChannel(this.pipelineChannel.getProtobufMessage()); return builder.build(); } /** - * Returns containerName if available. - * - * @return String. - */ - public String getContainerName() { - return containerName; - } - - /** * Returns private data that is set on this pipeline. * * @return blob, the user can interpret it any way they like. @@ -223,7 +210,6 @@ public class Pipeline { pipelineChannel.getDatanodes().keySet().stream() .forEach(id -> b. append(id.endsWith(pipelineChannel.getLeaderID()) ? "*" + id : id)); - b.append("] container:").append(containerName); b.append(" name:").append(getPipelineName()); if (getType() != null) { b.append(" type:").append(getType().toString()); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/PipelineChannel.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/PipelineChannel.java index ebd52e9984..655751d737 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/PipelineChannel.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/PipelineChannel.java @@ -40,6 +40,8 @@ public class PipelineChannel { private ReplicationType type; private ReplicationFactor factor; private String name; + // TODO: change to long based id + //private long id; public PipelineChannel(String leaderID, LifeCycleState lifeCycleState, ReplicationType replicationType, ReplicationFactor replicationFactor, diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java index f100fc702c..c8d4a809fa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java @@ -26,7 +26,6 @@ import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; import java.io.IOException; import java.util.List; -import java.util.Set; /** * ScmBlockLocationProtocol is used by an HDFS node to find the set of nodes @@ -35,17 +34,6 @@ import java.util.Set; public interface ScmBlockLocationProtocol { /** - * Find the set of nodes to read/write a block, as - * identified by the block key. This method supports batch lookup by - * passing multiple keys. - * - * @param keys batch of block keys to find - * @return allocated blocks for each block key - * @throws IOException if there is any failure - */ - Set<AllocatedBlock> getBlockLocations(Set<String> keys) throws IOException; - - /** * Asks SCM where a block should be allocated. SCM responds with the * set of datanodes that should be used creating this block. * @param size - size of the block. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java index a60fbb2f22..e8d85e0084 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java @@ -38,19 +38,20 @@ public interface StorageContainerLocationProtocol { * set of datanodes that should be used creating this container. * */ - Pipeline allocateContainer(HddsProtos.ReplicationType replicationType, - HddsProtos.ReplicationFactor factor, String containerName, String owner) + ContainerInfo allocateContainer(HddsProtos.ReplicationType replicationType, + HddsProtos.ReplicationFactor factor, String owner) throws IOException; /** * Ask SCM the location of the container. SCM responds with a group of * nodes where this container and its replicas are located. * - * @param containerName - Name of the container. - * @return Pipeline - the pipeline where container locates. + * @param containerID - ID of the container. + * @return ContainerInfo - the container info such as where the pipeline + * is located. * @throws IOException */ - Pipeline getContainer(String containerName) throws IOException; + ContainerInfo getContainer(long containerID) throws IOException; /** * Ask SCM a list of containers with a range of container names @@ -59,8 +60,7 @@ public interface StorageContainerLocationProtocol { * use prefix name to filter the result. the max size of the * searching range cannot exceed the value of count. * - * @param startName start name, if null, start searching at the head. - * @param prefixName prefix name, if null, then filter is disabled. + * @param startContainerID start container ID. * @param count count, if count < 0, the max size is unlimited.( * Usually the count will be replace with a very big * value instead of being unlimited in case the db is very big) @@ -68,18 +68,18 @@ public interface StorageContainerLocationProtocol { * @return a list of container. * @throws IOException */ - List<ContainerInfo> listContainer(String startName, String prefixName, - int count) throws IOException; + List<ContainerInfo> listContainer(long startContainerID, int count) + throws IOException; /** * Deletes a container in SCM. * - * @param containerName + * @param containerID * @throws IOException * if failed to delete the container mapping from db store * or container doesn't exist. */ - void deleteContainer(String containerName) throws IOException; + void deleteContainer(long containerID) throws IOException; /** * Queries a list of Node Statuses. @@ -94,12 +94,12 @@ public interface StorageContainerLocationProtocol { * or containers on datanodes. * Container will be in Operational state after that. * @param type object type - * @param name object name + * @param id object id * @param op operation type (e.g., create, close, delete) * @param stage creation stage */ void notifyObjectStageChange( - ObjectStageChangeRequestProto.Type type, String name, + ObjectStageChangeRequestProto.Type type, long id, ObjectStageChangeRequestProto.Op op, ObjectStageChangeRequestProto.Stage stage) throws IOException; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java index 0012f3e4a8..aed0fb7c22 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java @@ -17,10 +17,10 @@ package org.apache.hadoop.hdds.scm.protocolPB; import com.google.common.base.Preconditions; -import com.google.common.collect.Sets; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; @@ -35,13 +35,7 @@ import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos .DeleteScmKeyBlocksResponseProto; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos - .GetScmBlockLocationsRequestProto; -import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos - .GetScmBlockLocationsResponseProto; -import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos .KeyBlocks; -import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos - .ScmLocatedBlockProto; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolTranslator; import org.apache.hadoop.ipc.RPC; @@ -52,7 +46,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; /** @@ -82,41 +75,6 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB } /** - * Find the set of nodes to read/write a block, as - * identified by the block key. This method supports batch lookup by - * passing multiple keys. - * - * @param keys batch of block keys to find - * @return allocated blocks for each block key - * @throws IOException if there is any failure - */ - @Override - public Set<AllocatedBlock> getBlockLocations(Set<String> keys) - throws IOException { - GetScmBlockLocationsRequestProto.Builder req = - GetScmBlockLocationsRequestProto.newBuilder(); - for (String key : keys) { - req.addKeys(key); - } - final GetScmBlockLocationsResponseProto resp; - try { - resp = rpcProxy.getScmBlockLocations(NULL_RPC_CONTROLLER, - req.build()); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } - Set<AllocatedBlock> locatedBlocks = - Sets.newLinkedHashSetWithExpectedSize(resp.getLocatedBlocksCount()); - for (ScmLocatedBlockProto locatedBlock : resp.getLocatedBlocksList()) { - locatedBlocks.add(new AllocatedBlock.Builder() - .setKey(locatedBlock.getKey()) - .setPipeline(Pipeline.getFromProtoBuf(locatedBlock.getPipeline())) - .build()); - } - return locatedBlocks; - } - - /** * Asks SCM where a block should be allocated. SCM responds with the * set of datanodes that should be used creating this block. * @param size - size of the block. @@ -144,7 +102,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB response.getErrorMessage() : "Allocate block failed."); } AllocatedBlock.Builder builder = new AllocatedBlock.Builder() - .setKey(response.getKey()) + .setBlockID(BlockID.getFromProtobuf(response.getBlockID())) .setPipeline(Pipeline.getFromProtoBuf(response.getPipeline())) .setShouldCreateContainer(response.getCreateContainer()); return builder.build(); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index 3638f63e65..bba4e172be 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -17,7 +17,6 @@ package org.apache.hadoop.hdds.scm.protocolPB; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; @@ -92,20 +91,14 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB * supports replication factor of either 1 or 3. * @param type - Replication Type * @param factor - Replication Count - * @param containerName - Name * @return * @throws IOException */ @Override - public Pipeline allocateContainer(HddsProtos.ReplicationType type, - HddsProtos.ReplicationFactor factor, String - containerName, String owner) throws IOException { + public ContainerInfo allocateContainer(HddsProtos.ReplicationType type, + HddsProtos.ReplicationFactor factor, String owner) throws IOException { - Preconditions.checkNotNull(containerName, "Container Name cannot be Null"); - Preconditions.checkState(!containerName.isEmpty(), "Container name cannot" + - " be empty"); ContainerRequestProto request = ContainerRequestProto.newBuilder() - .setContainerName(containerName) .setReplicationFactor(factor) .setReplicationType(type) .setOwner(owner) @@ -121,22 +114,20 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB throw new IOException(response.hasErrorMessage() ? response.getErrorMessage() : "Allocate container failed."); } - return Pipeline.getFromProtoBuf(response.getPipeline()); + return ContainerInfo.fromProtobuf(response.getContainerInfo()); } - public Pipeline getContainer(String containerName) throws IOException { - Preconditions.checkNotNull(containerName, - "Container Name cannot be Null"); - Preconditions.checkState(!containerName.isEmpty(), - "Container name cannot be empty"); + public ContainerInfo getContainer(long containerID) throws IOException { + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative"); GetContainerRequestProto request = GetContainerRequestProto .newBuilder() - .setContainerName(containerName) + .setContainerID(containerID) .build(); try { GetContainerResponseProto response = rpcProxy.getContainer(NULL_RPC_CONTROLLER, request); - return Pipeline.getFromProtoBuf(response.getPipeline()); + return ContainerInfo.fromProtobuf(response.getContainerInfo()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } @@ -146,16 +137,15 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB * {@inheritDoc} */ @Override - public List<ContainerInfo> listContainer(String startName, String prefixName, - int count) throws IOException { + public List<ContainerInfo> listContainer(long startContainerID, int count) + throws IOException { + Preconditions.checkState(startContainerID >= 0, + "Container ID cannot be negative."); + Preconditions.checkState(count > 0, + "Container count must be greater than 0."); SCMListContainerRequestProto.Builder builder = SCMListContainerRequestProto .newBuilder(); - if (prefixName != null) { - builder.setPrefixName(prefixName); - } - if (startName != null) { - builder.setStartName(startName); - } + builder.setStartContainerID(startContainerID); builder.setCount(count); SCMListContainerRequestProto request = builder.build(); @@ -177,17 +167,17 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB * Ask SCM to delete a container by name. SCM will remove * the container mapping in its database. * - * @param containerName + * @param containerID * @throws IOException */ @Override - public void deleteContainer(String containerName) + public void deleteContainer(long containerID) throws IOException { - Preconditions.checkState(!Strings.isNullOrEmpty(containerName), - "Container name cannot be null or empty"); + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative"); SCMDeleteContainerRequestProto request = SCMDeleteContainerRequestProto .newBuilder() - .setContainerName(containerName) + .setContainerID(containerID) .build(); try { rpcProxy.deleteContainer(NULL_RPC_CONTROLLER, request); @@ -226,21 +216,21 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB /** * Notify from client that creates object on datanodes. * @param type object type - * @param name object name + * @param id object id * @param op operation type (e.g., create, close, delete) * @param stage object creation stage : begin/complete */ @Override public void notifyObjectStageChange( - ObjectStageChangeRequestProto.Type type, String name, + ObjectStageChangeRequestProto.Type type, long id, ObjectStageChangeRequestProto.Op op, ObjectStageChangeRequestProto.Stage stage) throws IOException { - Preconditions.checkState(!Strings.isNullOrEmpty(name), - "Object name cannot be null or empty"); + Preconditions.checkState(id >= 0, + "Object id cannot be negative."); ObjectStageChangeRequestProto request = ObjectStageChangeRequestProto.newBuilder() .setType(type) - .setName(name) + .setId(id) .setOp(op) .setStage(stage) .build(); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 1559816bc4..970e932b10 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos .WriteChunkRequestProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue; +import org.apache.hadoop.hdds.client.BlockID; import java.io.IOException; @@ -79,7 +80,6 @@ public final class ContainerProtocolCalls { KeyData containerKeyData, String traceID) throws IOException { GetKeyRequestProto.Builder readKeyRequest = GetKeyRequestProto .newBuilder() - .setPipeline(xceiverClient.getPipeline().getProtobufMessage()) .setKeyData(containerKeyData); String id = xceiverClient.getPipeline().getLeader().getUuidString(); ContainerCommandRequestProto request = ContainerCommandRequestProto @@ -106,7 +106,6 @@ public final class ContainerProtocolCalls { KeyData containerKeyData, String traceID) throws IOException { PutKeyRequestProto.Builder createKeyRequest = PutKeyRequestProto .newBuilder() - .setPipeline(xceiverClient.getPipeline().getProtobufMessage()) .setKeyData(containerKeyData); String id = xceiverClient.getPipeline().getLeader().getUuidString(); ContainerCommandRequestProto request = ContainerCommandRequestProto @@ -125,18 +124,16 @@ public final class ContainerProtocolCalls { * * @param xceiverClient client to perform call * @param chunk information about chunk to read - * @param key the key name + * @param blockID ID of the block * @param traceID container protocol call args * @return container protocol read chunk response * @throws IOException if there is an I/O error while performing the call */ public static ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient, - ChunkInfo chunk, String key, String traceID) - throws IOException { + ChunkInfo chunk, BlockID blockID, String traceID) throws IOException { ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto .newBuilder() - .setPipeline(xceiverClient.getPipeline().getProtobufMessage()) - .setKeyName(key) + .setBlockID(blockID.getProtobuf()) .setChunkData(chunk); String id = xceiverClient.getPipeline().getLeader().getUuidString(); ContainerCommandRequestProto request = ContainerCommandRequestProto @@ -156,18 +153,17 @@ public final class ContainerProtocolCalls { * * @param xceiverClient client to perform call * @param chunk information about chunk to write - * @param key the key name + * @param blockID ID of the block * @param data the data of the chunk to write * @param traceID container protocol call args * @throws IOException if there is an I/O error while performing the call */ public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk, - String key, ByteString data, String traceID) + BlockID blockID, ByteString data, String traceID) throws IOException { WriteChunkRequestProto.Builder writeChunkRequest = WriteChunkRequestProto .newBuilder() - .setPipeline(xceiverClient.getPipeline().getProtobufMessage()) - .setKeyName(key) + .setBlockID(blockID.getProtobuf()) .setChunkData(chunk) .setData(data); String id = xceiverClient.getPipeline().getLeader().getUuidString(); @@ -189,30 +185,29 @@ public final class ContainerProtocolCalls { * than 1 MB. * * @param client - client that communicates with the container. - * @param containerName - Name of the container - * @param key - Name of the Key + * @param blockID - ID of the block * @param data - Data to be written into the container. * @param traceID - Trace ID for logging purpose. * @throws IOException */ public static void writeSmallFile(XceiverClientSpi client, - String containerName, String key, byte[] data, String traceID) + BlockID blockID, byte[] data, String traceID) throws IOException { KeyData containerKeyData = - KeyData.newBuilder().setContainerName(containerName).setName(key) + KeyData.newBuilder().setBlockID(blockID.getProtobuf()) .build(); PutKeyRequestProto.Builder createKeyRequest = PutKeyRequestProto.newBuilder() - .setPipeline(client.getPipeline().getProtobufMessage()) .setKeyData(containerKeyData); KeyValue keyValue = KeyValue.newBuilder().setKey("OverWriteRequested").setValue("true") .build(); ChunkInfo chunk = - ChunkInfo.newBuilder().setChunkName(key + "_chunk").setOffset(0) - .setLen(data.length).addMetadata(keyValue).build(); + ChunkInfo.newBuilder().setChunkName(blockID.getLocalID() + + "_chunk").setOffset(0).setLen(data.length). + addMetadata(keyValue).build(); PutSmallFileRequestProto putSmallFileRequest = PutSmallFileRequestProto.newBuilder().setChunkInfo(chunk) @@ -234,17 +229,18 @@ public final class ContainerProtocolCalls { /** * createContainer call that creates a container on the datanode. * @param client - client + * @param containerID - ID of container * @param traceID - traceID * @throws IOException */ - public static void createContainer(XceiverClientSpi client, String traceID) - throws IOException { + public static void createContainer(XceiverClientSpi client, long containerID, + String traceID) throws IOException { ContainerProtos.CreateContainerRequestProto.Builder createRequest = ContainerProtos.CreateContainerRequestProto .newBuilder(); ContainerProtos.ContainerData.Builder containerData = ContainerProtos .ContainerData.newBuilder(); - containerData.setName(client.getPipeline().getContainerName()); + containerData.setContainerID(containerID); createRequest.setPipeline(client.getPipeline().getProtobufMessage()); createRequest.setContainerData(containerData.build()); @@ -268,12 +264,11 @@ public final class ContainerProtocolCalls { * @param traceID * @throws IOException */ - public static void deleteContainer(XceiverClientSpi client, + public static void deleteContainer(XceiverClientSpi client, long containerID, boolean force, String traceID) throws IOException { ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest = ContainerProtos.DeleteContainerRequestProto.newBuilder(); - deleteRequest.setName(client.getPipeline().getContainerName()); - deleteRequest.setPipeline(client.getPipeline().getProtobufMessage()); + deleteRequest.setContainerID(containerID); deleteRequest.setForceDelete(force); String id = client.getPipeline().getLeader().getUuidString(); ContainerCommandRequestProto.Builder request = @@ -291,14 +286,15 @@ public final class ContainerProtocolCalls { * Close a container. * * @param client + * @param containerID * @param traceID * @throws IOException */ - public static void closeContainer(XceiverClientSpi client, String traceID) - throws IOException { + public static void closeContainer(XceiverClientSpi client, + long containerID, String traceID) throws IOException { ContainerProtos.CloseContainerRequestProto.Builder closeRequest = ContainerProtos.CloseContainerRequestProto.newBuilder(); - closeRequest.setPipeline(client.getPipeline().getProtobufMessage()); + closeRequest.setContainerID(containerID); String id = client.getPipeline().getLeader().getUuidString(); ContainerCommandRequestProto.Builder request = @@ -320,11 +316,11 @@ public final class ContainerProtocolCalls { * @throws IOException */ public static ReadContainerResponseProto readContainer( - XceiverClientSpi client, String containerName, + XceiverClientSpi client, long containerID, String traceID) throws IOException { ReadContainerRequestProto.Builder readRequest = ReadContainerRequestProto.newBuilder(); - readRequest.setName(containerName); + readRequest.setContainerID(containerID); readRequest.setPipeline(client.getPipeline().getProtobufMessage()); String id = client.getPipeline().getLeader().getUuidString(); ContainerCommandRequestProto.Builder request = @@ -340,25 +336,23 @@ public final class ContainerProtocolCalls { } /** - * Reads the data given the container name and key. + * Reads the data given the blockID * * @param client - * @param containerName - name of the container - * @param key - key + * @param blockID - ID of the block * @param traceID - trace ID * @return GetSmallFileResponseProto * @throws IOException */ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, - String containerName, String key, String traceID) throws IOException { + BlockID blockID, String traceID) throws IOException { KeyData containerKeyData = KeyData .newBuilder() - .setContainerName(containerName) - .setName(key).build(); + .setBlockID(blockID.getProtobuf()) + .build(); GetKeyRequestProto.Builder getKey = GetKeyRequestProto .newBuilder() - .setPipeline(client.getPipeline().getProtobufMessage()) .setKeyData(containerKeyData); ContainerProtos.GetSmallFileRequestProto getSmallFileRequest = GetSmallFileRequestProto diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java index 38ce6ccb0a..7a5403f290 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java @@ -17,9 +17,12 @@ package org.apache.hadoop.ozone.common; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos .KeyBlocks; +import java.util.ArrayList; import java.util.List; /** @@ -28,13 +31,13 @@ import java.util.List; public final class BlockGroup { private String groupID; - private List<String> blockIDs; - private BlockGroup(String groupID, List<String> blockIDs) { + private List<BlockID> blockIDs; + private BlockGroup(String groupID, List<BlockID> blockIDs) { this.groupID = groupID; this.blockIDs = blockIDs; } - public List<String> getBlockIDList() { + public List<BlockID> getBlockIDList() { return blockIDs; } @@ -43,8 +46,11 @@ public final class BlockGroup { } public KeyBlocks getProto() { - return KeyBlocks.newBuilder().setKey(groupID) - .addAllBlocks(blockIDs).build(); + KeyBlocks.Builder kbb = KeyBlocks.newBuilder(); + for (BlockID block : blockIDs) { + kbb.addBlocks(block.getProtobuf()); + } + return kbb.setKey(groupID).build(); } /** @@ -53,8 +59,12 @@ public final class BlockGroup { * @return a group of blocks. */ public static BlockGroup getFromProto(KeyBlocks proto) { + List<BlockID> blockIDs = new ArrayList<>(); + for (HddsProtos.BlockID block : proto.getBlocksList()) { + blockIDs.add(new BlockID(block.getContainerID(), block.getLocalID())); + } return BlockGroup.newBuilder().setKeyName(proto.getKey()) - .addAllBlockIDs(proto.getBlocksList()).build(); + .addAllBlockIDs(blockIDs).build(); } public static Builder newBuilder() { @@ -67,14 +77,14 @@ public final class BlockGroup { public static class Builder { private String groupID; - private List<String> blockIDs; + private List<BlockID> blockIDs; public Builder setKeyName(String blockGroupID) { this.groupID = blockGroupID; return this; } - public Builder addAllBlockIDs(List<String> keyBlocks) { + public Builder addAllBlockIDs(List<BlockID> keyBlocks) { this.blockIDs = keyBlocks; return this; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java index ec54ac5407..892b695153 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.ozone.common; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos .DeleteScmBlockResult; @@ -52,7 +53,7 @@ public class DeleteBlockGroupResult { new ArrayList<>(blockResultList.size()); for (DeleteBlockResult result : blockResultList) { DeleteScmBlockResult proto = DeleteScmBlockResult.newBuilder() - .setKey(result.getKey()) + .setBlockID(result.getBlockID().getProtobuf()) .setResult(result.getResult()).build(); resultProtoList.add(proto); } @@ -63,8 +64,8 @@ public class DeleteBlockGroupResult { List<DeleteScmBlockResult> results) { List<DeleteBlockResult> protoResults = new ArrayList<>(results.size()); for (DeleteScmBlockResult result : results) { - protoResults.add(new DeleteBlockResult(result.getKey(), - result.getResult())); + protoResults.add(new DeleteBlockResult(BlockID.getFromProtobuf( + result.getBlockID()), result.getResult())); } return protoResults; } @@ -87,10 +88,10 @@ public class DeleteBlockGroupResult { /** * @return A list of deletion failed block IDs. */ - public List<String> getFailedBlocks() { - List<String> failedBlocks = blockResultList.stream() + public List<BlockID> getFailedBlocks() { + List<BlockID> failedBlocks = blockResultList.stream() .filter(result -> result.getResult() != Result.success) - .map(DeleteBlockResult::getKey).collect(Collectors.toList()); + .map(DeleteBlockResult::getBlockID).collect(Collectors.toList()); return failedBlocks; } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java index be546c75c3..c3de5ed25f 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.helpers; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.client.BlockID; import java.io.IOException; import java.util.Collections; @@ -30,8 +31,7 @@ import java.util.TreeMap; * Helper class to convert Protobuf to Java classes. */ public class KeyData { - private final String containerName; - private final String keyName; + private final BlockID blockID; private final Map<String, String> metadata; /** @@ -44,12 +44,10 @@ public class KeyData { /** * Constructs a KeyData Object. * - * @param containerName - * @param keyName + * @param blockID */ - public KeyData(String containerName, String keyName) { - this.containerName = containerName; - this.keyName = keyName; + public KeyData(BlockID blockID) { + this.blockID = blockID; this.metadata = new TreeMap<>(); } @@ -62,7 +60,7 @@ public class KeyData { */ public static KeyData getFromProtoBuf(ContainerProtos.KeyData data) throws IOException { - KeyData keyData = new KeyData(data.getContainerName(), data.getName()); + KeyData keyData = new KeyData(BlockID.getFromProtobuf(data.getBlockID())); for (int x = 0; x < data.getMetadataCount(); x++) { keyData.addMetadata(data.getMetadata(x).getKey(), data.getMetadata(x).getValue()); @@ -78,8 +76,7 @@ public class KeyData { public ContainerProtos.KeyData getProtoBufMessage() { ContainerProtos.KeyData.Builder builder = ContainerProtos.KeyData.newBuilder(); - builder.setContainerName(this.containerName); - builder.setName(this.getKeyName()); + builder.setBlockID(this.blockID.getProtobuf()); builder.addAllChunks(this.chunks); for (Map.Entry<String, String> entry : metadata.entrySet()) { HddsProtos.KeyValue.Builder keyValBuilder = @@ -135,19 +132,27 @@ public class KeyData { } /** - * Returns container Name. - * @return String. + * Returns container ID. + * @return long. */ - public String getContainerName() { - return containerName; + public long getContainerID() { + return blockID.getContainerID(); } /** - * Returns KeyName. - * @return String. + * Returns LocalID. + * @return long. */ - public String getKeyName() { - return keyName; + public long getLocalID() { + return blockID.getLocalID(); + } + + /** + * Return Block ID. + * @return BlockID. + */ + public BlockID getBlockID() { + return blockID; } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java index fa793419bf..37a13095f6 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.ozone.protocolPB; -import com.google.common.collect.Sets; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; @@ -38,18 +37,11 @@ import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos .DeleteScmKeyBlocksRequestProto; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos .DeleteScmKeyBlocksResponseProto; -import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos - .GetScmBlockLocationsRequestProto; -import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos - .GetScmBlockLocationsResponseProto; -import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos - .ScmLocatedBlockProto; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; import java.io.IOException; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; /** @@ -73,34 +65,6 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB this.impl = impl; } - - @Override - public GetScmBlockLocationsResponseProto getScmBlockLocations( - RpcController controller, GetScmBlockLocationsRequestProto req) - throws ServiceException { - Set<String> keys = Sets.newLinkedHashSetWithExpectedSize( - req.getKeysCount()); - for (String key : req.getKeysList()) { - keys.add(key); - } - final Set<AllocatedBlock> blocks; - try { - blocks = impl.getBlockLocations(keys); - } catch (IOException ex) { - throw new ServiceException(ex); - } - GetScmBlockLocationsResponseProto.Builder resp = - GetScmBlockLocationsResponseProto.newBuilder(); - for (AllocatedBlock block: blocks) { - ScmLocatedBlockProto.Builder locatedBlock = - ScmLocatedBlockProto.newBuilder() - .setKey(block.getKey()) - .setPipeline(block.getPipeline().getProtobufMessage()); - resp.addLocatedBlocks(locatedBlock.build()); - } - return resp.build(); - } - @Override public AllocateScmBlockResponseProto allocateScmBlock( RpcController controller, AllocateScmBlockRequestProto request) @@ -112,7 +76,7 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB if (allocatedBlock != null) { return AllocateScmBlockResponseProto.newBuilder() - .setKey(allocatedBlock.getKey()) + .setBlockID(allocatedBlock.getBlockID().getProtobuf()) .setPipeline(allocatedBlock.getPipeline().getProtobufMessage()) .setCreateContainer(allocatedBlock.getCreateContainer()) .setErrorCode(AllocateScmBlockResponseProto.Error.success) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java index 4974268bcb..70a0e8a8da 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -23,7 +23,6 @@ import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -83,11 +82,10 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB public ContainerResponseProto allocateContainer(RpcController unused, ContainerRequestProto request) throws ServiceException { try { - Pipeline pipeline = impl.allocateContainer(request.getReplicationType(), - request.getReplicationFactor(), request.getContainerName(), - request.getOwner()); + ContainerInfo container = impl.allocateContainer(request.getReplicationType(), + request.getReplicationFactor(), request.getOwner()); return ContainerResponseProto.newBuilder() - .setPipeline(pipeline.getProtobufMessage()) + .setContainerInfo(container.getProtobuf()) .setErrorCode(ContainerResponseProto.Error.success) .build(); @@ -101,9 +99,9 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB RpcController controller, GetContainerRequestProto request) throws ServiceException { try { - Pipeline pipeline = impl.getContainer(request.getContainerName()); + ContainerInfo container = impl.getContainer(request.getContainerID()); return GetContainerResponseProto.newBuilder() - .setPipeline(pipeline.getProtobufMessage()) + .setContainerInfo(container.getProtobuf()) .build(); } catch (IOException e) { throw new ServiceException(e); @@ -114,23 +112,17 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB public SCMListContainerResponseProto listContainer(RpcController controller, SCMListContainerRequestProto request) throws ServiceException { try { - String startName = null; - String prefixName = null; + long startContainerID = 0; int count = -1; // Arguments check. - if (request.hasPrefixName()) { + if (request.hasStartContainerID()) { // End container name is given. - prefixName = request.getPrefixName(); + startContainerID = request.getStartContainerID(); } - if (request.hasStartName()) { - // End container name is given. - startName = request.getStartName(); - } - count = request.getCount(); List<ContainerInfo> containerList = - impl.listContainer(startName, prefixName, count); + impl.listContainer(startContainerID, count); SCMListContainerResponseProto.Builder builder = SCMListContainerResponseProto.newBuilder(); for (ContainerInfo container : containerList) { @@ -147,7 +139,7 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB RpcController controller, SCMDeleteContainerRequestProto request) throws ServiceException { try { - impl.deleteContainer(request.getContainerName()); + impl.deleteContainer(request.getContainerID()); return SCMDeleteContainerResponseProto.newBuilder().build(); } catch (IOException e) { throw new ServiceException(e); @@ -178,7 +170,7 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB RpcController controller, ObjectStageChangeRequestProto request) throws ServiceException { try { - impl.notifyObjectStageChange(request.getType(), request.getName(), + impl.notifyObjectStageChange(request.getType(), request.getId(), request.getOp(), request.getStage()); return ObjectStageChangeResponseProto.newBuilder().build(); } catch (IOException e) { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java index 83ca83d80d..13b918015e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java @@ -76,7 +76,9 @@ public class LevelDBStore implements MetadataStore { } private void openDB(File dbPath, Options options) throws IOException { - dbPath.getParentFile().mkdirs(); + if (dbPath.getParentFile().mkdirs()) { + LOG.debug("Db path {} created.", dbPath.getParentFile()); + } db = JniDBFactory.factory.open(dbPath, options); if (LOG.isDebugEnabled()) { LOG.debug("LevelDB successfully opened"); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java index 3ff0a948a8..d3a29435de 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.utils; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.ozone.OzoneConsts; /** @@ -94,8 +94,8 @@ public final class MetadataKeyFilters { if (Strings.isNullOrEmpty(keyPrefix)) { accept = true; } else { - if (currentKey != null && - DFSUtil.bytes2String(currentKey).startsWith(keyPrefix)) { + byte [] prefixBytes = keyPrefix.getBytes(); + if (currentKey != null && prefixMatch(prefixBytes, currentKey)) { keysHinted++; accept = true; } else { @@ -114,5 +114,19 @@ public final class MetadataKeyFilters { public int getKeysHintedNum() { return keysHinted; } + + private boolean prefixMatch(byte[] prefix, byte[] key) { + Preconditions.checkNotNull(prefix); + Preconditions.checkNotNull(key); + if (key.length < prefix.length) { + return false; + } + for (int i = 0; i < prefix.length; i++) { + if (key[i] != prefix[i]) { + return false; + } + } + return true; + } } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java index a60e98d9ab..0dfca20a8f 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java @@ -367,6 +367,7 @@ public class RocksDBStore implements MetadataStore { public void close() throws IOException { if (statMBeanName != null) { MBeans.unregister(statMBeanName); + statMBeanName = null; } if (db != null) { db.close(); diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto index a6270eff50..e7494ee4c6 100644 --- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto @@ -197,17 +197,15 @@ message ContainerCommandResponseProto { } message ContainerData { - required string name = 1; + required int64 containerID = 1; repeated KeyValue metadata = 2; optional string dbPath = 3; optional string containerPath = 4; - optional string hash = 6; - optional int64 bytesUsed = 7; - optional int64 size = 8; - optional int64 keyCount = 9; - //TODO: change required after we switch container ID from string to long - optional int64 containerID = 10; - optional LifeCycleState state = 11 [default = OPEN]; + optional string hash = 5; + optional int64 bytesUsed = 6; + optional int64 size = 7; + optional int64 keyCount = 8; + optional LifeCycleState state = 9 [default = OPEN]; } message ContainerMeta { @@ -226,7 +224,7 @@ message CreateContainerResponseProto { message ReadContainerRequestProto { required Pipeline pipeline = 1; - required string name = 2; + required int64 containerID = 2; } message ReadContainerResponseProto { @@ -243,19 +241,16 @@ message UpdateContainerResponseProto { } message DeleteContainerRequestProto { - required Pipeline pipeline = 1; - required string name = 2; - optional bool forceDelete = 3 [default = false]; + required int64 containerID = 1; + optional bool forceDelete = 2 [default = false]; } message DeleteContainerResponseProto { } message ListContainerRequestProto { - required Pipeline pipeline = 1; - optional string prefix = 2; - required uint32 count = 3; // Max Results to return - optional string prevKey = 4; // if this is not set query from start. + required int64 startContainerID = 1; + optional uint32 count = 2; // Max Results to return } message ListContainerResponseProto { @@ -263,34 +258,31 @@ message ListContainerResponseProto { } message CloseContainerRequestProto { - required Pipeline pipeline = 1; + required int64 containerID = 1; } message CloseContainerResponseProto { - optional Pipeline pipeline = 1; optional string hash = 2; + optional int64 containerID = 3; } message KeyData { - required string containerName = 1; - required string name = 2; - optional int64 flags = 3; // for future use. - repeated KeyValue metadata = 4; - repeated ChunkInfo chunks = 5; + required BlockID blockID = 1; + optional int64 flags = 2; // for future use. + repeated KeyValue metadata = 3; + repeated ChunkInfo chunks = 4; } // Key Messages. message PutKeyRequestProto { - required Pipeline pipeline = 1; - required KeyData keyData = 2; + required KeyData keyData = 1; } message PutKeyResponseProto { } message GetKeyRequestProto { - required Pipeline pipeline = 1; - required KeyData keyData = 2; + required KeyData keyData = 1; } message GetKeyResponseProto { @@ -299,17 +291,15 @@ message GetKeyResponseProto { message DeleteKeyRequestProto { - required Pipeline pipeline = 1; - required string name = 2; + required BlockID blockID = 1; } message DeleteKeyResponseProto { } message ListKeyRequestProto { - required Pipeline pipeline = 1; - optional string prefix = 2; // if specified returns keys that match prefix. - required string prevKey = 3; + required int64 containerID = 1; + optional int64 startLocalID = 2; required uint32 count = 4; } @@ -335,31 +325,28 @@ enum Stage { } message WriteChunkRequestProto { - required Pipeline pipeline = 1; - required string keyName = 2; - required ChunkInfo chunkData = 3; - optional bytes data = 4; - optional Stage stage = 5 [default = COMBINED]; + required BlockID blockID = 1; + required ChunkInfo chunkData = 2; + optional bytes data = 3; + optional Stage stage = 4 [default = COMBINED]; } message WriteChunkResponseProto { } message ReadChunkRequestProto { - required Pipeline pipeline = 1; - required string keyName = 2; - required ChunkInfo chunkData = 3; + required BlockID blockID = 1; + required ChunkInfo chunkData = 2; } message ReadChunkResponseProto { - required Pipeline pipeline = 1; + required BlockID blockID = 1; required ChunkInfo chunkData = 2; required bytes data = 3; } message DeleteChunkRequestProto { - required Pipeline pipeline = 1; - required string keyName = 2; + required BlockID blockID = 1; required ChunkInfo chunkData = 3; } @@ -367,10 +354,9 @@ message DeleteChunkResponseProto { } message ListChunkRequestProto { - required Pipeline pipeline = 1; - required string keyName = 2; - required string prevChunkName = 3; - required uint32 count = 4; + required BlockID blockID = 1; + required string prevChunkName = 2; + required uint32 count = 3; } message ListChunkResponseProto { @@ -400,7 +386,7 @@ message GetSmallFileResponseProto { } message CopyContainerRequestProto { - required string containerName = 1; + required int64 containerID = 1; required uint64 readOffset = 2; optional uint64 len = 3; } diff --git a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto index 38d2e16ce8..7bea82ab86 100644 --- a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto @@ -33,28 +33,6 @@ import "hdds.proto"; // SCM Block protocol -/** - * keys - batch of block keys to find - */ -message GetScmBlockLocationsRequestProto { - repeated string keys = 1; -} - -/** - * locatedBlocks - for each requested hash, nodes that currently host the - * container for that object key hash - */ -message GetScmBlockLocationsResponseProto { - repeated ScmLocatedBlockProto locatedBlocks = 1; -} - -/** - * Holds the nodes that currently host the blocks for a key. - */ -message ScmLocatedBlockProto { - required string key = 1; - required hadoop.hdds.Pipeline pipeline = 2; -} /** * Request send to SCM asking allocate block of specified size. @@ -84,7 +62,7 @@ message DeleteScmKeyBlocksRequestProto { */ message KeyBlocks { required string key = 1; - repeated string blocks = 2; + repeated BlockID blocks = 2; } /** @@ -112,7 +90,7 @@ message DeleteScmBlockResult { unknownFailure = 4; } required Result result = 1; - required string key = 2; + required BlockID blockID = 2; } /** @@ -126,7 +104,7 @@ message AllocateScmBlockResponseProto { unknownFailure = 4; } required Error errorCode = 1; - required string key = 2; + required BlockID blockID = 2; required hadoop.hdds.Pipeline pipeline = 3; required bool createContainer = 4; optional string errorMessage = 5; @@ -139,14 +117,6 @@ message AllocateScmBlockResponseProto { service ScmBlockLocationProtocolService { /** - * Find the set of nodes that currently host the block, as - * identified by the key. This method supports batch lookup by - * passing multiple keys. - */ - rpc getScmBlockLocations(GetScmBlockLocationsRequestProto) - returns (GetScmBlockLocationsResponseProto); - - /** * Creates a block entry in SCM. */ rpc allocateScmBlock(AllocateScmBlockRequestProto) diff --git a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto index d7540a3fe4..090e6ebde4 100644 --- a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto @@ -35,7 +35,6 @@ import "hdds.proto"; * Request send to SCM asking where the container should be created. */ message ContainerRequestProto { - required string containerName = 1; // Ozone only support replciation of either 1 or 3. required ReplicationFactor replicationFactor = 2; required ReplicationType replicationType = 3; @@ -53,30 +52,29 @@ message ContainerResponseProto { errorContainerMissing = 3; } required Error errorCode = 1; - required Pipeline pipeline = 2; + required SCMContainerInfo containerInfo = 2; optional string errorMessage = 3; } message GetContainerRequestProto { - required string containerName = 1; + required int64 containerID = 1; } message GetContainerResponseProto { - required Pipeline pipeline = 1; + required SCMContainerInfo containerInfo = 1; } message SCMListContainerRequestProto { required uint32 count = 1; - optional string startName = 2; - optional string prefixName = 3; -} + optional uint64 startContainerID = 2; + } message SCMListContainerResponseProto { repeated SCMContainerInfo containers = 1; } message SCMDeleteContainerRequestProto { - required string containerName = 1; + required int64 containerID = 1; } message SCMDeleteContainerResponseProto { @@ -97,7 +95,7 @@ message ObjectStageChangeRequestProto { begin = 1; complete = 2; } - required string name = 1; + required int64 id = 1; required Type type = 2; required Op op= 3; required Stage stage = 4; diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto index 0b650b45ea..6ea5727963 100644 --- a/hadoop-hdds/common/src/main/proto/hdds.proto +++ b/hadoop-hdds/common/src/main/proto/hdds.proto @@ -50,7 +50,6 @@ message PipelineChannel { // A pipeline is composed of PipelineChannel (Ratis/StandAlone) that back a // container. message Pipeline { - required string containerName = 1; required PipelineChannel pipelineChannel = 2; } @@ -135,8 +134,7 @@ enum LifeCycleEvent { } message SCMContainerInfo { - // TODO : Remove the container name from pipeline. - required string containerName = 1; + required int64 containerID = 1; required LifeCycleState state = 2; required Pipeline pipeline = 3; // This is not total size of container, but space allocated by SCM for @@ -146,7 +144,6 @@ message SCMContainerInfo { required uint64 numberOfKeys = 6; optional int64 stateEnterTime = 7; required string owner = 8; - required int64 containerID = 9; } message GetScmInfoRequestProto { @@ -168,3 +165,11 @@ enum ReplicationFactor { ONE = 1; THREE = 3; } + +/** + * Block ID that uniquely identify a block by SCM. + */ +message BlockID { + required int64 containerID = 1; + required int64 localID = 2; +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java index 68bf4421f6..8c5609d63c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import org.apache.commons.codec.binary.Hex; import org.apache.commons.codec.digest.DigestUtils; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; @@ -105,18 +104,17 @@ public final class ChunkUtils { * Validates chunk data and returns a file object to Chunk File that we are * expected to write data to. * - * @param pipeline - pipeline. * @param data - container data. * @param info - chunk info. * @return File * @throws StorageContainerException */ - public static File validateChunk(Pipeline pipeline, ContainerData data, + public static File validateChunk(ContainerData data, ChunkInfo info) throws StorageContainerException { Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class); - File chunkFile = getChunkFile(pipeline, data, info); + File chunkFile = getChunkFile(data, info); if (ChunkUtils.isOverWriteRequested(chunkFile, info)) { if (!ChunkUtils.isOverWritePermitted(info)) { log.error("Rejecting write chunk request. Chunk overwrite " + @@ -132,21 +130,21 @@ public final class ChunkUtils { /** * Validates that Path to chunk file exists. * - * @param pipeline - Container Info. * @param data - Container Data * @param info - Chunk info * @return - File. * @throws StorageContainerException */ - public static File getChunkFile(Pipeline pipeline, ContainerData data, + public static File getChunkFile(ContainerData data, ChunkInfo info) throws StorageContainerException { + Preconditions.checkNotNull(data, "Container data can't be null"); Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class); - if (data == null) { - log.error("Invalid container Name: {}", pipeline.getContainerName()); - throw new StorageContainerException("Unable to find the container Name:" + + if (data.getContainerID() < 0) { + log.error("Invalid container id: {}", data.getContainerID()); + throw new StorageContainerException("Unable to find the container id:" + " " + - pipeline.getContainerName(), CONTAINER_NOT_FOUND); + data.getContainerID(), CONTAINER_NOT_FOUND); } File dataDir = ContainerUtils.getDataDirectory(data).toFile(); @@ -335,7 +333,7 @@ public final class ChunkUtils { ContainerProtos.ReadChunkResponseProto.newBuilder(); response.setChunkData(info.getProtoBufMessage()); response.setData(ByteString.copyFrom(data)); - response.setPipeline(msg.getReadChunk().getPipeline()); + response.setBlockID(msg.getReadChunk().getBlockID()); ContainerProtos.ContainerCommandResponseProto.Builder builder = ContainerUtils.getContainerResponse(msg, ContainerProtos.Result diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java index c29374c07c..c20282adaf 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java @@ -40,7 +40,6 @@ import java.util.concurrent.atomic.AtomicLong; */ public class ContainerData { - private final String containerName; private final Map<String, String> metadata; private String dbPath; // Path to Level DB Store. // Path to Physical file system where container and checksum are stored. @@ -48,18 +47,18 @@ public class ContainerData { private String hash; private AtomicLong bytesUsed; private long maxSize; - private Long containerID; + private long containerID; private HddsProtos.LifeCycleState state; /** * Constructs a ContainerData Object. * - * @param containerName - Name + * @param containerID - ID + * @param conf - Configuration */ - public ContainerData(String containerName, Long containerID, + public ContainerData(long containerID, Configuration conf) { this.metadata = new TreeMap<>(); - this.containerName = containerName; this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY, ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB; this.bytesUsed = new AtomicLong(0L); @@ -76,7 +75,7 @@ public class ContainerData { public static ContainerData getFromProtBuf( ContainerProtos.ContainerData protoData, Configuration conf) throws IOException { - ContainerData data = new ContainerData(protoData.getName(), + ContainerData data = new ContainerData( protoData.getContainerID(), conf); for (int x = 0; x < protoData.getMetadataCount(); x++) { data.addMetadata(protoData.getMetadata(x).getKey(), @@ -117,7 +116,6 @@ public class ContainerData { public ContainerProtos.ContainerData getProtoBufMessage() { ContainerProtos.ContainerData.Builder builder = ContainerProtos .ContainerData.newBuilder(); - builder.setName(this.getContainerName()); builder.setContainerID(this.getContainerID()); if (this.getDBPath() != null) { @@ -157,15 +155,6 @@ public class ContainerData { } /** - * Returns the name of the container. - * - * @return - name - */ - public String getContainerName() { - return containerName; - } - - /** * Adds metadata. */ public void addMetadata(String key, String value) throws IOException { @@ -231,9 +220,11 @@ public class ContainerData { * * @return String Name. */ - public String getName() { - return getContainerName(); - } + // TODO: check the ContainerCache class to see if we are using the ContainerID instead. + /* + public String getName() { + return getContainerID(); + }*/ /** * Get container file path. @@ -255,7 +246,7 @@ public class ContainerData { * Get container ID. * @return - container ID. */ - public synchronized Long getContainerID() { + public synchronized long getContainerID() { return containerID; } @@ -284,7 +275,7 @@ public class ContainerData { // Some thing brain dead for now. name + Time stamp of when we get the close // container message. - setHash(DigestUtils.sha256Hex(this.getContainerName() + + setHash(DigestUtils.sha256Hex(this.getContainerID() + Long.toString(Time.monotonicNow()))); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java index 50d2da3975..19634f48b8 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolPro */ public class ContainerReport { private static final int UNKNOWN = -1; - private final String containerName; private final String finalhash; private long size; private long keyCount; @@ -51,11 +50,11 @@ public class ContainerReport { /** * Constructs the ContainerReport. * - * @param containerName - Container Name. + * @param containerID - Container ID. * @param finalhash - Final Hash. */ - public ContainerReport(String containerName, String finalhash) { - this.containerName = containerName; + public ContainerReport(long containerID, String finalhash) { + this.containerID = containerID; this.finalhash = finalhash; this.size = UNKNOWN; this.keyCount = UNKNOWN; @@ -74,7 +73,7 @@ public class ContainerReport { */ public static ContainerReport getFromProtoBuf(ContainerInfo info) { Preconditions.checkNotNull(info); - ContainerReport report = new ContainerReport(info.getContainerName(), + ContainerReport report = new ContainerReport(info.getContainerID(), info.getFinalhash()); if (info.hasSize()) { report.setSize(info.getSize()); @@ -103,15 +102,6 @@ public class ContainerReport { } /** - * Gets the container name. - * - * @return - Name - */ - public String getContainerName() { - return containerName; - } - - /** * Returns the final signature for this container. * * @return - hash @@ -203,7 +193,6 @@ public class ContainerReport { */ public ContainerInfo getProtoBufMessage() { return ContainerInfo.newBuilder() - .setContainerName(this.getContainerName()) .setKeyCount(this.getKeyCount()) .setSize(this.getSize()) .setUsed(this.getBytesUsed()) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java index 1818188cb6..e24435418d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java @@ -184,6 +184,12 @@ public final class ContainerUtils { removeExtension(containerFile.getName())).toString(); } + public static long getContainerIDFromFile(File containerFile) { + Preconditions.checkNotNull(containerFile); + String containerID = getContainerNameFromFile(containerFile); + return Long.parseLong(containerID); + } + /** * Verifies that this in indeed a new container. * @@ -289,8 +295,8 @@ public final class ContainerUtils { */ public static File getMetadataFile(ContainerData containerData, Path location) { - return location.resolve(containerData - .getContainerName().concat(CONTAINER_META)) + return location.resolve(Long.toString(containerData + .getContainerID()).concat(CONTAINER_META)) .toFile(); } @@ -303,8 +309,8 @@ public final class ContainerUtils { */ public static File getContainerFile(ContainerData containerData, Path location) { - return location.resolve(containerData - .getContainerName().concat(CONTAINER_EXTENSION)) + return location.resolve(Long.toString(containerData + .getContainerID()).concat(CONTAINER_EXTENSION)) .toFile(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java index ade162a263..9d0ec957f2 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.helpers; import com.google.common.collect.Maps; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; +import org.apache.hadoop.util.StringUtils; import java.util.List; import java.util.Map; @@ -37,7 +38,7 @@ public final class DeletedContainerBlocksSummary { // value : the number of blocks need to be deleted in this container // if the message contains multiple entries for same block, // blocks will be merged - private final Map<String, Integer> blockSummary; + private final Map<Long, Integer> blockSummary; // total number of blocks in this message private int numOfBlocks; @@ -47,14 +48,14 @@ public final class DeletedContainerBlocksSummary { blockSummary = Maps.newHashMap(); blocks.forEach(entry -> { txSummary.put(entry.getTxID(), entry.getCount()); - if (blockSummary.containsKey(entry.getContainerName())) { - blockSummary.put(entry.getContainerName(), - blockSummary.get(entry.getContainerName()) - + entry.getBlockIDCount()); + if (blockSummary.containsKey(entry.getContainerID())) { + blockSummary.put(entry.getContainerID(), + blockSummary.get(entry.getContainerID()) + + entry.getLocalIDCount()); } else { - blockSummary.put(entry.getContainerName(), entry.getBlockIDCount()); + blockSummary.put(entry.getContainerID(), entry.getLocalIDCount()); } - numOfBlocks += entry.getBlockIDCount(); + numOfBlocks += entry.getLocalIDCount(); }); } @@ -93,9 +94,9 @@ public final class DeletedContainerBlocksSummary { .append("TimesProceed=") .append(blks.getCount()) .append(", ") - .append(blks.getContainerName()) + .append(blks.getContainerID()) .append(" : [") - .append(String.join(",", blks.getBlockIDList())).append("]") + .append(StringUtils.join(',', blks.getLocalIDList())).append("]") .append("\n"); } return sb.toString(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java index 566db02510..ec274525e4 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java @@ -65,7 +65,8 @@ public final class FileUtils { ContainerProtos.ReadChunkResponseProto.newBuilder(); readChunkresponse.setChunkData(info.getProtoBufMessage()); readChunkresponse.setData(ByteString.copyFrom(data)); - readChunkresponse.setPipeline(msg.getGetSmallFile().getKey().getPipeline()); + readChunkresponse.setBlockID(msg.getGetSmallFile().getKey(). + getKeyData().getBlockID()); ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile = ContainerProtos.GetSmallFileResponseProto.newBuilder(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java index 33eb911d4e..dbd5772d32 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java @@ -63,11 +63,11 @@ public final class KeyUtils { ContainerCache cache = ContainerCache.getInstance(conf); Preconditions.checkNotNull(cache); try { - return cache.getDB(container.getContainerName(), container.getDBPath()); + return cache.getDB(container.getContainerID(), container.getDBPath()); } catch (IOException ex) { String message = String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s", - container.getContainerName(), container.getDBPath(), ex.getMessage()); + container.getContainerID(), container.getDBPath(), ex.getMessage()); throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB); } } @@ -83,7 +83,7 @@ public final class KeyUtils { Preconditions.checkNotNull(container); ContainerCache cache = ContainerCache.getInstance(conf); Preconditions.checkNotNull(cache); - cache.removeDB(container.getContainerName()); + cache.removeDB(container.getContainerID()); } /** * Shutdown all DB Handles. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java index 457c417b10..350519692c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java @@ -19,11 +19,11 @@ package org.apache.hadoop.ozone.container.common.impl; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; @@ -66,13 +66,12 @@ public class ChunkManagerImpl implements ChunkManager { /** * writes a given chunk. * - * @param pipeline - Name and the set of machines that make this container. - * @param keyName - Name of the Key. + * @param blockID - ID of the block. * @param info - ChunkInfo. * @throws StorageContainerException */ @Override - public void writeChunk(Pipeline pipeline, String keyName, ChunkInfo info, + public void writeChunk(BlockID blockID, ChunkInfo info, byte[] data, ContainerProtos.Stage stage) throws StorageContainerException { // we don't want container manager to go away while we are writing chunks. @@ -80,13 +79,13 @@ public class ChunkManagerImpl implements ChunkManager { // TODO : Take keyManager Write lock here. try { - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - String containerName = pipeline.getContainerName(); - Preconditions.checkNotNull(containerName, - "Container name cannot be null"); + Preconditions.checkNotNull(blockID, "Block ID cannot be null."); + long containerID = blockID.getContainerID(); + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative"); ContainerData container = - containerManager.readContainer(containerName); - File chunkFile = ChunkUtils.validateChunk(pipeline, container, info); + containerManager.readContainer(containerID); + File chunkFile = ChunkUtils.validateChunk(container, info); File tmpChunkFile = getTmpChunkFile(chunkFile, info); LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file", @@ -96,16 +95,16 @@ public class ChunkManagerImpl implements ChunkManager { ChunkUtils.writeData(tmpChunkFile, info, data); break; case COMMIT_DATA: - commitChunk(tmpChunkFile, chunkFile, containerName, info.getLen()); + commitChunk(tmpChunkFile, chunkFile, containerID, info.getLen()); break; case COMBINED: // directly write to the chunk file long oldSize = chunkFile.length(); ChunkUtils.writeData(chunkFile, info, data); long newSize = chunkFile.length(); - containerManager.incrBytesUsed(containerName, newSize - oldSize); - containerManager.incrWriteCount(containerName); - containerManager.incrWriteBytes(containerName, info.getLen()); + containerManager.incrBytesUsed(containerID, newSize - oldSize); + containerManager.incrWriteCount(containerID); + containerManager.incrWriteBytes(containerID, info.getLen()); break; default: throw new IOException("Can not identify write operation."); @@ -136,22 +135,21 @@ public class ChunkManagerImpl implements ChunkManager { // Commit the chunk by renaming the temporary chunk file to chunk file private void commitChunk(File tmpChunkFile, File chunkFile, - String containerName, long chunkLen) throws IOException { + long containerID, long chunkLen) throws IOException { long sizeDiff = tmpChunkFile.length() - chunkFile.length(); // It is safe to replace here as the earlier chunk if existing should be // caught as part of validateChunk Files.move(tmpChunkFile.toPath(), chunkFile.toPath(), StandardCopyOption.REPLACE_EXISTING); - containerManager.incrBytesUsed(containerName, sizeDiff); - containerManager.incrWriteCount(containerName); - containerManager.incrWriteBytes(containerName, chunkLen); + containerManager.incrBytesUsed(containerID, sizeDiff); + containerManager.incrWriteCount(containerID); + containerManager.incrWriteBytes(containerID, chunkLen); } /** * reads the data defined by a chunk. * - * @param pipeline - container pipeline. - * @param keyName - Name of the Key + * @param blockID - ID of the block. * @param info - ChunkInfo. * @return byte array * @throws StorageContainerException @@ -159,20 +157,20 @@ public class ChunkManagerImpl implements ChunkManager { * TODO: Explore if we need to do that for ozone. */ @Override - public byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info) + public byte[] readChunk(BlockID blockID, ChunkInfo info) throws StorageContainerException { containerManager.readLock(); try { - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - String containerName = pipeline.getContainerName(); - Preconditions.checkNotNull(containerName, - "Container name cannot be null"); + Preconditions.checkNotNull(blockID, "Block ID cannot be null."); + long containerID = blockID.getContainerID(); + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative"); ContainerData container = - containerManager.readContainer(containerName); - File chunkFile = ChunkUtils.getChunkFile(pipeline, container, info); + containerManager.readContainer(containerID); + File chunkFile = ChunkUtils.getChunkFile(container, info); ByteBuffer data = ChunkUtils.readData(chunkFile, info); - containerManager.incrReadCount(containerName); - containerManager.incrReadBytes(containerName, chunkFile.length()); + containerManager.incrReadCount(containerID); + containerManager.incrReadBytes(containerID, chunkFile.length()); return data.array(); } catch (ExecutionException | NoSuchAlgorithmException e) { LOG.error("read data failed. error: {}", e); @@ -191,25 +189,25 @@ public class ChunkManagerImpl implements ChunkManager { /** * Deletes a given chunk. * - * @param pipeline - Pipeline. - * @param keyName - Key Name + * @param blockID - ID of the block. * @param info - Chunk Info * @throws StorageContainerException */ @Override - public void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) + public void deleteChunk(BlockID blockID, ChunkInfo info) throws StorageContainerException { containerManager.readLock(); try { - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - String containerName = pipeline.getContainerName(); - Preconditions.checkNotNull(containerName, - "Container name cannot be null"); - File chunkFile = ChunkUtils.getChunkFile(pipeline, containerManager - .readContainer(containerName), info); + Preconditions.checkNotNull(blockID, "Block ID cannot be null."); + long containerID = blockID.getContainerID(); + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative"); + + File chunkFile = ChunkUtils.getChunkFile(containerManager + .readContainer(containerID), info); if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) { FileUtil.fullyDelete(chunkFile); - containerManager.decrBytesUsed(containerName, chunkFile.length()); + containerManager.decrBytesUsed(containerID, chunkFile.length()); } else { LOG.error("Not Supported Operation. Trying to delete a " + "chunk that is in shared file. chunk info : " + info.toString()); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java index 5e7375cd9d..1893b3b3b8 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java @@ -24,7 +24,6 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; import org.apache.hadoop.hdfs.server.datanode.StorageLocation; @@ -113,7 +112,9 @@ public class ContainerManagerImpl implements ContainerManager { static final Logger LOG = LoggerFactory.getLogger(ContainerManagerImpl.class); - private final ConcurrentSkipListMap<String, ContainerStatus> + // TODO: consider primitive collection like eclipse-collections + // to avoid autoboxing overhead + private final ConcurrentSkipListMap<Long, ContainerStatus> containerMap = new ConcurrentSkipListMap<>(); // Use a non-fair RW lock for better throughput, we may revisit this decision @@ -229,6 +230,7 @@ public class ContainerManagerImpl implements ContainerManager { Preconditions.checkNotNull(keyName, "Container Name to container key mapping is null"); + long containerID = Long.parseLong(keyName); try { String containerFileName = containerName.concat(CONTAINER_EXTENSION); String metaFileName = containerName.concat(CONTAINER_META); @@ -249,7 +251,7 @@ public class ContainerManagerImpl implements ContainerManager { // when loading the info we get a null, this often means last time // SCM was ending up at some middle phase causing that the metadata // was not populated. Such containers are marked as inactive. - containerMap.put(keyName, new ContainerStatus(null)); + containerMap.put(containerID, new ContainerStatus(null)); return; } containerData = ContainerData.getFromProtBuf(containerDataProto, conf); @@ -263,7 +265,7 @@ public class ContainerManagerImpl implements ContainerManager { // Hopefully SCM will ask us to delete this container and rebuild it. LOG.error("Invalid SHA found for container data. Name :{}" + "cowardly refusing to read invalid data", containerName); - containerMap.put(keyName, new ContainerStatus(null)); + containerMap.put(containerID, new ContainerStatus(null)); return; } @@ -295,7 +297,7 @@ public class ContainerManagerImpl implements ContainerManager { }).sum(); containerStatus.setBytesUsed(bytesUsed); - containerMap.put(keyName, containerStatus); + containerMap.put(containerID, containerStatus); } catch (IOException | NoSuchAlgorithmException ex) { LOG.error("read failed for file: {} ex: {}", containerName, ex.getMessage()); @@ -303,7 +305,7 @@ public class ContainerManagerImpl implements ContainerManager { // TODO : Add this file to a recovery Queue. // Remember that this container is busted and we cannot use it. - containerMap.put(keyName, new ContainerStatus(null)); + containerMap.put(containerID, new ContainerStatus(null)); throw new StorageContainerException("Unable to read container info", UNABLE_TO_READ_METADATA_DB); } finally { @@ -316,18 +318,17 @@ public class ContainerManagerImpl implements ContainerManager { /** * Creates a container with the given name. * - * @param pipeline -- Nodes which make up this container. * @param containerData - Container Name and metadata. * @throws StorageContainerException - Exception */ @Override - public void createContainer(Pipeline pipeline, ContainerData containerData) + public void createContainer(ContainerData containerData) throws StorageContainerException { Preconditions.checkNotNull(containerData, "Container data cannot be null"); writeLock(); try { - if (containerMap.containsKey(containerData.getName())) { - LOG.debug("container already exists. {}", containerData.getName()); + if (containerMap.containsKey(containerData.getContainerID())) { + LOG.debug("container already exists. {}", containerData.getContainerID()); throw new StorageContainerException("container already exists.", CONTAINER_EXISTS); } @@ -399,7 +400,7 @@ public class ContainerManagerImpl implements ContainerManager { location); File metadataFile = ContainerUtils.getMetadataFile(containerData, location); - String containerName = containerData.getContainerName(); + String containerName = Long.toString(containerData.getContainerID()); if(!overwrite) { ContainerUtils.verifyIsNewContainer(containerFile, metadataFile); @@ -446,7 +447,7 @@ public class ContainerManagerImpl implements ContainerManager { LOG.error("Creation of container failed. Name: {}, we might need to " + "cleanup partially created artifacts. ", - containerData.getContainerName(), ex); + containerData.getContainerID(), ex); throw new StorageContainerException("Container creation failed. ", ex, CONTAINER_INTERNAL_ERROR); } finally { @@ -459,45 +460,45 @@ public class ContainerManagerImpl implements ContainerManager { /** * Deletes an existing container. * - * @param pipeline - nodes that make this container. - * @param containerName - name of the container. + * @param containerID - ID of the container. * @param forceDelete - whether this container should be deleted forcibly. * @throws StorageContainerException */ @Override - public void deleteContainer(Pipeline pipeline, String containerName, + public void deleteContainer(long containerID, boolean forceDelete) throws StorageContainerException { - Preconditions.checkNotNull(containerName, "Container name cannot be null"); - Preconditions.checkState(containerName.length() > 0, - "Container name length cannot be zero."); + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative."); writeLock(); try { - if (isOpen(pipeline.getContainerName())) { + if (isOpen(containerID)) { throw new StorageContainerException( "Deleting an open container is not allowed.", UNCLOSED_CONTAINER_IO); } - ContainerStatus status = containerMap.get(containerName); + ContainerStatus status = containerMap.get(containerID); if (status == null) { - LOG.debug("No such container. Name: {}", containerName); - throw new StorageContainerException("No such container. Name : " + - containerName, CONTAINER_NOT_FOUND); + LOG.debug("No such container. ID: {}", containerID); + throw new StorageContainerException("No such container. ID : " + + containerID, CONTAINER_NOT_FOUND); } if (status.getContainer() == null) { - LOG.debug("Invalid container data. Name: {}", containerName); + LOG.debug("Invalid container data. ID: {}", containerID); throw new StorageContainerException("Invalid container data. Name : " + - containerName, CONTAINER_NOT_FOUND); + containerID, CONTAINER_NOT_FOUND); } ContainerUtils.removeContainer(status.getContainer(), conf, forceDelete); - containerMap.remove(containerName); + containerMap.remove(containerID); } catch (StorageContainerException e) { throw e; } catch (IOException e) { // TODO : An I/O error during delete can leave partial artifacts on the // disk. We will need the cleaner thread to cleanup this information. - LOG.error("Failed to cleanup container. Name: {}", containerName, e); - throw new StorageContainerException(containerName, e, IO_EXCEPTION); + String errMsg = String.format("Failed to cleanup container. ID: %d", + containerID); + LOG.error(errMsg, e); + throw new StorageContainerException(errMsg, e, IO_EXCEPTION); } finally { writeUnlock(); } @@ -511,25 +512,29 @@ public class ContainerManagerImpl implements ContainerManager { * time. It is possible that using this iteration you can miss certain * container from the listing. * - * @param prefix - Return keys that match this prefix. + * @param startContainerID - Return containers with ID >= startContainerID. * @param count - how many to return - * @param prevKey - Previous Key Value or empty String. * @param data - Actual containerData * @throws StorageContainerException */ @Override - public void listContainer(String prefix, long count, String prevKey, + public void listContainer(long startContainerID, long count, List<ContainerData> data) throws StorageContainerException { - // TODO : Support list with Prefix and PrevKey Preconditions.checkNotNull(data, "Internal assertion: data cannot be null"); + Preconditions.checkState(startContainerID >= 0, + "Start container ID cannot be negative"); + Preconditions.checkState(count > 0, + "max number of containers returned " + + "must be positive"); + readLock(); try { - ConcurrentNavigableMap<String, ContainerStatus> map; - if (prevKey == null || prevKey.isEmpty()) { + ConcurrentNavigableMap<Long, ContainerStatus> map; + if (startContainerID == 0) { map = containerMap.tailMap(containerMap.firstKey(), true); } else { - map = containerMap.tailMap(prevKey, false); + map = containerMap.tailMap(startContainerID, false); } int currentCount = 0; @@ -549,24 +554,23 @@ public class ContainerManagerImpl implements ContainerManager { /** * Get metadata about a specific container. * - * @param containerName - Name of the container + * @param containerID - ID of the container * @return ContainerData - Container Data. * @throws StorageContainerException */ @Override - public ContainerData readContainer(String containerName) throws - StorageContainerException { - Preconditions.checkNotNull(containerName, "Container name cannot be null"); - Preconditions.checkState(containerName.length() > 0, - "Container name length cannot be zero."); - if (!containerMap.containsKey(containerName)) { - throw new StorageContainerException("Unable to find the container. Name: " - + containerName, CONTAINER_NOT_FOUND); + public ContainerData readContainer(long containerID) + throws StorageContainerException { + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative."); + if (!containerMap.containsKey(containerID)) { + throw new StorageContainerException("Unable to find the container. ID: " + + containerID, CONTAINER_NOT_FOUND); } - ContainerData cData = containerMap.get(containerName).getContainer(); + ContainerData cData = containerMap.get(containerID).getContainer(); if (cData == null) { - throw new StorageContainerException("Invalid container data. Name: " - + containerName, CONTAINER_INTERNAL_ERROR); + throw new StorageContainerException("Invalid container data. ID: " + + containerID, CONTAINER_INTERNAL_ERROR); } return cData; } @@ -575,13 +579,13 @@ public class ContainerManagerImpl implements ContainerManager { * Closes a open container, if it is already closed or does not exist a * StorageContainerException is thrown. * - * @param containerName - Name of the container. + * @param containerID - ID of the container. * @throws StorageContainerException */ @Override - public void closeContainer(String containerName) + public void closeContainer(long containerID) throws StorageContainerException, NoSuchAlgorithmException { - ContainerData containerData = readContainer(containerName); + ContainerData containerData = readContainer(containerID); containerData.closeContainer(); writeContainerInfo(containerData, true); MetadataStore db = KeyUtils.getDB(containerData, conf); @@ -602,15 +606,13 @@ public class ContainerManagerImpl implements ContainerManager { // issues. ContainerStatus status = new ContainerStatus(containerData); - containerMap.put(containerName, status); + containerMap.put(containerID, status); } @Override - public void updateContainer(Pipeline pipeline, String containerName, - ContainerData data, boolean forceUpdate) - throws StorageContainerException { - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - Preconditions.checkNotNull(containerName, "Container name cannot be null"); + public void updateContainer(long containerID, ContainerData data, + boolean forceUpdate) throws StorageContainerException { + Preconditions.checkState(containerID >= 0, "Container ID cannot be negative."); Preconditions.checkNotNull(data, "Container data cannot be null"); FileOutputStream containerStream = null; DigestOutputStream dos = null; @@ -618,9 +620,9 @@ public class ContainerManagerImpl implements ContainerManager { File containerFileBK = null, containerFile = null; boolean deleted = false; - if(!containerMap.containsKey(containerName)) { + if(!containerMap.containsKey(containerID)) { throw new StorageContainerException("Container doesn't exist. Name :" - + containerName, CONTAINER_NOT_FOUND); + + containerID, CONTAINER_NOT_FOUND); } try { @@ -633,7 +635,7 @@ public class ContainerManagerImpl implements ContainerManager { try { Path location = locationManager.getContainerPath(); - ContainerData orgData = containerMap.get(containerName).getContainer(); + ContainerData orgData = containerMap.get(containerID).getContainer(); if (orgData == null) { // updating a invalid container throw new StorageContainerException("Update a container with invalid" + @@ -642,7 +644,7 @@ public class ContainerManagerImpl implements ContainerManager { if (!forceUpdate && !orgData.isOpen()) { throw new StorageContainerException( - "Update a closed container is not allowed. Name: " + containerName, + "Update a closed container is not allowed. ID: " + containerID, UNSUPPORTED_REQUEST); } @@ -652,7 +654,7 @@ public class ContainerManagerImpl implements ContainerManager { if (!forceUpdate) { if (!containerFile.exists() || !containerFile.canWrite()) { throw new StorageContainerException( - "Container file not exists or corrupted. Name: " + containerName, + "Container file not exists or corrupted. ID: " + containerID, CONTAINER_INTERNAL_ERROR); } @@ -672,7 +674,7 @@ public class ContainerManagerImpl implements ContainerManager { // Update the in-memory map ContainerStatus newStatus = new ContainerStatus(data); - containerMap.replace(containerName, newStatus); + containerMap.replace(containerID, newStatus); } catch (IOException e) { // Restore the container file from backup if(containerFileBK != null && containerFileBK.exists() && deleted) { @@ -683,8 +685,8 @@ public class ContainerManagerImpl implements ContainerManager { CONTAINER_INTERNAL_ERROR); } else { throw new StorageContainerException( - "Failed to restore container data from the backup. Name: " - + containerName, CONTAINER_INTERNAL_ERROR); + "Failed to restore container data from the backup. ID: " + + containerID, CONTAINER_INTERNAL_ERROR); } } else { throw new StorageContainerException( @@ -711,22 +713,22 @@ public class ContainerManagerImpl implements ContainerManager { /** * Checks if a container exists. * - * @param containerName - Name of the container. + * @param containerID - ID of the container. * @return true if the container is open false otherwise. * @throws StorageContainerException - Throws Exception if we are not able to * find the container. */ @Override - public boolean isOpen(String containerName) throws StorageContainerException { - final ContainerStatus status = containerMap.get(containerName); + public boolean isOpen(long containerID) throws StorageContainerException { + final ContainerStatus status = containerMap.get(containerID); if (status == null) { throw new StorageContainerException( - "Container status not found: " + containerName, CONTAINER_NOT_FOUND); + "Container status not found: " + containerID, CONTAINER_NOT_FOUND); } final ContainerData cData = status.getContainer(); if (cData == null) { throw new StorageContainerException( - "Container not found: " + containerName, CONTAINER_NOT_FOUND); + "Container not found: " + containerID, CONTAINER_NOT_FOUND); } return cData.isOpen(); } @@ -746,7 +748,7 @@ public class ContainerManagerImpl implements ContainerManager { @VisibleForTesting - public ConcurrentSkipListMap<String, ContainerStatus> getContainerMap() { + public ConcurrentSkipListMap<Long, ContainerStatus> getContainerMap() { return containerMap; } @@ -901,7 +903,7 @@ public class ContainerManagerImpl implements ContainerManager { for (ContainerStatus container: containers) { StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); - ciBuilder.setContainerName(container.getContainer().getContainerName()) + ciBuilder.setContainerID(container.getContainer().getContainerID()) .setSize(container.getContainer().getMaxSize()) .setUsed(container.getContainer().getBytesUsed()) .setKeyCount(container.getContainer().getKeyCount()) @@ -966,7 +968,7 @@ public class ContainerManagerImpl implements ContainerManager { } @Override - public void incrPendingDeletionBlocks(int numBlocks, String containerId) { + public void incrPendingDeletionBlocks(int numBlocks, long containerId) { writeLock(); try { ContainerStatus status = containerMap.get(containerId); @@ -977,7 +979,7 @@ public class ContainerManagerImpl implements ContainerManager { } @Override - public void decrPendingDeletionBlocks(int numBlocks, String containerId) { + public void decrPendingDeletionBlocks(int numBlocks, long containerId) { writeLock(); try { ContainerStatus status = containerMap.get(containerId); @@ -990,35 +992,35 @@ public class ContainerManagerImpl implements ContainerManager { /** * Increase the read count of the container. * - * @param containerName - Name of the container. + * @param containerId - ID of the container. */ @Override - public void incrReadCount(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public void incrReadCount(long containerId) { + ContainerStatus status = containerMap.get(containerId); status.incrReadCount(); } - public long getReadCount(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public long getReadCount(long containerId) { + ContainerStatus status = containerMap.get(containerId); return status.getReadCount(); } /** * Increse the read counter for bytes read from the container. * - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param readBytes - bytes read from the container. */ @Override - public void incrReadBytes(String containerName, long readBytes) { - ContainerStatus status = containerMap.get(containerName); + public void incrReadBytes(long containerId, long readBytes) { + ContainerStatus status = containerMap.get(containerId); status.incrReadBytes(readBytes); } - public long getReadBytes(String containerName) { + public long getReadBytes(long containerId) { readLock(); try { - ContainerStatus status = containerMap.get(containerName); + ContainerStatus status = containerMap.get(containerId); return status.getReadBytes(); } finally { readUnlock(); @@ -1028,76 +1030,76 @@ public class ContainerManagerImpl implements ContainerManager { /** * Increase the write count of the container. * - * @param containerName - Name of the container. + * @param containerId - Name of the container. */ @Override - public void incrWriteCount(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public void incrWriteCount(long containerId) { + ContainerStatus status = containerMap.get(containerId); status.incrWriteCount(); } - public long getWriteCount(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public long getWriteCount(long containerId) { + ContainerStatus status = containerMap.get(containerId); return status.getWriteCount(); } /** * Increse the write counter for bytes write into the container. * - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param writeBytes - bytes write into the container. */ @Override - public void incrWriteBytes(String containerName, long writeBytes) { - ContainerStatus status = containerMap.get(containerName); + public void incrWriteBytes(long containerId, long writeBytes) { + ContainerStatus status = containerMap.get(containerId); status.incrWriteBytes(writeBytes); } - public long getWriteBytes(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public long getWriteBytes(long containerId) { + ContainerStatus status = containerMap.get(containerId); return status.getWriteBytes(); } /** * Increase the bytes used by the container. * - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param used - additional bytes used by the container. * @return the current bytes used. */ @Override - public long incrBytesUsed(String containerName, long used) { - ContainerStatus status = containerMap.get(containerName); + public long incrBytesUsed(long containerId, long used) { + ContainerStatus status = containerMap.get(containerId); return status.incrBytesUsed(used); } /** * Decrease the bytes used by the container. * - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param used - additional bytes reclaimed by the container. * @return the current bytes used. */ @Override - public long decrBytesUsed(String containerName, long used) { - ContainerStatus status = containerMap.get(containerName); + public long decrBytesUsed(long containerId, long used) { + ContainerStatus status = containerMap.get(containerId); return status.decrBytesUsed(used); } - public long getBytesUsed(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public long getBytesUsed(long containerId) { + ContainerStatus status = containerMap.get(containerId); return status.getBytesUsed(); } /** * Get the number of keys in the container. * - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @return the current key count. */ @Override - public long getNumKeys(String containerName) { - ContainerStatus status = containerMap.get(containerName); + public long getNumKeys(long containerId) { + ContainerStatus status = containerMap.get(containerId); return status.getNumKeys(); } /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java index d319565012..46bd842953 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.proto.ContainerProtos .ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Type; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; @@ -186,7 +187,7 @@ public class Dispatcher implements ContainerDispatcher { } catch (IOException ex) { LOG.warn("Container operation failed. " + "Container: {} Operation: {} trace ID: {} Error: {}", - msg.getCreateContainer().getContainerData().getName(), + msg.getCreateContainer().getContainerData().getContainerID(), msg.getCmdType().name(), msg.getTraceID(), ex.toString(), ex); @@ -230,7 +231,7 @@ public class Dispatcher implements ContainerDispatcher { } catch (IOException ex) { LOG.warn("Container operation failed. " + "Container: {} Operation: {} trace ID: {} Error: {}", - msg.getCreateContainer().getContainerData().getName(), + msg.getCreateContainer().getContainerData().getContainerID(), msg.getCmdType().name(), msg.getTraceID(), ex.toString(), ex); @@ -273,7 +274,7 @@ public class Dispatcher implements ContainerDispatcher { } catch (IOException ex) { LOG.warn("Container operation failed. " + "Container: {} Operation: {} trace ID: {} Error: {}", - msg.getCreateContainer().getContainerData().getName(), + msg.getCreateContainer().getContainerData().getContainerID(), msg.getCmdType().name(), msg.getTraceID(), ex.toString(), ex); @@ -318,17 +319,14 @@ public class Dispatcher implements ContainerDispatcher { msg.getTraceID()); return ContainerUtils.malformedRequest(msg); } - - Pipeline pipeline = Pipeline.getFromProtoBuf( - msg.getUpdateContainer().getPipeline()); - String containerName = msg.getUpdateContainer() - .getContainerData().getName(); + long containerID = msg.getUpdateContainer() + .getContainerData().getContainerID(); ContainerData data = ContainerData.getFromProtBuf( msg.getUpdateContainer().getContainerData(), conf); boolean forceUpdate = msg.getUpdateContainer().getForceUpdate(); - this.containerManager.updateContainer( - pipeline, containerName, data, forceUpdate); + this.containerManager.updateContainer(containerID, + data, forceUpdate); return ContainerUtils.getContainerResponse(msg); } @@ -349,8 +347,9 @@ public class Dispatcher implements ContainerDispatcher { return ContainerUtils.malformedRequest(msg); } - String name = msg.getReadContainer().getName(); - ContainerData container = this.containerManager.readContainer(name); + long containerID = msg.getReadContainer().getContainerID(); + ContainerData container = this.containerManager. + readContainer(containerID); return ContainerUtils.getReadContainerResponse(msg, container); } @@ -370,12 +369,9 @@ public class Dispatcher implements ContainerDispatcher { return ContainerUtils.malformedRequest(msg); } - Pipeline pipeline = Pipeline.getFromProtoBuf( - msg.getDeleteContainer().getPipeline()); - Preconditions.checkNotNull(pipeline); - String name = msg.getDeleteContainer().getName(); + long containerID = msg.getDeleteContainer().getContainerID(); boolean forceDelete = msg.getDeleteContainer().getForceDelete(); - this.containerManager.deleteContainer(pipeline, name, forceDelete); + this.containerManager.deleteContainer(containerID, forceDelete); return ContainerUtils.getContainerResponse(msg); } @@ -401,7 +397,7 @@ public class Dispatcher implements ContainerDispatcher { msg.getCreateContainer().getPipeline()); Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - this.containerManager.createContainer(pipeline, cData); + this.containerManager.createContainer(cData); return ContainerUtils.getContainerResponse(msg); } @@ -420,14 +416,12 @@ public class Dispatcher implements ContainerDispatcher { msg.getTraceID()); return ContainerUtils.malformedRequest(msg); } - Pipeline pipeline = Pipeline.getFromProtoBuf(msg.getCloseContainer() - .getPipeline()); - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - if (!this.containerManager.isOpen(pipeline.getContainerName())) { + long containerID = msg.getCloseContainer().getContainerID(); + if (!this.containerManager.isOpen(containerID)) { throw new StorageContainerException("Attempting to close a closed " + "container.", CLOSED_CONTAINER_IO); } - this.containerManager.closeContainer(pipeline.getContainerName()); + this.containerManager.closeContainer(containerID); return ContainerUtils.getContainerResponse(msg); } catch (NoSuchAlgorithmException e) { throw new StorageContainerException("No such Algorithm", e, @@ -449,11 +443,9 @@ public class Dispatcher implements ContainerDispatcher { msg.getTraceID()); return ContainerUtils.malformedRequest(msg); } - String keyName = msg.getWriteChunk().getKeyName(); - Pipeline pipeline = Pipeline.getFromProtoBuf( - msg.getWriteChunk().getPipeline()); - Preconditions.checkNotNull(pipeline); - if (!this.containerManager.isOpen(pipeline.getContainerName())) { + BlockID blockID = BlockID.getFromProtobuf( + msg.getWriteChunk().getBlockID()); + if (!this.containerManager.isOpen(blockID.getContainerID())) { throw new StorageContainerException("Write to closed container.", CLOSED_CONTAINER_IO); } @@ -469,7 +461,7 @@ public class Dispatcher implements ContainerDispatcher { } this.containerManager.getChunkManager() - .writeChunk(pipeline, keyName, chunkInfo, + .writeChunk(blockID, chunkInfo, data, msg.getWriteChunk().getStage()); return ChunkUtils.getChunkResponse(msg); @@ -489,17 +481,13 @@ public class Dispatcher implements ContainerDispatcher { msg.getTraceID()); return ContainerUtils.malformedRequest(msg); } - - String keyName = msg.getReadChunk().getKeyName(); - Pipeline pipeline = Pipeline.getFromProtoBuf( - msg.getReadChunk().getPipeline()); - Preconditions.checkNotNull(pipeline); - + BlockID blockID = BlockID.getFromProtobuf( + msg.getReadChunk().getBlockID()); ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getReadChunk() .getChunkData()); Preconditions.checkNotNull(chunkInfo); - byte[] data = this.containerManager.getChunkManager().readChunk(pipeline, - keyName, chunkInfo); + byte[] data = this.containerManager.getChunkManager(). + readChunk(blockID, chunkInfo); metrics.incContainerBytesStats(Type.ReadChunk, data.length); return ChunkUtils.getReadChunkResponse(msg, data, chunkInfo); } @@ -519,11 +507,10 @@ public class Dispatcher implements ContainerDispatcher { return ContainerUtils.malformedRequest(msg); } - String keyName = msg.getDeleteChunk().getKeyName(); - Pipeline pipeline = Pipeline.getFromProtoBuf( - msg.getDeleteChunk().getPipeline()); - Preconditions.checkNotNull(pipeline); - if (!this.containerManager.isOpen(pipeline.getContainerName())) { + BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteChunk() + .getBlockID()); + long containerID = blockID.getContainerID(); + if (!this.containerManager.isOpen(containerID)) { throw new StorageContainerException("Write to closed container.", CLOSED_CONTAINER_IO); } @@ -531,7 +518,7 @@ public class Dispatcher implements ContainerDispatcher { .getChunkData()); Preconditions.checkNotNull(chunkInfo); - this.containerManager.getChunkManager().deleteChunk(pipeline, keyName, + this.containerManager.getChunkManager().deleteChunk(blockID, chunkInfo); return ChunkUtils.getChunkResponse(msg); } @@ -550,15 +537,16 @@ public class Dispatcher implements ContainerDispatcher { msg.getTraceID()); return ContainerUtils.malformedRequest(msg); } - Pipeline pipeline = Pipeline.getFromProtoBuf(msg.getPutKey().getPipeline()); - Preconditions.checkNotNull(pipeline); - if (!this.containerManager.isOpen(pipeline.getContainerName())) { + BlockID blockID = BlockID.getFromProtobuf( + msg.getPutKey().getKeyData().getBlockID()); + long containerID = blockID.getContainerID(); + if (!this.containerManager.isOpen(containerID)) { throw new StorageContainerException("Write to closed container.", CLOSED_CONTAINER_IO); } KeyData keyData = KeyData.getFromProtoBuf(msg.getPutKey().getKeyData()); Preconditions.checkNotNull(keyData); - this.containerManager.getKeyManager().putKey(pipeline, keyData); + this.containerManager.getKeyManager().putKey(keyData); long numBytes = keyData.getProtoBufMessage().toByteArray().length; metrics.incContainerBytesStats(Type.PutKey, numBytes); return KeyUtils.getKeyResponse(msg); @@ -601,17 +589,15 @@ public class Dispatcher implements ContainerDispatcher { msg.getTraceID()); return ContainerUtils.malformedRequest(msg); } - Pipeline pipeline = - Pipeline.getFromProtoBuf(msg.getDeleteKey().getPipeline()); - Preconditions.checkNotNull(pipeline); - if (!this.containerManager.isOpen(pipeline.getContainerName())) { + BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteKey() + .getBlockID()); + Preconditions.checkNotNull(blockID); + long containerID = blockID.getContainerID(); + if (!this.containerManager.isOpen(containerID)) { throw new StorageContainerException("Write to closed container.", CLOSED_CONTAINER_IO); } - String keyName = msg.getDeleteKey().getName(); - Preconditions.checkNotNull(keyName); - Preconditions.checkState(!keyName.isEmpty()); - this.containerManager.getKeyManager().deleteKey(pipeline, keyName); + this.containerManager.getKeyManager().deleteKey(blockID); return KeyUtils.getKeyResponse(msg); } @@ -632,12 +618,11 @@ public class Dispatcher implements ContainerDispatcher { } try { - Pipeline pipeline = - Pipeline.getFromProtoBuf(msg.getPutSmallFile() - .getKey().getPipeline()); + BlockID blockID = BlockID.getFromProtobuf(msg. + getPutSmallFile().getKey().getKeyData().getBlockID()); + long containerID = blockID.getContainerID(); - Preconditions.checkNotNull(pipeline); - if (!this.containerManager.isOpen(pipeline.getContainerName())) { + if (!this.containerManager.isOpen(containerID)) { throw new StorageContainerException("Write to closed container.", CLOSED_CONTAINER_IO); } @@ -648,12 +633,12 @@ public class Dispatcher implements ContainerDispatcher { byte[] data = msg.getPutSmallFile().getData().toByteArray(); metrics.incContainerBytesStats(Type.PutSmallFile, data.length); - this.containerManager.getChunkManager().writeChunk(pipeline, keyData - .getKeyName(), chunkInfo, data, ContainerProtos.Stage.COMBINED); + this.containerManager.getChunkManager().writeChunk(blockID, + chunkInfo, data, ContainerProtos.Stage.COMBINED); List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>(); chunks.add(chunkInfo.getProtoBufMessage()); keyData.setChunks(chunks); - this.containerManager.getKeyManager().putKey(pipeline, keyData); + this.containerManager.getKeyManager().putKey(keyData); return FileUtils.getPutFileResponse(msg); } catch (StorageContainerException e) { return ContainerUtils.logAndReturnError(LOG, e, msg); @@ -680,12 +665,7 @@ public class Dispatcher implements ContainerDispatcher { return ContainerUtils.malformedRequest(msg); } try { - Pipeline pipeline = - Pipeline.getFromProtoBuf(msg.getGetSmallFile() - .getKey().getPipeline()); - long bytes = 0; - Preconditions.checkNotNull(pipeline); KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile() .getKey().getKeyData()); KeyData data = this.containerManager.getKeyManager().getKey(keyData); @@ -694,9 +674,8 @@ public class Dispatcher implements ContainerDispatcher { bytes += chunk.getSerializedSize(); ByteString current = ByteString.copyFrom(this.containerManager.getChunkManager() - .readChunk( - pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf( - chunk))); + .readChunk(keyData.getBlockID(), + ChunkInfo.getFromProtoBuf(chunk))); dataBuf = dataBuf.concat(current); c = chunk; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java index cf6bf12214..f920ded293 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java @@ -19,20 +19,18 @@ package org.apache.hadoop.ozone.container.common.impl; import com.google.common.base.Preconditions; +import com.google.common.primitives.Longs; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; -import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.KeyData; import org.apache.hadoop.ozone.container.common.helpers.KeyUtils; import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager; import org.apache.hadoop.ozone.container.common.interfaces.KeyManager; import org.apache.hadoop.ozone.container.common.utils.ContainerCache; -import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter; -import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter; import org.apache.hadoop.utils.MetadataStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,22 +71,21 @@ public class KeyManagerImpl implements KeyManager { * {@inheritDoc} */ @Override - public void putKey(Pipeline pipeline, KeyData data) throws IOException { + public void putKey(KeyData data) throws IOException { + Preconditions.checkNotNull(data, "KeyData cannot be null for put operation."); + Preconditions.checkState(data.getContainerID() >= 0, "Container ID cannot be negative"); containerManager.readLock(); try { // We are not locking the key manager since LevelDb serializes all actions // against a single DB. We rely on DB level locking to avoid conflicts. - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - String containerName = pipeline.getContainerName(); - Preconditions.checkNotNull(containerName, - "Container name cannot be null"); - ContainerData cData = containerManager.readContainer(containerName); + ContainerData cData = containerManager.readContainer( + data.getContainerID()); MetadataStore db = KeyUtils.getDB(cData, conf); // This is a post condition that acts as a hint to the user. // Should never fail. Preconditions.checkNotNull(db, "DB cannot be null here"); - db.put(data.getKeyName().getBytes(KeyUtils.ENCODING), data + db.put(Longs.toByteArray(data.getLocalID()), data .getProtoBufMessage().toByteArray()); } finally { containerManager.readUnlock(); @@ -103,17 +100,17 @@ public class KeyManagerImpl implements KeyManager { containerManager.readLock(); try { Preconditions.checkNotNull(data, "Key data cannot be null"); - Preconditions.checkNotNull(data.getContainerName(), + Preconditions.checkNotNull(data.getContainerID(), "Container name cannot be null"); ContainerData cData = containerManager.readContainer(data - .getContainerName()); + .getContainerID()); MetadataStore db = KeyUtils.getDB(cData, conf); // This is a post condition that acts as a hint to the user. // Should never fail. Preconditions.checkNotNull(db, "DB cannot be null here"); - byte[] kData = db.get(data.getKeyName().getBytes(KeyUtils.ENCODING)); + byte[] kData = db.get(Longs.toByteArray(data.getLocalID())); if (kData == null) { throw new StorageContainerException("Unable to find the key.", NO_SUCH_KEY); @@ -130,15 +127,19 @@ public class KeyManagerImpl implements KeyManager { * {@inheritDoc} */ @Override - public void deleteKey(Pipeline pipeline, String keyName) + public void deleteKey(BlockID blockID) throws IOException { + Preconditions.checkNotNull(blockID, "block ID cannot be null."); + Preconditions.checkState(blockID.getContainerID() >= 0, + "Container ID cannot be negative."); + Preconditions.checkState(blockID.getLocalID() >= 0, + "Local ID cannot be negative."); + containerManager.readLock(); try { - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); - String containerName = pipeline.getContainerName(); - Preconditions.checkNotNull(containerName, - "Container name cannot be null"); - ContainerData cData = containerManager.readContainer(containerName); + + ContainerData cData = containerManager + .readContainer(blockID.getContainerID()); MetadataStore db = KeyUtils.getDB(cData, conf); // This is a post condition that acts as a hint to the user. @@ -149,12 +150,13 @@ public class KeyManagerImpl implements KeyManager { // to delete a key which might have just gotten inserted after // the get check. - byte[] kData = db.get(keyName.getBytes(KeyUtils.ENCODING)); + byte[] kKey = Longs.toByteArray(blockID.getLocalID()); + byte[] kData = db.get(kKey); if (kData == null) { throw new StorageContainerException("Unable to find the key.", NO_SUCH_KEY); } - db.delete(keyName.getBytes(KeyUtils.ENCODING)); + db.delete(kKey); } finally { containerManager.readUnlock(); } @@ -165,26 +167,22 @@ public class KeyManagerImpl implements KeyManager { */ @Override public List<KeyData> listKey( - Pipeline pipeline, String prefix, String startKey, int count) + long containerID, long startLocalID, int count) throws IOException { - Preconditions.checkNotNull(pipeline, - "Pipeline cannot be null."); + Preconditions.checkState(containerID >= 0, "Container ID cannot be negative"); + Preconditions.checkState(startLocalID >= 0, "startLocal ID cannot be negative"); Preconditions.checkArgument(count > 0, "Count must be a positive number."); - ContainerData cData = containerManager.readContainer(pipeline - .getContainerName()); + ContainerData cData = containerManager.readContainer(containerID); MetadataStore db = KeyUtils.getDB(cData, conf); - List<KeyData> result = new ArrayList<KeyData>(); - byte[] startKeyInBytes = startKey == null ? null : - DFSUtil.string2Bytes(startKey); - MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefix); + List<KeyData> result = new ArrayList<>(); + byte[] startKeyInBytes = Longs.toByteArray(startLocalID); List<Map.Entry<byte[], byte[]>> range = - db.getSequentialRangeKVs(startKeyInBytes, count, prefixFilter); + db.getSequentialRangeKVs(startKeyInBytes, count, null); for (Map.Entry<byte[], byte[]> entry : range) { - String keyName = KeyUtils.getKeyName(entry.getKey()); KeyData value = KeyUtils.getKeyData(entry.getValue()); - KeyData data = new KeyData(value.getContainerName(), keyName); + KeyData data = new KeyData(value.getBlockID()); result.add(data); } return result; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java index 3e267d2b37..06177cb332 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java @@ -41,7 +41,7 @@ public class RandomContainerDeletionChoosingPolicy @Override public List<ContainerData> chooseContainerForBlockDeletion(int count, - Map<String, ContainerStatus> candidateContainers) + Map<Long, ContainerStatus> candidateContainers) throws StorageContainerException { Preconditions.checkNotNull(candidateContainers, "Internal assertion: candidate containers cannot be null"); @@ -58,7 +58,7 @@ public class RandomContainerDeletionChoosingPolicy LOG.debug("Select container {} for block deletion, " + "pending deletion blocks num: {}.", - entry.getContainer().getContainerName(), + entry.getContainer().getContainerID(), entry.getNumPendingDeletionBlocks()); } else { break; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java index 0169a96cf9..246342672f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java @@ -53,7 +53,7 @@ public class TopNOrderedContainerDeletionChoosingPolicy @Override public List<ContainerData> chooseContainerForBlockDeletion(int count, - Map<String, ContainerStatus> candidateContainers) + Map<Long, ContainerStatus> candidateContainers) throws StorageContainerException { Preconditions.checkNotNull(candidateContainers, "Internal assertion: candidate containers cannot be null"); @@ -74,7 +74,7 @@ public class TopNOrderedContainerDeletionChoosingPolicy LOG.debug( "Select container {} for block deletion, " + "pending deletion blocks num: {}.", - entry.getContainer().getContainerName(), + entry.getContainer().getContainerID(), entry.getNumPendingDeletionBlocks()); } else { LOG.debug("Stop looking for next container, there is no" diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java index f55d74ca2b..26dcf217cd 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java @@ -18,10 +18,10 @@ package org.apache.hadoop.ozone.container.common.interfaces; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; /** @@ -32,20 +32,18 @@ public interface ChunkManager { /** * writes a given chunk. - * @param pipeline - Name and the set of machines that make this container. - * @param keyName - Name of the Key. + * @param blockID - ID of the block. * @param info - ChunkInfo. * @param stage - Chunk Stage write. * @throws StorageContainerException */ - void writeChunk(Pipeline pipeline, String keyName, - ChunkInfo info, byte[] data, ContainerProtos.Stage stage) + void writeChunk(BlockID blockID, + ChunkInfo info, byte[] data, ContainerProtos.Stage stage) throws StorageContainerException; /** * reads the data defined by a chunk. - * @param pipeline - container pipeline. - * @param keyName - Name of the Key + * @param blockID - ID of the block. * @param info - ChunkInfo. * @return byte array * @throws StorageContainerException @@ -53,17 +51,16 @@ public interface ChunkManager { * TODO: Right now we do not support partial reads and writes of chunks. * TODO: Explore if we need to do that for ozone. */ - byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws + byte[] readChunk(BlockID blockID, ChunkInfo info) throws StorageContainerException; /** * Deletes a given chunk. - * @param pipeline - Pipeline. - * @param keyName - Key Name + * @param blockID - ID of the block. * @param info - Chunk Info * @throws StorageContainerException */ - void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws + void deleteChunk(BlockID blockID, ChunkInfo info) throws StorageContainerException; // TODO : Support list operations. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java index f7280e2a3c..6b60c52719 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java @@ -41,6 +41,6 @@ public interface ContainerDeletionChoosingPolicy { * @throws StorageContainerException */ List<ContainerData> chooseContainerForBlockDeletion(int count, - Map<String, ContainerStatus> candidateContainers) + Map<Long, ContainerStatus> candidateContainers) throws StorageContainerException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java index 2ff636e87f..84d95f8612 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java @@ -21,7 +21,6 @@ package org.apache.hadoop.ozone.container.common.interfaces; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; import org.apache.hadoop.hdfs.server.datanode.StorageLocation; @@ -60,48 +59,43 @@ public interface ContainerManager extends RwLock { /** * Creates a container with the given name. * - * @param pipeline -- Nodes which make up this container. * @param containerData - Container Name and metadata. * @throws StorageContainerException */ - void createContainer(Pipeline pipeline, ContainerData containerData) + void createContainer(ContainerData containerData) throws StorageContainerException; /** * Deletes an existing container. * - * @param pipeline - nodes that make this container. - * @param containerName - name of the container. + * @param containerID - ID of the container. * @param forceDelete - whether this container should be deleted forcibly. * @throws StorageContainerException */ - void deleteContainer(Pipeline pipeline, String containerName, + void deleteContainer(long containerID, boolean forceDelete) throws StorageContainerException; /** * Update an existing container. * - * @param pipeline container nodes - * @param containerName name of the container + * @param containerID ID of the container * @param data container data * @param forceUpdate if true, update container forcibly. * @throws StorageContainerException */ - void updateContainer(Pipeline pipeline, String containerName, - ContainerData data, boolean forceUpdate) throws StorageContainerException; + void updateContainer(long containerID, ContainerData data, + boolean forceUpdate) throws StorageContainerException; /** * As simple interface for container Iterations. * - * @param prefix - Return only values matching this prefix - * @param count - how many to return - * @param prevKey - Previous key - Server returns results from this point. - * @param data - Actual containerData + * @param startContainerID - Return containers with ID >= startContainerID. + * @param count - how many to return + * @param data - Actual containerData * @throws StorageContainerException */ - void listContainer(String prefix, long count, String prevKey, - List<ContainerData> data) - throws StorageContainerException; + void listContainer(long startContainerID, long count, + List<ContainerData> data) throws StorageContainerException; /** * Choose containers for block deletion. @@ -115,30 +109,30 @@ public interface ContainerManager extends RwLock { /** * Get metadata about a specific container. * - * @param containerName - Name of the container + * @param containerID - ID of the container. * @return ContainerData - Container Data. * @throws StorageContainerException */ - ContainerData readContainer(String containerName) + ContainerData readContainer(long containerID) throws StorageContainerException; /** * Closes a open container, if it is already closed or does not exist a * StorageContainerException is thrown. - * @param containerName - Name of the container. + * @param containerID - ID of the container. * @throws StorageContainerException */ - void closeContainer(String containerName) + void closeContainer(long containerID) throws StorageContainerException, NoSuchAlgorithmException; /** * Checks if a container exists. - * @param containerName - Name of the container. + * @param containerID - ID of the container. * @return true if the container is open false otherwise. * @throws StorageContainerException - Throws Exception if we are not * able to find the container. */ - boolean isOpen(String containerName) throws StorageContainerException; + boolean isOpen(long containerID) throws StorageContainerException; /** * Supports clean shutdown of container. @@ -203,7 +197,7 @@ public interface ContainerManager extends RwLock { * @param containerId * container id */ - void incrPendingDeletionBlocks(int numBlocks, String containerId); + void incrPendingDeletionBlocks(int numBlocks, long containerId); /** * Decrease pending deletion blocks count number of specified container. @@ -213,64 +207,64 @@ public interface ContainerManager extends RwLock { * @param containerId * container id */ - void decrPendingDeletionBlocks(int numBlocks, String containerId); + void decrPendingDeletionBlocks(int numBlocks, long containerId); /** * Increase the read count of the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. */ - void incrReadCount(String containerName); + void incrReadCount(long containerId); /** * Increse the read counter for bytes read from the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param readBytes - bytes read from the container. */ - void incrReadBytes(String containerName, long readBytes); + void incrReadBytes(long containerId, long readBytes); /** * Increase the write count of the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. */ - void incrWriteCount(String containerName); + void incrWriteCount(long containerId); /** * Increase the write counter for bytes write into the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param writeBytes - bytes write into the container. */ - void incrWriteBytes(String containerName, long writeBytes); + void incrWriteBytes(long containerId, long writeBytes); /** * Increase the bytes used by the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param used - additional bytes used by the container. * @return the current bytes used. */ - long incrBytesUsed(String containerName, long used); + long incrBytesUsed(long containerId, long used); /** * Decrease the bytes used by the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @param used - additional bytes reclaimed by the container. * @return the current bytes used. */ - long decrBytesUsed(String containerName, long used); + long decrBytesUsed(long containerId, long used); /** * Get the bytes used by the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @return the current bytes used by the container. */ - long getBytesUsed(String containerName); + long getBytesUsed(long containerId); /** * Get the number of keys in the container. - * @param containerName - Name of the container. + * @param containerId - ID of the container. * @return the current key count. */ - long getNumKeys(String containerName); + long getNumKeys(long containerId); /** * Get the container report state to send via HB to SCM. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java index 8c27ba94c4..158ce38efc 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.ozone.container.common.interfaces; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.container.common.helpers.KeyData; import java.io.IOException; @@ -32,11 +32,10 @@ public interface KeyManager { /** * Puts or overwrites a key. * - * @param pipeline - Pipeline. * @param data - Key Data. * @throws IOException */ - void putKey(Pipeline pipeline, KeyData data) throws IOException; + void putKey(KeyData data) throws IOException; /** * Gets an existing key. @@ -50,23 +49,21 @@ public interface KeyManager { /** * Deletes an existing Key. * - * @param pipeline - Pipeline. - * @param keyName Key Data. + * @param blockID - ID of the block. * @throws StorageContainerException */ - void deleteKey(Pipeline pipeline, String keyName) + void deleteKey(BlockID blockID) throws IOException; /** * List keys in a container. * - * @param pipeline - pipeline. - * @param prefix - Prefix in needed. - * @param startKey - Key to start from, EMPTY_STRING to begin. + * @param containerID - ID of the container. + * @param startLocalID - Key to start from, 0 to begin. * @param count - Number of keys to return. * @return List of Keys that match the criteria. */ - List<KeyData> listKey(Pipeline pipeline, String prefix, String startKey, + List<KeyData> listKey(long containerID, long startLocalID, int count) throws IOException; /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java index ac95b2a12c..7c3fa30bd1 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java @@ -180,12 +180,12 @@ public class BlockDeletingService extends BackgroundService{ meta.getSequentialRangeKVs(null, blockLimitPerTask, filter); if (toDeleteBlocks.isEmpty()) { LOG.debug("No under deletion block found in container : {}", - containerData.getContainerName()); + containerData.getContainerID()); } List<String> succeedBlocks = new LinkedList<>(); LOG.debug("Container : {}, To-Delete blocks : {}", - containerData.getContainerName(), toDeleteBlocks.size()); + containerData.getContainerID(), toDeleteBlocks.size()); File dataDir = ContainerUtils.getDataDirectory(containerData).toFile(); if (!dataDir.exists() || !dataDir.isDirectory()) { LOG.error("Invalid container data dir {} : " @@ -220,11 +220,11 @@ public class BlockDeletingService extends BackgroundService{ meta.writeBatch(batch); // update count of pending deletion blocks in in-memory container status containerManager.decrPendingDeletionBlocks(succeedBlocks.size(), - containerData.getContainerName()); + containerData.getContainerID()); if (!succeedBlocks.isEmpty()) { LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms", - containerData.getContainerName(), succeedBlocks.size(), + containerData.getContainerID(), succeedBlocks.size(), Time.monotonicNow() - startTime); } crr.addAll(succeedBlocks); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java index f7b49b7590..d8adc7df0f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java @@ -58,19 +58,20 @@ public class CloseContainerHandler implements CommandHandler { LOG.debug("Processing Close Container command."); invocationCount++; long startTime = Time.monotonicNow(); - String containerName = "UNKNOWN"; + // TODO: define this as INVALID_CONTAINER_ID in HddsConsts.java (TBA) + long containerID = -1; try { SCMCloseContainerCmdResponseProto closeContainerProto = SCMCloseContainerCmdResponseProto .parseFrom(command.getProtoBufMessage()); - containerName = closeContainerProto.getContainerName(); + containerID = closeContainerProto.getContainerID(); - container.getContainerManager().closeContainer(containerName); + container.getContainerManager().closeContainer(containerID); } catch (Exception e) { - LOG.error("Can't close container " + containerName, e); + LOG.error("Can't close container " + containerID, e); } finally { long endTime = Time.monotonicNow(); totalTime += endTime - startTime; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java index f106e3d55f..5231660b6d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java @@ -16,6 +16,7 @@ */ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; +import com.google.common.primitives.Longs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.protocol.proto @@ -108,7 +109,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler { txResultBuilder.setSuccess(true); } catch (IOException e) { LOG.warn("Failed to delete blocks for container={}, TXID={}", - entry.getContainerName(), entry.getTxID(), e); + entry.getContainerID(), entry.getTxID(), e); txResultBuilder.setSuccess(false); } resultBuilder.addResults(txResultBuilder.build()); @@ -150,7 +151,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler { */ private void deleteContainerBlocks(DeletedBlocksTransaction delTX, Configuration config) throws IOException { - String containerId = delTX.getContainerName(); + long containerId = delTX.getContainerID(); ContainerData containerInfo = containerManager.readContainer(containerId); if (LOG.isDebugEnabled()) { LOG.debug("Processing Container : {}, DB path : {}", containerId, @@ -159,9 +160,9 @@ public class DeleteBlocksCommandHandler implements CommandHandler { int newDeletionBlocks = 0; MetadataStore containerDB = KeyUtils.getDB(containerInfo, config); - for (String blk : delTX.getBlockIDList()) { + for (Long blk : delTX.getLocalIDList()) { BatchOperation batch = new BatchOperation(); - byte[] blkBytes = DFSUtil.string2Bytes(blk); + byte[] blkBytes = Longs.toByteArray(blk); byte[] blkInfo = containerDB.get(blkBytes); if (blkInfo != null) { // Found the block in container db, diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java index 5dee10f44b..eba565db64 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java @@ -170,7 +170,7 @@ public class HeartbeatEndpointTask commandResponseProto.getCloseContainerProto()); if (LOG.isDebugEnabled()) { LOG.debug("Received SCM container close request for container {}", - closeContainer.getContainerName()); + closeContainer.getContainerID()); } this.context.addCommand(closeContainer); break; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index 1a89e44bd1..89eaaced03 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -94,7 +94,7 @@ public class ContainerStateMachine extends BaseStateMachine { private ThreadPoolExecutor writeChunkExecutor; private final ConcurrentHashMap<Long, CompletableFuture<Message>> writeChunkFutureMap; - private final ConcurrentHashMap<String, CompletableFuture<Message>> + private final ConcurrentHashMap<Long, CompletableFuture<Message>> createContainerFutureMap; ContainerStateMachine(ContainerDispatcher dispatcher, @@ -146,8 +146,7 @@ public class ContainerStateMachine extends BaseStateMachine { // create the log entry proto final WriteChunkRequestProto commitWriteChunkProto = WriteChunkRequestProto.newBuilder() - .setPipeline(write.getPipeline()) - .setKeyName(write.getKeyName()) + .setBlockID(write.getBlockID()) .setChunkData(write.getChunkData()) // skipping the data field as it is // already set in statemachine data proto @@ -196,9 +195,9 @@ public class ContainerStateMachine extends BaseStateMachine { private CompletableFuture<Message> handleWriteChunk( ContainerCommandRequestProto requestProto, long entryIndex) { final WriteChunkRequestProto write = requestProto.getWriteChunk(); - String containerName = write.getPipeline().getContainerName(); + long containerID = write.getBlockID().getContainerID(); CompletableFuture<Message> future = - createContainerFutureMap.get(containerName); + createContainerFutureMap.get(containerID); CompletableFuture<Message> writeChunkFuture; if (future != null) { writeChunkFuture = future.thenApplyAsync( @@ -213,10 +212,10 @@ public class ContainerStateMachine extends BaseStateMachine { private CompletableFuture<Message> handleCreateContainer( ContainerCommandRequestProto requestProto) { - String containerName = - requestProto.getCreateContainer().getContainerData().getName(); + long containerID = + requestProto.getCreateContainer().getContainerData().getContainerID(); createContainerFutureMap. - computeIfAbsent(containerName, k -> new CompletableFuture<>()); + computeIfAbsent(containerID, k -> new CompletableFuture<>()); return CompletableFuture.completedFuture(() -> ByteString.EMPTY); } @@ -270,9 +269,9 @@ public class ContainerStateMachine extends BaseStateMachine { } else { Message message = runCommand(requestProto); if (cmdType == ContainerProtos.Type.CreateContainer) { - String containerName = - requestProto.getCreateContainer().getContainerData().getName(); - createContainerFutureMap.remove(containerName).complete(message); + long containerID = + requestProto.getCreateContainer().getContainerData().getContainerID(); + createContainerFutureMap.remove(containerID).complete(message); } return CompletableFuture.completedFuture(message); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java index 6ae45b6d08..4d9c690311 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java @@ -69,15 +69,15 @@ public final class ContainerCache extends LRUMap { /** * Closes a db instance. * - * @param container - name of the container to be closed. + * @param containerID - ID of the container to be closed. * @param db - db instance to close. */ - private void closeDB(String container, MetadataStore db) { + private void closeDB(long containerID, MetadataStore db) { if (db != null) { try { db.close(); } catch (IOException e) { - LOG.error("Error closing DB. Container: " + container, e); + LOG.error("Error closing DB. Container: " + containerID, e); } } } @@ -93,7 +93,7 @@ public final class ContainerCache extends LRUMap { while (iterator.hasNext()) { iterator.next(); MetadataStore db = (MetadataStore) iterator.getValue(); - closeDB(iterator.getKey().toString(), db); + closeDB(((Number)iterator.getKey()).longValue(), db); } // reset the cache cache.clear(); @@ -110,7 +110,7 @@ public final class ContainerCache extends LRUMap { lock.lock(); try { MetadataStore db = (MetadataStore) entry.getValue(); - closeDB(entry.getKey().toString(), db); + closeDB(((Number)entry.getKey()).longValue(), db); } finally { lock.unlock(); } @@ -120,28 +120,27 @@ public final class ContainerCache extends LRUMap { /** * Returns a DB handle if available, create the handler otherwise. * - * @param containerName - Name of the container. + * @param containerID - ID of the container. * @return MetadataStore. */ - public MetadataStore getDB(String containerName, String containerDBPath) + public MetadataStore getDB(long containerID, String containerDBPath) throws IOException { - Preconditions.checkNotNull(containerName); - Preconditions.checkState(!containerName.isEmpty()); + Preconditions.checkState(containerID >= 0, "Container ID cannot be negative."); lock.lock(); try { - MetadataStore db = (MetadataStore) this.get(containerName); + MetadataStore db = (MetadataStore) this.get(containerID); if (db == null) { db = MetadataStoreBuilder.newBuilder() .setDbFile(new File(containerDBPath)) .setCreateIfMissing(false) .build(); - this.put(containerName, db); + this.put(containerID, db); } return db; } catch (Exception e) { LOG.error("Error opening DB. Container:{} ContainerPath:{}", - containerName, containerDBPath, e); + containerID, containerDBPath, e); throw e; } finally { lock.unlock(); @@ -151,16 +150,15 @@ public final class ContainerCache extends LRUMap { /** * Remove a DB handler from cache. * - * @param containerName - Name of the container. + * @param containerID - ID of the container. */ - public void removeDB(String containerName) { - Preconditions.checkNotNull(containerName); - Preconditions.checkState(!containerName.isEmpty()); + public void removeDB(long containerID) { + Preconditions.checkState(containerID >= 0, "Container ID cannot be negative."); lock.lock(); try { - MetadataStore db = (MetadataStore)this.get(containerName); - closeDB(containerName, db); - this.remove(containerName); + MetadataStore db = (MetadataStore)this.get(containerID); + closeDB(containerID, db); + this.remove(containerID); } finally { lock.unlock(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java index b1cdbc4913..d1d6488134 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java @@ -32,10 +32,10 @@ import static org.apache.hadoop.hdds.protocol.proto public class CloseContainerCommand extends SCMCommand<SCMCloseContainerCmdResponseProto> { - private String containerName; + private long containerID; - public CloseContainerCommand(String containerName) { - this.containerName = containerName; + public CloseContainerCommand(long containerID) { + this.containerID = containerID; } /** @@ -60,17 +60,17 @@ public class CloseContainerCommand public SCMCloseContainerCmdResponseProto getProto() { return SCMCloseContainerCmdResponseProto.newBuilder() - .setContainerName(containerName).build(); + .setContainerID(containerID).build(); } public static CloseContainerCommand getFromProtobuf( SCMCloseContainerCmdResponseProto closeContainerProto) { Preconditions.checkNotNull(closeContainerProto); - return new CloseContainerCommand(closeContainerProto.getContainerName()); + return new CloseContainerCommand(closeContainerProto.getContainerID()); } - public String getContainerName() { - return containerName; + public long getContainerID() { + return containerID; } } diff --git a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto index 03b85e500a..bc7fb7a7a3 100644 --- a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto +++ b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto @@ -87,7 +87,6 @@ message NodeContianerMapping { A container report contains the following information. */ message ContainerInfo { - required string containerName = 1; optional string finalhash = 2; optional int64 size = 3; optional int64 used = 4; @@ -102,10 +101,12 @@ message ContainerInfo { } // The deleted blocks which are stored in deletedBlock.db of scm. +// We don't use BlockID because this only contians multiple localIDs +// of the same containerID. message DeletedBlocksTransaction { required int64 txID = 1; - required string containerName = 2; - repeated string blockID = 3; + required int64 containerID = 2; + repeated int64 localID = 3; // the retry time of sending deleting command to datanode. required int32 count = 4; } @@ -201,7 +202,7 @@ message SendContainerReportProto { This command asks the datanode to close a specific container. */ message SCMCloseContainerCmdResponseProto { - required string containerName = 1; + required int64 containerID = 1; } /** diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java index 41a8a8012c..764ccfdda7 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java @@ -239,7 +239,7 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol { for (StorageContainerDatanodeProtocolProtos.ContainerInfo report: reports.getReportsList()) { - containers.put(report.getContainerName(), report); + containers.put(report.getContainerID(), report); } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java index 4ab251641c..f9aa0cd4f7 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hdds.scm.block; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.client.BlockID; import java.io.Closeable; import java.io.IOException; @@ -43,14 +43,6 @@ public interface BlockManager extends Closeable { HddsProtos.ReplicationFactor factor, String owner) throws IOException; /** - * Give the key to the block, get the pipeline info. - * @param key - key to the block. - * @return - Pipeline that used to access the block. - * @throws IOException - */ - Pipeline getBlock(String key) throws IOException; - - /** * Deletes a list of blocks in an atomic operation. Internally, SCM * writes these blocks into a {@link DeletedBlockLog} and deletes them * from SCM DB. If this is successful, given blocks are entering pending @@ -60,7 +52,7 @@ public interface BlockManager extends Closeable { * a particular object key. * @throws IOException if exception happens, non of the blocks is deleted. */ - void deleteBlocks(List<String> blockIDs) throws IOException; + void deleteBlocks(List<BlockID> blockIDs) throws IOException; /** * @return the block deletion transaction log maintained by SCM. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java index d9661124f5..5a98e8537f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java @@ -16,30 +16,25 @@ */ package org.apache.hadoop.hdds.scm.block; -import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.Mapping; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.utils.BatchOperation; -import org.apache.hadoop.utils.MetadataStore; -import org.apache.hadoop.utils.MetadataStoreBuilder; +import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.management.ObjectName; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -54,10 +49,7 @@ import java.util.concurrent.locks.ReentrantLock; import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes .CHILL_MODE_EXCEPTION; import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes - .FAILED_TO_FIND_BLOCK; -import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes .INVALID_BLOCK_SIZE; -import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath; import static org.apache.hadoop.ozone.OzoneConfigKeys .OZONE_BLOCK_DELETING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.OzoneConfigKeys @@ -66,7 +58,6 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys .OZONE_BLOCK_DELETING_SERVICE_TIMEOUT; import static org.apache.hadoop.ozone.OzoneConfigKeys .OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT; -import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB; /** Block Manager manages the block access for SCM. */ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { @@ -78,11 +69,9 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { private final NodeManager nodeManager; private final Mapping containerManager; - private final MetadataStore blockStore; private final Lock lock; private final long containerSize; - private final long cacheSize; private final DeletedBlockLog deletedBlockLog; private final SCMBlockDeletingService blockDeletingService; @@ -97,30 +86,17 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { * @param conf - configuration. * @param nodeManager - node manager. * @param containerManager - container manager. - * @param cacheSizeMB - cache size for level db store. * @throws IOException */ public BlockManagerImpl(final Configuration conf, - final NodeManager nodeManager, final Mapping containerManager, - final int cacheSizeMB) throws IOException { + final NodeManager nodeManager, final Mapping containerManager) + throws IOException { this.nodeManager = nodeManager; this.containerManager = containerManager; - this.cacheSize = cacheSizeMB; this.containerSize = OzoneConsts.GB * conf.getInt( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT); - File metaDir = getOzoneMetaDirPath(conf); - String scmMetaDataDir = metaDir.getPath(); - - // Write the block key to container name mapping. - File blockContainerDbPath = new File(scmMetaDataDir, BLOCK_DB); - blockStore = - MetadataStoreBuilder.newBuilder() - .setConf(conf) - .setDbFile(blockContainerDbPath) - .setCacheSize(this.cacheSize * OzoneConsts.MB) - .build(); this.containerProvisionBatchSize = conf.getInt( @@ -181,12 +157,11 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { lock.lock(); try { for (int i = 0; i < count; i++) { - String containerName = UUID.randomUUID().toString(); ContainerInfo containerInfo = null; try { // TODO: Fix this later when Ratis is made the Default. containerInfo = containerManager.allocateContainer(type, factor, - containerName, owner); + owner); if (containerInfo == null) { LOG.warn("Unable to allocate container."); @@ -267,7 +242,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { size, owner, type, factor, HddsProtos.LifeCycleState .ALLOCATED); if (containerInfo != null) { - containerManager.updateContainerState(containerInfo.getContainerName(), + containerManager.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED); } @@ -297,7 +272,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { size, owner, type, factor, HddsProtos.LifeCycleState .ALLOCATED); if (containerInfo != null) { - containerManager.updateContainerState(containerInfo.getContainerName(), + containerManager.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED); } @@ -327,68 +302,27 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { ContainerInfo containerInfo, HddsProtos.LifeCycleState state) throws IOException { - // TODO : Replace this with Block ID. - String blockKey = UUID.randomUUID().toString(); - boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED); - - AllocatedBlock.Builder abb = - new AllocatedBlock.Builder() - .setKey(blockKey) - // TODO : Use containerinfo instead of pipeline. - .setPipeline(containerInfo.getPipeline()) - .setShouldCreateContainer(createContainer); - LOG.trace("New block allocated : {} Container ID: {}", blockKey, - containerInfo.toString()); - if (containerInfo.getPipeline().getMachines().size() == 0) { LOG.error("Pipeline Machine count is zero."); return null; } - // Persist this block info to the blockStore DB, so getBlock(key) can - // find which container the block lives. - // TODO : Remove this DB in future - // and make this a KSM operation. Category: SCALABILITY. - if (containerInfo.getPipeline().getMachines().size() > 0) { - blockStore.put( - DFSUtil.string2Bytes(blockKey), - DFSUtil.string2Bytes(containerInfo.getPipeline().getContainerName())); - } - return abb.build(); - } + // 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 containerID = containerInfo.getContainerID(); - /** - * Given a block key, return the Pipeline information. - * - * @param key - block key assigned by SCM. - * @return Pipeline (list of DNs and leader) to access the block. - * @throws IOException - */ - @Override - public Pipeline getBlock(final String key) throws IOException { - lock.lock(); - try { - byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key)); - if (containerBytes == null) { - throw new SCMException( - "Specified block key does not exist. key : " + key, - FAILED_TO_FIND_BLOCK); - } + boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED); - String containerName = DFSUtil.bytes2String(containerBytes); - ContainerInfo containerInfo = containerManager.getContainer( - containerName); - if (containerInfo == null) { - LOG.debug("Container {} allocated by block service" - + "can't be found in SCM", containerName); - throw new SCMException( - "Unable to find container for the block", - SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); - } - return containerInfo.getPipeline(); - } finally { - lock.unlock(); - } + AllocatedBlock.Builder abb = + new AllocatedBlock.Builder() + .setBlockID(new BlockID(containerID, localID)) + .setPipeline(containerInfo.getPipeline()) + .setShouldCreateContainer(createContainer); + LOG.trace("New block allocated : {} Container ID: {}", localID, + containerID); + return abb.build(); } /** @@ -403,40 +337,28 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { * @throws IOException if exception happens, non of the blocks is deleted. */ @Override - public void deleteBlocks(List<String> blockIDs) throws IOException { + public void deleteBlocks(List<BlockID> blockIDs) throws IOException { if (!nodeManager.isOutOfChillMode()) { throw new SCMException("Unable to delete block while in chill mode", CHILL_MODE_EXCEPTION); } lock.lock(); - LOG.info("Deleting blocks {}", String.join(",", blockIDs)); - Map<String, List<String>> containerBlocks = new HashMap<>(); - BatchOperation batch = new BatchOperation(); - BatchOperation rollbackBatch = new BatchOperation(); + LOG.info("Deleting blocks {}", StringUtils.join(",", blockIDs)); + Map<Long, List<Long>> containerBlocks = new HashMap<>(); // TODO: track the block size info so that we can reclaim the container // TODO: used space when the block is deleted. try { - for (String blockKey : blockIDs) { - byte[] blockKeyBytes = DFSUtil.string2Bytes(blockKey); - byte[] containerBytes = blockStore.get(blockKeyBytes); - if (containerBytes == null) { - throw new SCMException( - "Specified block key does not exist. key : " + blockKey, - FAILED_TO_FIND_BLOCK); - } - batch.delete(blockKeyBytes); - rollbackBatch.put(blockKeyBytes, containerBytes); - + for (BlockID block : blockIDs) { // Merge blocks to a container to blocks mapping, // prepare to persist this info to the deletedBlocksLog. - String containerName = DFSUtil.bytes2String(containerBytes); - if (containerBlocks.containsKey(containerName)) { - containerBlocks.get(containerName).add(blockKey); + long containerID = block.getContainerID(); + if (containerBlocks.containsKey(containerID)) { + containerBlocks.get(containerID).add(block.getLocalID()); } else { - List<String> item = new ArrayList<>(); - item.add(blockKey); - containerBlocks.put(containerName, item); + List<Long> item = new ArrayList<>(); + item.add(block.getLocalID()); + containerBlocks.put(containerID, item); } } @@ -445,34 +367,13 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { // removed. If we write the log first, once log is written, the // async deleting service will start to scan and might be picking // up some blocks to do real deletions, that might cause data loss. - blockStore.writeBatch(batch); try { deletedBlockLog.addTransactions(containerBlocks); } catch (IOException e) { - try { - // If delLog update is failed, we need to rollback the changes. - blockStore.writeBatch(rollbackBatch); - } catch (IOException rollbackException) { - // This is a corner case. AddTX fails and rollback also fails, - // this will leave these blocks in inconsistent state. They were - // moved to pending deletion state in SCM DB but were not written - // into delLog so real deletions would not be done. Blocks become - // to be invisible from namespace but actual data are not removed. - // We log an error here so admin can manually check and fix such - // errors. - LOG.error( - "Blocks might be in inconsistent state because" - + " they were moved to pending deletion state in SCM DB but" - + " not written into delLog. Admin can manually add them" - + " into delLog for deletions. Inconsistent block list: {}", - String.join(",", blockIDs), - e); - throw rollbackException; - } throw new IOException( "Skip writing the deleted blocks info to" + " the delLog because addTransaction fails. Batch skipped: " - + String.join(",", blockIDs), + + StringUtils.join(",", blockIDs), e); } // TODO: Container report handling of the deleted blocks: @@ -488,11 +389,6 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { return this.deletedBlockLog; } - @VisibleForTesting - public String getDeletedKeyName(String key) { - return StringUtils.format(".Deleted/%s", key); - } - /** * Close the resources for BlockManager. * @@ -500,9 +396,6 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { */ @Override public void close() throws IOException { - if (blockStore != null) { - blockStore.close(); - } if (deletedBlockLog != null) { deletedBlockLog.close(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java index 47074d28ec..32290cc99b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java @@ -56,7 +56,7 @@ public class DatanodeDeletedBlockTransactions { public void addTransaction(DeletedBlocksTransaction tx) throws IOException { ContainerInfo info = null; try { - info = mappingService.getContainer(tx.getContainerName()); + info = mappingService.getContainer(tx.getContainerID()); } catch (IOException e) { SCMBlockDeletingService.LOG.warn("Got container info error.", e); } @@ -64,7 +64,7 @@ public class DatanodeDeletedBlockTransactions { if (info == null) { SCMBlockDeletingService.LOG.warn( "Container {} not found, continue to process next", - tx.getContainerName()); + tx.getContainerID()); return; } @@ -75,7 +75,7 @@ public class DatanodeDeletedBlockTransactions { if (txs != null && txs.size() < maximumAllowedTXNum) { boolean hasContained = false; for (DeletedBlocksTransaction t : txs) { - if (t.getContainerName().equals(tx.getContainerName())) { + if (t.getContainerID() == tx.getContainerID()) { hasContained = true; break; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java index f7b770eca0..cc32b35a4d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java @@ -89,12 +89,12 @@ public interface DeletedBlockLog extends Closeable { /** * Creates a block deletion transaction and adds that into the log. * - * @param containerName - container name. + * @param containerID - container ID. * @param blocks - blocks that belong to the same container. * * @throws IOException */ - void addTransaction(String containerName, List<String> blocks) + void addTransaction(long containerID, List<Long> blocks) throws IOException; /** @@ -110,7 +110,7 @@ public interface DeletedBlockLog extends Closeable { * @param containerBlocksMap a map of containerBlocks. * @throws IOException */ - void addTransactions(Map<String, List<String>> containerBlocksMap) + void addTransactions(Map<Long, List<Long>> containerBlocksMap) throws IOException; /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java index 0f4988afa3..cabcb46e1d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java @@ -190,8 +190,14 @@ public class DeletedBlockLogImpl implements DeletedBlockLog { try { for(Long txID : txIDs) { try { + byte [] deleteBlockBytes = + deletedStore.get(Longs.toByteArray(txID)); + if (deleteBlockBytes == null) { + LOG.warn("Delete txID {} not found", txID); + continue; + } DeletedBlocksTransaction block = DeletedBlocksTransaction - .parseFrom(deletedStore.get(Longs.toByteArray(txID))); + .parseFrom(deleteBlockBytes); DeletedBlocksTransaction.Builder builder = block.toBuilder(); int currentCount = block.getCount(); if (currentCount > -1) { @@ -216,11 +222,11 @@ public class DeletedBlockLogImpl implements DeletedBlockLog { } private DeletedBlocksTransaction constructNewTransaction(long txID, - String containerName, List<String> blocks) { + long containerID, List<Long> blocks) { return DeletedBlocksTransaction.newBuilder() .setTxID(txID) - .setContainerName(containerName) - .addAllBlockID(blocks) + .setContainerID(containerID) + .addAllLocalID(blocks) .setCount(0) .build(); } @@ -250,18 +256,18 @@ public class DeletedBlockLogImpl implements DeletedBlockLog { /** * {@inheritDoc} * - * @param containerName - container name. + * @param containerID - container ID. * @param blocks - blocks that belong to the same container. * @throws IOException */ @Override - public void addTransaction(String containerName, List<String> blocks) + public void addTransaction(long containerID, List<Long> blocks) throws IOException { BatchOperation batch = new BatchOperation(); lock.lock(); try { DeletedBlocksTransaction tx = constructNewTransaction(lastTxID + 1, - containerName, blocks); + containerID, blocks); byte[] key = Longs.toByteArray(lastTxID + 1); batch.put(key, tx.toByteArray()); @@ -303,13 +309,13 @@ public class DeletedBlockLogImpl implements DeletedBlockLog { * @throws IOException */ @Override - public void addTransactions(Map<String, List<String>> containerBlocksMap) + public void addTransactions(Map<Long, List<Long>> containerBlocksMap) throws IOException { BatchOperation batch = new BatchOperation(); lock.lock(); try { long currentLatestID = lastTxID; - for (Map.Entry<String, List<String>> entry : + for (Map.Entry<Long, List<Long>> entry : containerBlocksMap.entrySet()) { currentLatestID += 1; byte[] key = Longs.toByteArray(currentLatestID); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java index 63cb3a3c1f..e569874aaf 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.container; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.primitives.Longs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser; @@ -26,7 +27,6 @@ import org.apache.hadoop.hdds.scm.container.replication.ContainerSupervisor; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector; -import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; @@ -38,8 +38,6 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.lease.Lease; import org.apache.hadoop.ozone.lease.LeaseException; import org.apache.hadoop.ozone.lease.LeaseManager; -import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter; -import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter; import org.apache.hadoop.utils.MetadataStore; import org.apache.hadoop.utils.MetadataStoreBuilder; import org.slf4j.Logger; @@ -149,16 +147,15 @@ public class ContainerMapping implements Mapping { * {@inheritDoc} */ @Override - public ContainerInfo getContainer(final String containerName) throws + public ContainerInfo getContainer(final long containerID) throws IOException { ContainerInfo containerInfo; lock.lock(); try { - byte[] containerBytes = containerStore.get(containerName.getBytes( - encoding)); + byte[] containerBytes = containerStore.get(Longs.toByteArray(containerID)); if (containerBytes == null) { throw new SCMException( - "Specified key does not exist. key : " + containerName, + "Specified key does not exist. key : " + containerID, SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); } @@ -175,19 +172,18 @@ public class ContainerMapping implements Mapping { * {@inheritDoc} */ @Override - public List<ContainerInfo> listContainer(String startName, - String prefixName, int count) throws IOException { + public List<ContainerInfo> listContainer(long startContainerID, + int count) throws IOException { List<ContainerInfo> containerList = new ArrayList<>(); lock.lock(); try { if (containerStore.isEmpty()) { throw new IOException("No container exists in current db"); } - MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefixName); - byte[] startKey = startName == null ? null : DFSUtil.string2Bytes( - startName); + byte[] startKey = startContainerID <= 0 ? null : + Longs.toByteArray(startContainerID); List<Map.Entry<byte[], byte[]>> range = - containerStore.getSequentialRangeKVs(startKey, count, prefixFilter); + containerStore.getSequentialRangeKVs(startKey, count, null); // Transform the values into the pipelines. // TODO: filter by container state @@ -209,7 +205,6 @@ public class ContainerMapping implements Mapping { * Allocates a new container. * * @param replicationFactor - replication factor of the container. - * @param containerName - Name of the container. * @param owner - The string name of the Service that owns this container. * @return - Pipeline that makes up this container. * @throws IOException - Exception @@ -218,11 +213,8 @@ public class ContainerMapping implements Mapping { public ContainerInfo allocateContainer( ReplicationType type, ReplicationFactor replicationFactor, - final String containerName, String owner) throws IOException { - Preconditions.checkNotNull(containerName); - Preconditions.checkState(!containerName.isEmpty()); ContainerInfo containerInfo; if (!nodeManager.isOutOfChillMode()) { @@ -233,19 +225,12 @@ public class ContainerMapping implements Mapping { lock.lock(); try { - byte[] containerBytes = containerStore.get(containerName.getBytes( - encoding)); - if (containerBytes != null) { - throw new SCMException( - "Specified container already exists. key : " + containerName, - SCMException.ResultCodes.CONTAINER_EXISTS); - } containerInfo = containerStateManager.allocateContainer( - pipelineSelector, type, replicationFactor, containerName, - owner); - containerStore.put( - containerName.getBytes(encoding), containerInfo.getProtobuf() + pipelineSelector, type, replicationFactor, owner); + + byte[] containerIDBytes = Longs.toByteArray(containerInfo.getContainerID()); + containerStore.put(containerIDBytes, containerInfo.getProtobuf() .toByteArray()); } finally { lock.unlock(); @@ -256,20 +241,20 @@ public class ContainerMapping implements Mapping { /** * Deletes a container from SCM. * - * @param containerName - Container name + * @param containerID - Container ID * @throws IOException if container doesn't exist or container store failed * to delete the * specified key. */ @Override - public void deleteContainer(String containerName) throws IOException { + public void deleteContainer(long containerID) throws IOException { lock.lock(); try { - byte[] dbKey = containerName.getBytes(encoding); + byte[] dbKey = Longs.toByteArray(containerID); byte[] containerBytes = containerStore.get(dbKey); if (containerBytes == null) { throw new SCMException( - "Failed to delete container " + containerName + ", reason : " + + "Failed to delete container " + containerID + ", reason : " + "container doesn't exist.", SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); } @@ -284,17 +269,17 @@ public class ContainerMapping implements Mapping { */ @Override public HddsProtos.LifeCycleState updateContainerState( - String containerName, HddsProtos.LifeCycleEvent event) throws + long containerID, HddsProtos.LifeCycleEvent event) throws IOException { ContainerInfo containerInfo; lock.lock(); try { - byte[] dbKey = containerName.getBytes(encoding); + byte[] dbKey = Longs.toByteArray(containerID); byte[] containerBytes = containerStore.get(dbKey); if (containerBytes == null) { throw new SCMException( "Failed to update container state" - + containerName + + containerID + ", reason : container doesn't exist.", SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); } @@ -310,7 +295,7 @@ public class ContainerMapping implements Mapping { containerLeaseManager.acquire(containerInfo); // Register callback to be executed in case of timeout containerLease.registerCallBack(() -> { - updateContainerState(containerName, + updateContainerState(containerID, HddsProtos.LifeCycleEvent.TIMEOUT); return null; }); @@ -388,7 +373,7 @@ public class ContainerMapping implements Mapping { containerSupervisor.handleContainerReport(reports); for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState : containerInfos) { - byte[] dbKey = datanodeState.getContainerNameBytes().toByteArray(); + byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID()); lock.lock(); try { byte[] containerBytes = containerStore.get(dbKey); @@ -409,14 +394,14 @@ public class ContainerMapping implements Mapping { // If the container is closed, then state is already written to SCM // DB.TODO: So can we can write only once to DB. if (closeContainerIfNeeded(newState)) { - LOG.info("Closing the Container: {}", newState.getContainerName()); + LOG.info("Closing the Container: {}", newState.getContainerID()); } } else { // Container not found in our container db. LOG.error("Error while processing container report from datanode :" + " {}, for container: {}, reason: container doesn't exist in" + "container database.", reports.getDatanodeDetails(), - datanodeState.getContainerName()); + datanodeState.getContainerID()); } } finally { lock.unlock(); @@ -436,7 +421,7 @@ public class ContainerMapping implements Mapping { HddsProtos.SCMContainerInfo knownState) { HddsProtos.SCMContainerInfo.Builder builder = HddsProtos.SCMContainerInfo.newBuilder(); - builder.setContainerName(knownState.getContainerName()); + builder.setContainerID(knownState.getContainerID()); builder.setPipeline(knownState.getPipeline()); // If used size is greater than allocated size, we will be updating // allocated size with used size. This update is done as a fallback @@ -473,7 +458,7 @@ public class ContainerMapping implements Mapping { float containerUsedPercentage = 1.0f * newState.getUsedBytes() / this.size; - ContainerInfo scmInfo = getContainer(newState.getContainerName()); + ContainerInfo scmInfo = getContainer(newState.getContainerID()); if (containerUsedPercentage >= containerCloseThreshold && !isClosed(scmInfo)) { // We will call closer till get to the closed state. @@ -488,13 +473,13 @@ public class ContainerMapping implements Mapping { // closed state from container reports. This state change should be // invoked once and only once. HddsProtos.LifeCycleState state = updateContainerState( - scmInfo.getContainerName(), + scmInfo.getContainerID(), HddsProtos.LifeCycleEvent.FINALIZE); if (state != HddsProtos.LifeCycleState.CLOSING) { LOG.error("Failed to close container {}, reason : Not able " + "to " + "update container state, current container state: {}.", - newState.getContainerName(), state); + newState.getContainerID(), state); return false; } return true; @@ -561,11 +546,11 @@ public class ContainerMapping implements Mapping { @VisibleForTesting public void flushContainerInfo() throws IOException { List<ContainerInfo> containers = containerStateManager.getAllContainers(); - List<String> failedContainers = new ArrayList<>(); + List<Long> failedContainers = new ArrayList<>(); for (ContainerInfo info : containers) { // even if some container updated failed, others can still proceed try { - byte[] dbKey = info.getContainerName().getBytes(encoding); + byte[] dbKey = Longs.toByteArray(info.getContainerID()); byte[] containerBytes = containerStore.get(dbKey); // TODO : looks like when a container is deleted, the container is // removed from containerStore but not containerStateManager, so it can @@ -577,7 +562,6 @@ public class ContainerMapping implements Mapping { ContainerInfo oldInfo = ContainerInfo.fromProtobuf(oldInfoProto); ContainerInfo newInfo = new ContainerInfo.Builder() .setAllocatedBytes(info.getAllocatedBytes()) - .setContainerName(oldInfo.getContainerName()) .setNumberOfKeys(oldInfo.getNumberOfKeys()) .setOwner(oldInfo.getOwner()) .setPipeline(oldInfo.getPipeline()) @@ -588,10 +572,10 @@ public class ContainerMapping implements Mapping { } else { LOG.debug("Container state manager has container {} but not found " + "in container store, a deleted container?", - info.getContainerName()); + info.getContainerID()); } } catch (IOException ioe) { - failedContainers.add(info.getContainerName()); + failedContainers.add(info.getContainerID()); } } if (!failedContainers.isEmpty()) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java index 227eca04e8..f11a50cf71 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java @@ -157,8 +157,7 @@ public class ContainerStateManager implements Closeable { List<ContainerInfo> containerList; try { - containerList = containerMapping.listContainer(null, - null, Integer.MAX_VALUE); + containerList = containerMapping.listContainer(0, Integer.MAX_VALUE); // if there are no container to load, let us return. if (containerList == null || containerList.size() == 0) { @@ -280,24 +279,21 @@ public class ContainerStateManager implements Closeable { * @param selector -- Pipeline selector class. * @param type -- Replication type. * @param replicationFactor - Replication replicationFactor. - * @param containerName - Container Name. * @return Container Info. * @throws IOException on Failure. */ public ContainerInfo allocateContainer(PipelineSelector selector, HddsProtos .ReplicationType type, HddsProtos.ReplicationFactor replicationFactor, - final String containerName, String owner) throws - IOException { + String owner) throws IOException { Pipeline pipeline = selector.getReplicationPipeline(type, - replicationFactor, containerName); + replicationFactor); Preconditions.checkNotNull(pipeline, "Pipeline type=%s/" + "replication=%s couldn't be found for the new container. " + "Do you have enough nodes?", type, replicationFactor); ContainerInfo containerInfo = new ContainerInfo.Builder() - .setContainerName(containerName) .setState(HddsProtos.LifeCycleState.ALLOCATED) .setPipeline(pipeline) // This is bytes allocated for blocks inside container, not the @@ -332,7 +328,7 @@ public class ContainerStateManager implements Closeable { String error = String.format("Failed to update container state %s, " + "reason: invalid state transition from state: %s upon " + "event: %s.", - info.getPipeline().getContainerName(), info.getState(), event); + info.getContainerID(), info.getState(), event); LOG.error(error); throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java index c949c6c4b5..61dee2b35d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java @@ -31,62 +31,57 @@ import java.util.List; */ public interface Mapping extends Closeable { /** - * Returns the ContainerInfo from the container name. + * Returns the ContainerInfo from the container ID. * - * @param containerName - Name + * @param containerID - ID of container. * @return - ContainerInfo such as creation state and the pipeline. * @throws IOException */ - ContainerInfo getContainer(String containerName) throws IOException; + ContainerInfo getContainer(long containerID) throws IOException; /** * Returns containers under certain conditions. - * Search container names from start name(exclusive), - * and use prefix name to filter the result. The max - * size of the searching range cannot exceed the + * Search container IDs from start ID(exclusive), + * The max size of the searching range cannot exceed the * value of count. * - * @param startName start name, if null, start searching at the head. - * @param prefixName prefix name, if null, then filter is disabled. - * @param count count, if count < 0, the max size is unlimited.( + * @param startContainerID start containerID, >=0, start searching at the head if 0. + * @param count count must be >= 0 * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) + * value instead of being unlimited in case the db is very big. * * @return a list of container. * @throws IOException */ - List<ContainerInfo> listContainer(String startName, String prefixName, - int count) throws IOException; + List<ContainerInfo> listContainer(long startContainerID, int count) throws IOException; /** * Allocates a new container for a given keyName and replication factor. * * @param replicationFactor - replication factor of the container. - * @param containerName - Name. * @param owner * @return - Container Info. * @throws IOException */ ContainerInfo allocateContainer(HddsProtos.ReplicationType type, - HddsProtos.ReplicationFactor replicationFactor, - String containerName, String owner) throws IOException; + HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException; /** * Deletes a container from SCM. * - * @param containerName - Container Name + * @param containerID - Container ID * @throws IOException */ - void deleteContainer(String containerName) throws IOException; + void deleteContainer(long containerID) throws IOException; /** * Update container state. - * @param containerName - Container Name + * @param containerID - Container ID * @param event - container life cycle event * @return - new container state * @throws IOException */ - HddsProtos.LifeCycleState updateContainerState(String containerName, + HddsProtos.LifeCycleState updateContainerState(long containerID, HddsProtos.LifeCycleEvent event) throws IOException; /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java index b5d4da9ed1..75ec8e103c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java @@ -51,7 +51,7 @@ public class ContainerCloser { private static final long MULTIPLIER = 3L; private static final int CLEANUP_WATER_MARK = 1000; private final NodeManager nodeManager; - private final Map<String, Long> commandIssued; + private final Map<Long, Long> commandIssued; private final Configuration configuration; private final AtomicInteger mapCount; private final long reportInterval; @@ -93,12 +93,12 @@ public class ContainerCloser { */ public void close(HddsProtos.SCMContainerInfo info) { - if (commandIssued.containsKey(info.getContainerName())) { + if (commandIssued.containsKey(info.getContainerID())) { // We check if we issued a close command in last 3 * reportInterval secs. - long commandQueueTime = commandIssued.get(info.getContainerName()); + long commandQueueTime = commandIssued.get(info.getContainerID()); long currentTime = TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow()); if (currentTime > commandQueueTime + (MULTIPLIER * reportInterval)) { - commandIssued.remove(info.getContainerName()); + commandIssued.remove(info.getContainerID()); mapCount.decrementAndGet(); } else { // Ignore this request, since we just issued a close command. We @@ -131,10 +131,10 @@ public class ContainerCloser { pipeline.getPipelineChannel().getMembersList()) { nodeManager.addDatanodeCommand( DatanodeDetails.getFromProtoBuf(datanodeDetails).getUuid(), - new CloseContainerCommand(info.getContainerName())); + new CloseContainerCommand(info.getContainerID())); } - if (!commandIssued.containsKey(info.getContainerName())) { - commandIssued.put(info.getContainerName(), + if (!commandIssued.containsKey(info.getContainerID())) { + commandIssued.put(info.getContainerID(), TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow())); mapCount.incrementAndGet(); } @@ -150,7 +150,7 @@ public class ContainerCloser { Runnable entryCleaner = () -> { LOG.debug("Starting close container Hash map cleaner."); try { - for (Map.Entry<String, Long> entry : commandIssued.entrySet()) { + for (Map.Entry<Long, Long> entry : commandIssued.entrySet()) { long commandQueueTime = entry.getValue(); if (commandQueueTime + (MULTIPLIER * reportInterval) > TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow())) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java index ddbd213402..af878bf97f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java @@ -61,7 +61,7 @@ public final class InProgressPool { private final NodeManager nodeManager; private final NodePoolManager poolManager; private final ExecutorService executorService; - private final Map<String, Integer> containerCountMap; + private final Map<Long, Integer> containerCountMap; private final Map<UUID, Boolean> processedNodeSet; private final long startTime; private ProgressStatus status; @@ -258,12 +258,12 @@ public final class InProgressPool { for (ContainerInfo info : reports.getReportsList()) { containerProcessedCount.incrementAndGet(); LOG.debug("Total Containers processed: {} Container Name: {}", - containerProcessedCount.get(), info.getContainerName()); + containerProcessedCount.get(), info.getContainerID()); // Update the container map with count + 1 if the key exists or // update the map with 1. Since this is a concurrentMap the // computation and update is atomic. - containerCountMap.merge(info.getContainerName(), 1, Integer::sum); + containerCountMap.merge(info.getContainerID(), 1, Integer::sum); } } }; @@ -275,8 +275,8 @@ public final class InProgressPool { * @param predicate -- Predicate to filter by * @return A list of map entries. */ - public List<Map.Entry<String, Integer>> filterContainer( - Predicate<Map.Entry<String, Integer>> predicate) { + public List<Map.Entry<Long, Integer>> filterContainer( + Predicate<Map.Entry<Long, Integer>> predicate) { return containerCountMap.entrySet().stream() .filter(predicate).collect(Collectors.toList()); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodePoolManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodePoolManager.java index a4a6c51cdf..faf330ea1d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodePoolManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodePoolManager.java @@ -248,7 +248,7 @@ public final class SCMNodePoolManager implements NodePoolManager { throws SCMException { Preconditions.checkNotNull(datanodeDetails, "node is null"); try { - byte[] result = nodePoolStore.get( + byte[] result = nodePoolStore.get( datanodeDetails.getProtoBufMessage().toByteArray()); return result == null ? null : DFSUtil.bytes2String(result); } catch (IOException e) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java index 8e43528914..832fcc669a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java @@ -50,11 +50,10 @@ public abstract class PipelineManager { * needed and based on the replication type in the request appropriate * Interface is invoked. * - * @param containerName Name of the container * @param replicationFactor - Replication Factor * @return a Pipeline. */ - public synchronized final Pipeline getPipeline(String containerName, + public synchronized final Pipeline getPipeline( ReplicationFactor replicationFactor, ReplicationType replicationType) throws IOException { /** @@ -74,15 +73,17 @@ public abstract class PipelineManager { PipelineChannel pipelineChannel = allocatePipelineChannel(replicationFactor); if (pipelineChannel != null) { - LOG.debug("created new pipelineChannel:{} for container:{}", - pipelineChannel.getName(), containerName); + LOG.debug("created new pipelineChannel:{} for container with " + + "replicationType:{} replicationFactor:{}", + pipelineChannel.getName(), replicationType, replicationFactor); activePipelineChannels.add(pipelineChannel); } else { pipelineChannel = findOpenPipelineChannel(replicationType, replicationFactor); if (pipelineChannel != null) { - LOG.debug("re-used pipelineChannel:{} for container:{}", - pipelineChannel.getName(), containerName); + LOG.debug("re-used pipelineChannel:{} for container with " + + "replicationType:{} replicationFactor:{}", + pipelineChannel.getName(), replicationType, replicationFactor); } } if (pipelineChannel == null) { @@ -90,7 +91,7 @@ public abstract class PipelineManager { "free nodes or operational pipelineChannel."); return null; } else { - return new Pipeline(containerName, pipelineChannel); + return new Pipeline(pipelineChannel); } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java index f0c9eea441..d29bb84c78 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java @@ -166,14 +166,14 @@ public class PipelineSelector { */ public Pipeline getReplicationPipeline(ReplicationType replicationType, - HddsProtos.ReplicationFactor replicationFactor, String containerName) + HddsProtos.ReplicationFactor replicationFactor) throws IOException { PipelineManager manager = getPipelineManager(replicationType); Preconditions.checkNotNull(manager, "Found invalid pipeline manager"); - LOG.debug("Getting replication pipeline for {} : Replication {}", - containerName, replicationFactor.toString()); + LOG.debug("Getting replication pipeline forReplicationType {} : ReplicationFactor {}", + replicationType.toString(), replicationFactor.toString()); return manager. - getPipeline(containerName, replicationFactor, replicationType); + getPipeline(replicationFactor, replicationType); } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java index 089a1374a4..70489b9253 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java @@ -95,7 +95,7 @@ public class RatisManagerImpl extends PipelineManager { PipelineSelector.newPipelineFromNodes(newNodesList, LifeCycleState.OPEN, ReplicationType.RATIS, factor, conduitName); Pipeline pipeline = - new Pipeline("setup", pipelineChannel); + new Pipeline(pipelineChannel); try (XceiverClientRatis client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf)) { client.createPipeline(pipeline.getPipelineName(), newNodesList); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java index e0560a1c03..98fe9a1137 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB; @@ -37,6 +36,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ozone.common.BlockGroup; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; import org.apache.hadoop.ozone.protocolPB .ScmBlockLocationProtocolServerSideTranslatorPB; @@ -46,9 +46,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; -import java.util.Set; import static org.apache.hadoop.hdds.scm.ScmConfigKeys .OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY; @@ -139,20 +137,6 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol { } @Override - public Set<AllocatedBlock> getBlockLocations(Set<String> keys) throws - IOException { - Set<AllocatedBlock> locatedBlocks = new HashSet<>(); - for (String key : keys) { - Pipeline pipeline = scm.getScmBlockManager().getBlock(key); - AllocatedBlock block = new AllocatedBlock.Builder().setKey(key) - .setPipeline(pipeline).build(); - locatedBlocks.add(block); - } - return locatedBlocks; - - } - - @Override public AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor, String owner) throws IOException { @@ -202,7 +186,7 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol { .Result.unknownFailure; } List<DeleteBlockResult> blockResultList = new ArrayList<>(); - for (String blockKey : keyBlocks.getBlockIDList()) { + for (BlockID blockKey : keyBlocks.getBlockIDList()) { blockResultList.add(new DeleteBlockResult(blockKey, resultCode)); } results.add(new DeleteBlockGroupResult(keyBlocks.getGroupID(), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index 42cce2f955..246d053abc 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -21,6 +21,7 @@ */ package org.apache.hadoop.hdds.scm.server; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.protobuf.BlockingService; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -137,32 +138,31 @@ public class SCMClientProtocolServer implements } @Override - public Pipeline allocateContainer(HddsProtos.ReplicationType - replicationType, HddsProtos.ReplicationFactor factor, String - containerName, String owner) throws IOException { - scm.checkAdminAccess(); + public ContainerInfo allocateContainer(HddsProtos.ReplicationType + replicationType, HddsProtos.ReplicationFactor factor, + String owner) throws IOException { + getScm().checkAdminAccess(); return scm.getScmContainerManager() - .allocateContainer(replicationType, factor, containerName, owner) - .getPipeline(); + .allocateContainer(replicationType, factor, owner); } @Override - public Pipeline getContainer(String containerName) throws IOException { + public ContainerInfo getContainer(long containerID) throws IOException { return scm.getScmContainerManager() - .getContainer(containerName).getPipeline(); + .getContainer(containerID); } @Override - public List<ContainerInfo> listContainer(String startName, - String prefixName, int count) throws IOException { - return scm.getScmContainerManager() - .listContainer(startName, prefixName, count); + public List<ContainerInfo> listContainer(long startContainerID, + int count) throws IOException { + return scm.getScmContainerManager(). + listContainer(startContainerID, count); } @Override - public void deleteContainer(String containerName) throws IOException { - scm.checkAdminAccess(); - scm.getScmContainerManager().deleteContainer(containerName); + public void deleteContainer(long containerID) throws IOException { + getScm().checkAdminAccess(); + scm.getScmContainerManager().deleteContainer(containerID); } @@ -193,12 +193,12 @@ public class SCMClientProtocolServer implements @Override public void notifyObjectStageChange(StorageContainerLocationProtocolProtos - .ObjectStageChangeRequestProto.Type type, String name, + .ObjectStageChangeRequestProto.Type type, long id, StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto.Op op, StorageContainerLocationProtocolProtos .ObjectStageChangeRequestProto.Stage stage) throws IOException { - LOG.info("Object type {} name {} op {} new stage {}", type, name, op, + LOG.info("Object type {} id {} op {} new stage {}", type, id, op, stage); if (type == StorageContainerLocationProtocolProtos .ObjectStageChangeRequestProto.Type.container) { @@ -206,10 +206,10 @@ public class SCMClientProtocolServer implements .ObjectStageChangeRequestProto.Op.create) { if (stage == StorageContainerLocationProtocolProtos .ObjectStageChangeRequestProto.Stage.begin) { - scm.getScmContainerManager().updateContainerState(name, HddsProtos + scm.getScmContainerManager().updateContainerState(id, HddsProtos .LifeCycleEvent.CREATE); } else { - scm.getScmContainerManager().updateContainerState(name, HddsProtos + scm.getScmContainerManager().updateContainerState(id, HddsProtos .LifeCycleEvent.CREATED); } } else { @@ -217,10 +217,10 @@ public class SCMClientProtocolServer implements .ObjectStageChangeRequestProto.Op.close) { if (stage == StorageContainerLocationProtocolProtos .ObjectStageChangeRequestProto.Stage.begin) { - scm.getScmContainerManager().updateContainerState(name, HddsProtos + scm.getScmContainerManager().updateContainerState(id, HddsProtos .LifeCycleEvent.FINALIZE); } else { - scm.getScmContainerManager().updateContainerState(name, HddsProtos + scm.getScmContainerManager().updateContainerState(id, HddsProtos .LifeCycleEvent.CLOSE); } } @@ -292,6 +292,11 @@ public class SCMClientProtocolServer implements return resultList; } + @VisibleForTesting + public StorageContainerManager getScm() { + return scm; + } + /** * Query the System for Nodes. * diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index af7dd3fac1..a7248bb85b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -87,7 +87,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate; * create a container, which then can be used to store data. */ @InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"}) -public final class StorageContainerManager extends ServiceRuntimeInfoImpl +public class StorageContainerManager extends ServiceRuntimeInfoImpl implements SCMMXBean { private static final Logger LOG = LoggerFactory @@ -168,8 +168,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl cacheSize); scmBlockManager = - new BlockManagerImpl(conf, getScmNodeManager(), scmContainerManager, - cacheSize); + new BlockManagerImpl(conf, getScmNodeManager(), scmContainerManager); scmAdminUsernames = conf.getTrimmedStringCollection(OzoneConfigKeys .OZONE_ADMINISTRATORS); @@ -459,9 +458,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl } @VisibleForTesting - public ContainerInfo getContainerInfo(String containerName) throws + public ContainerInfo getContainerInfo(long containerID) throws IOException { - return scmContainerManager.getContainer(containerName); + return scmContainerManager.getContainer(containerID); } /** diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java index 0eff702cde..f3e42ea731 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.ContainerMapping; import org.apache.hadoop.hdds.scm.container.MockNodeManager; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.container.common.SCMTestUtils; @@ -40,7 +39,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Paths; import java.util.Collections; -import java.util.UUID; import static org.apache.hadoop.ozone.OzoneConsts.GB; import static org.apache.hadoop.ozone.OzoneConsts.MB; @@ -76,7 +74,7 @@ public class TestBlockManager { } nodeManager = new MockNodeManager(true, 10); mapping = new ContainerMapping(conf, nodeManager, 128); - blockManager = new BlockManagerImpl(conf, nodeManager, mapping, 128); + blockManager = new BlockManagerImpl(conf, nodeManager, mapping); if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){ factor = HddsProtos.ReplicationFactor.THREE; @@ -107,32 +105,12 @@ public class TestBlockManager { } @Test - public void testGetAllocatedBlock() throws IOException { - AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, - type, factor, containerOwner); - Assert.assertNotNull(block); - Pipeline pipeline = blockManager.getBlock(block.getKey()); - Assert.assertEquals(pipeline.getLeader().getUuid(), - block.getPipeline().getLeader().getUuid()); - } - - @Test public void testDeleteBlock() throws Exception { AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner); Assert.assertNotNull(block); - blockManager.deleteBlocks(Collections.singletonList(block.getKey())); - - // Deleted block can not be retrieved - thrown.expectMessage("Specified block key does not exist."); - blockManager.getBlock(block.getKey()); - - // Tombstone of the deleted block can be retrieved if it has not been - // cleaned yet. - String deletedKeyName = blockManager.getDeletedKeyName(block.getKey()); - Pipeline pipeline = blockManager.getBlock(deletedKeyName); - Assert.assertEquals(pipeline.getLeader().getUuid(), - block.getPipeline().getLeader().getUuid()); + blockManager.deleteBlocks(Collections.singletonList( + block.getBlockID())); } @Test @@ -143,12 +121,6 @@ public class TestBlockManager { type, factor, containerOwner); } - @Test - public void testGetNoneExistentContainer() throws IOException { - String nonExistBlockKey = UUID.randomUUID().toString(); - thrown.expectMessage("Specified block key does not exist."); - blockManager.getBlock(nonExistBlockKey); - } @Test public void testChillModeAllocateBlockFails() throws IOException { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java index 77030cdd54..f872e23159 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java @@ -82,17 +82,22 @@ public class TestDeletedBlockLog { FileUtils.deleteDirectory(testDir); } - private Map<String, List<String>> generateData(int dataSize) { - Map<String, List<String>> blockMap = new HashMap<>(); + private Map<Long, List<Long>> generateData(int dataSize) { + Map<Long, List<Long>> blockMap = new HashMap<>(); Random random = new Random(1); + int continerIDBase = random.nextInt(100); + int localIDBase = random.nextInt(1000); for (int i = 0; i < dataSize; i++) { - String containerName = "container-" + UUID.randomUUID().toString(); - List<String> blocks = new ArrayList<>(); + //String containerName = "container-" + UUID.randomUUID().toString(); + long containerID = continerIDBase + i; + List<Long> blocks = new ArrayList<>(); int blockSize = random.nextInt(30) + 1; for (int j = 0; j < blockSize; j++) { - blocks.add("block-" + UUID.randomUUID().toString()); + //blocks.add("block-" + UUID.randomUUID().toString()); + long localID = localIDBase + j; + blocks.add(localID); } - blockMap.put(containerName, blocks); + blockMap.put(containerID, blocks); } return blockMap; } @@ -104,7 +109,7 @@ public class TestDeletedBlockLog { Assert.assertEquals(0, blocks.size()); // Creates 40 TX in the log. - for (Map.Entry<String, List<String>> entry : generateData(40).entrySet()){ + for (Map.Entry<Long, List<Long>> entry : generateData(40).entrySet()){ deletedBlockLog.addTransaction(entry.getKey(), entry.getValue()); } @@ -143,7 +148,7 @@ public class TestDeletedBlockLog { int maxRetry = conf.getInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 20); // Create 30 TXs in the log. - for (Map.Entry<String, List<String>> entry : generateData(30).entrySet()){ + for (Map.Entry<Long, List<Long>> entry : generateData(30).entrySet()){ deletedBlockLog.addTransaction(entry.getKey(), entry.getValue()); } @@ -172,7 +177,7 @@ public class TestDeletedBlockLog { @Test public void testCommitTransactions() throws Exception { - for (Map.Entry<String, List<String>> entry : generateData(50).entrySet()){ + for (Map.Entry<Long, List<Long>> entry : generateData(50).entrySet()){ deletedBlockLog.addTransaction(entry.getKey(), entry.getValue()); } List<DeletedBlocksTransaction> blocks = @@ -203,7 +208,7 @@ public class TestDeletedBlockLog { for (int i = 0; i < 100; i++) { int state = random.nextInt(4); if (state == 0) { - for (Map.Entry<String, List<String>> entry : + for (Map.Entry<Long, List<Long>> entry : generateData(10).entrySet()){ deletedBlockLog.addTransaction(entry.getKey(), entry.getValue()); } @@ -234,7 +239,7 @@ public class TestDeletedBlockLog { @Test public void testPersistence() throws Exception { - for (Map.Entry<String, List<String>> entry : generateData(50).entrySet()){ + for (Map.Entry<Long, List<Long>> entry : generateData(50).entrySet()){ deletedBlockLog.addTransaction(entry.getKey(), entry.getValue()); } // close db and reopen it again to make sure @@ -257,10 +262,10 @@ public class TestDeletedBlockLog { int txNum = 10; int maximumAllowedTXNum = 5; List<DeletedBlocksTransaction> blocks = null; - List<String> containerNames = new LinkedList<>(); + List<Long> containerIDs = new LinkedList<>(); int count = 0; - String containerName = null; + long containerID = 0L; DatanodeDetails dnDd1 = DatanodeDetails.newBuilder() .setUuid(UUID.randomUUID().toString()) .setIpAddress("127.0.0.1") @@ -279,18 +284,18 @@ public class TestDeletedBlockLog { .build(); Mapping mappingService = mock(ContainerMapping.class); // Creates {TXNum} TX in the log. - for (Map.Entry<String, List<String>> entry : generateData(txNum) + for (Map.Entry<Long, List<Long>> entry : generateData(txNum) .entrySet()) { count++; - containerName = entry.getKey(); - containerNames.add(containerName); - deletedBlockLog.addTransaction(containerName, entry.getValue()); + containerID = entry.getKey(); + containerIDs.add(containerID); + deletedBlockLog.addTransaction(containerID, entry.getValue()); // make TX[1-6] for datanode1; TX[7-10] for datanode2 if (count <= (maximumAllowedTXNum + 1)) { - mockContainerInfo(mappingService, containerName, dnDd1); + mockContainerInfo(mappingService, containerID, dnDd1); } else { - mockContainerInfo(mappingService, containerName, dnId2); + mockContainerInfo(mappingService, containerID, dnId2); } } @@ -325,7 +330,7 @@ public class TestDeletedBlockLog { DeletedBlocksTransaction.Builder builder = DeletedBlocksTransaction.newBuilder(); builder.setTxID(11); - builder.setContainerName(containerName); + builder.setContainerID(containerID); builder.setCount(0); transactions.addTransaction(builder.build()); @@ -334,30 +339,29 @@ public class TestDeletedBlockLog { transactions.getDatanodeTransactions(dnId2.getUuid()).size()); // Add new TX in dnID2, then dnID2 will reach maximum value. - containerName = "newContainer"; builder = DeletedBlocksTransaction.newBuilder(); builder.setTxID(12); - builder.setContainerName(containerName); + builder.setContainerID(containerID); builder.setCount(0); - mockContainerInfo(mappingService, containerName, dnId2); + mockContainerInfo(mappingService, containerID, dnId2); transactions.addTransaction(builder.build()); // Since all node are full, then transactions is full. Assert.assertTrue(transactions.isFull()); } - private void mockContainerInfo(Mapping mappingService, String containerName, + private void mockContainerInfo(Mapping mappingService, long containerID, DatanodeDetails dd) throws IOException { PipelineChannel pipelineChannel = new PipelineChannel("fake", LifeCycleState.OPEN, ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "fake"); pipelineChannel.addMember(dd); - Pipeline pipeline = new Pipeline(containerName, pipelineChannel); + Pipeline pipeline = new Pipeline(pipelineChannel); ContainerInfo.Builder builder = new ContainerInfo.Builder(); builder.setPipeline(pipeline); ContainerInfo conatinerInfo = builder.build(); Mockito.doReturn(conatinerInfo).when(mappingService) - .getContainer(containerName); + .getContainer(containerID); } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java index 200a611d0f..a27068bb33 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java @@ -45,6 +45,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NavigableSet; +import java.util.Random; import java.util.Set; import java.util.TreeSet; import java.util.UUID; @@ -59,6 +60,7 @@ public class TestContainerMapping { private static File testDir; private static XceiverClientManager xceiverClientManager; private static String containerOwner = "OZONE"; + private static Random random; private static final long TIMEOUT = 10000; @@ -83,6 +85,7 @@ public class TestContainerMapping { nodeManager = new MockNodeManager(true, 10); mapping = new ContainerMapping(conf, nodeManager, 128); xceiverClientManager = new XceiverClientManager(conf); + random = new Random(); } @AfterClass @@ -103,7 +106,7 @@ public class TestContainerMapping { ContainerInfo containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), - UUID.randomUUID().toString(), containerOwner); + containerOwner); Assert.assertNotNull(containerInfo); } @@ -120,7 +123,7 @@ public class TestContainerMapping { ContainerInfo containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), - UUID.randomUUID().toString(), containerOwner); + containerOwner); Assert.assertNotNull(containerInfo); Assert.assertNotNull(containerInfo.getPipeline()); @@ -132,59 +135,41 @@ public class TestContainerMapping { @Test public void testGetContainer() throws IOException { - String containerName = UUID.randomUUID().toString(); - Pipeline pipeline = mapping.allocateContainer( + ContainerInfo containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerName, - containerOwner).getPipeline(); + xceiverClientManager.getFactor(), + containerOwner); + Pipeline pipeline = containerInfo.getPipeline(); Assert.assertNotNull(pipeline); - Pipeline newPipeline = mapping.getContainer(containerName).getPipeline(); + Pipeline newPipeline = mapping.getContainer( + containerInfo.getContainerID()).getPipeline(); Assert.assertEquals(pipeline.getLeader().getUuid(), newPipeline.getLeader().getUuid()); } @Test - public void testDuplicateAllocateContainerFails() throws IOException { - String containerName = UUID.randomUUID().toString(); - Pipeline pipeline = mapping.allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerName, - containerOwner).getPipeline(); - Assert.assertNotNull(pipeline); - thrown.expectMessage("Specified container already exists."); - mapping.allocateContainer(xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerName, - containerOwner); - } - - @Test public void testgetNoneExistentContainer() throws IOException { - String containerName = UUID.randomUUID().toString(); thrown.expectMessage("Specified key does not exist."); - mapping.getContainer(containerName); + mapping.getContainer(random.nextLong()); } @Test public void testChillModeAllocateContainerFails() throws IOException { - String containerName = UUID.randomUUID().toString(); nodeManager.setChillmode(true); thrown.expectMessage("Unable to create container while in chill mode"); mapping.allocateContainer(xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerName, - containerOwner); + xceiverClientManager.getFactor(), containerOwner); } @Test public void testContainerCreationLeaseTimeout() throws IOException, InterruptedException { - String containerName = UUID.randomUUID().toString(); nodeManager.setChillmode(false); ContainerInfo containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), - containerName, containerOwner); - mapping.updateContainerState(containerInfo.getContainerName(), + mapping.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); Thread.sleep(TIMEOUT + 1000); @@ -198,14 +183,13 @@ public class TestContainerMapping { thrown.expect(IOException.class); thrown.expectMessage("Lease Exception"); - mapping.updateContainerState(containerInfo.getContainerName(), + mapping.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); } @Test public void testFullContainerReport() throws IOException { - String containerName = UUID.randomUUID().toString(); - ContainerInfo info = createContainer(containerName); + ContainerInfo info = createContainer(); DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(); ContainerReportsRequestProto.reportType reportType = ContainerReportsRequestProto.reportType.fullReport; @@ -213,9 +197,7 @@ public class TestContainerMapping { new ArrayList<>(); StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); - ciBuilder.setContainerName(containerName) - //setting some random hash - .setFinalhash("e16cc9d6024365750ed8dbd194ea46d2") + ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2") .setSize(5368709120L) .setUsed(2000000000L) .setKeyCount(100000000L) @@ -234,15 +216,14 @@ public class TestContainerMapping { mapping.processContainerReports(crBuilder.build()); - ContainerInfo updatedContainer = mapping.getContainer(containerName); + ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID()); Assert.assertEquals(100000000L, updatedContainer.getNumberOfKeys()); Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes()); } @Test public void testContainerCloseWithContainerReport() throws IOException { - String containerName = UUID.randomUUID().toString(); - ContainerInfo info = createContainer(containerName); + ContainerInfo info = createContainer(); DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(); ContainerReportsRequestProto.reportType reportType = ContainerReportsRequestProto.reportType.fullReport; @@ -251,9 +232,7 @@ public class TestContainerMapping { StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); - ciBuilder.setContainerName(containerName) - //setting some random hash - .setFinalhash("7c45eb4d7ed5e0d2e89aaab7759de02e") + ciBuilder.setFinalhash("7c45eb4d7ed5e0d2e89aaab7759de02e") .setSize(5368709120L) .setUsed(5368705120L) .setKeyCount(500000000L) @@ -272,7 +251,7 @@ public class TestContainerMapping { mapping.processContainerReports(crBuilder.build()); - ContainerInfo updatedContainer = mapping.getContainer(containerName); + ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID()); Assert.assertEquals(500000000L, updatedContainer.getNumberOfKeys()); Assert.assertEquals(5368705120L, updatedContainer.getUsedBytes()); NavigableSet<ContainerID> pendingCloseContainers = mapping.getStateManager() @@ -287,9 +266,8 @@ public class TestContainerMapping { @Test public void testCloseContainer() throws IOException { - String containerName = UUID.randomUUID().toString(); - ContainerInfo info = createContainer(containerName); - mapping.updateContainerState(containerName, + ContainerInfo info = createContainer(); + mapping.updateContainerState(info.getContainerID(), HddsProtos.LifeCycleEvent.FINALIZE); NavigableSet<ContainerID> pendingCloseContainers = mapping.getStateManager() .getMatchingContainerIDs( @@ -298,7 +276,7 @@ public class TestContainerMapping { xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CLOSING); Assert.assertTrue(pendingCloseContainers.contains(info.containerID())); - mapping.updateContainerState(containerName, + mapping.updateContainerState(info.getContainerID(), HddsProtos.LifeCycleEvent.CLOSE); NavigableSet<ContainerID> closeContainers = mapping.getStateManager() .getMatchingContainerIDs( @@ -311,21 +289,18 @@ public class TestContainerMapping { /** * Creates a container with the given name in ContainerMapping. - * @param containerName - * Name of the container * @throws IOException */ - private ContainerInfo createContainer(String containerName) + private ContainerInfo createContainer() throws IOException { nodeManager.setChillmode(false); ContainerInfo containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), - containerName, containerOwner); - mapping.updateContainerState(containerInfo.getContainerName(), + mapping.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - mapping.updateContainerState(containerInfo.getContainerName(), + mapping.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); return containerInfo; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java index 2fec232421..f3f37c72d1 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdds.scm.container.closer; -import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.scm.TestUtils; @@ -92,15 +91,13 @@ public class TestContainerCloser { @Test public void testClose() throws IOException { - String containerName = "container-" + RandomStringUtils.randomNumeric(5); - ContainerInfo info = mapping.allocateContainer( HddsProtos.ReplicationType.STAND_ALONE, - HddsProtos.ReplicationFactor.ONE, containerName, "ozone"); + HddsProtos.ReplicationFactor.ONE, "ozone"); //Execute these state transitions so that we can close the container. - mapping.updateContainerState(containerName, CREATE); - mapping.updateContainerState(containerName, CREATED); + mapping.updateContainerState(info.getContainerID(), CREATE); + mapping.updateContainerState(info.getContainerID(), CREATED); long currentCount = mapping.getCloser().getCloseCount(); long runCount = mapping.getCloser().getThreadRunCount(); @@ -120,7 +117,7 @@ public class TestContainerCloser { long newUsed = (long) (size * 0.91f); sendContainerReport(info, newUsed); - // with only one container the cleaner thread should not run. + // with only one container the cleaner thread should not run. Assert.assertEquals(runCount, mapping.getCloser().getThreadRunCount()); // and close count will be one. @@ -140,14 +137,13 @@ public class TestContainerCloser { configuration.setTimeDuration(OZONE_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS); - String containerName = "container-" + RandomStringUtils.randomNumeric(5); ContainerInfo info = mapping.allocateContainer( HddsProtos.ReplicationType.STAND_ALONE, - HddsProtos.ReplicationFactor.ONE, containerName, "ozone"); + HddsProtos.ReplicationFactor.ONE, "ozone"); //Execute these state transitions so that we can close the container. - mapping.updateContainerState(containerName, CREATE); + mapping.updateContainerState(info.getContainerID(), CREATE); long currentCount = mapping.getCloser().getCloseCount(); long runCount = mapping.getCloser().getThreadRunCount(); @@ -187,12 +183,11 @@ public class TestContainerCloser { long runCount = mapping.getCloser().getThreadRunCount(); for (int x = 0; x < ContainerCloser.getCleanupWaterMark() + 10; x++) { - String containerName = "container-" + RandomStringUtils.randomNumeric(7); ContainerInfo info = mapping.allocateContainer( HddsProtos.ReplicationType.STAND_ALONE, - HddsProtos.ReplicationFactor.ONE, containerName, "ozone"); - mapping.updateContainerState(containerName, CREATE); - mapping.updateContainerState(containerName, CREATED); + HddsProtos.ReplicationFactor.ONE, "ozone"); + mapping.updateContainerState(info.getContainerID(), CREATE); + mapping.updateContainerState(info.getContainerID(), CREATED); sendContainerReport(info, 5 * GIGABYTE); } @@ -210,7 +205,7 @@ public class TestContainerCloser { StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); - ciBuilder.setContainerName(info.getContainerName()) + ciBuilder.setContainerID(info.getContainerID()) .setFinalhash("e16cc9d6024365750ed8dbd194ea46d2") .setSize(size) .setUsed(used) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java index ad50d971b3..6f994a9897 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.container.ContainerMapping; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.placement.algorithms .ContainerPlacementPolicy; @@ -160,13 +161,11 @@ public class TestContainerPlacement { assertTrue(nodeManager.isOutOfChillMode()); - String container1 = UUID.randomUUID().toString(); - Pipeline pipeline1 = containerManager.allocateContainer( + ContainerInfo containerInfo = containerManager.allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container1, "OZONE") - .getPipeline(); + xceiverClientManager.getFactor(), "OZONE"); assertEquals(xceiverClientManager.getFactor().getNumber(), - pipeline1.getMachines().size()); + containerInfo.getPipeline().getMachines().size()); } finally { IOUtils.closeQuietly(containerManager); IOUtils.closeQuietly(nodeManager); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java index d0839c52fb..ebfe978d48 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java @@ -17,6 +17,7 @@ package org.apache.hadoop.ozone.container.common; import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.scm.TestUtils; @@ -362,7 +363,7 @@ public class TestEndPoint { * @return */ ContainerReport getRandomContainerReport() { - return new ContainerReport(UUID.randomUUID().toString(), + return new ContainerReport(RandomUtils.nextLong(), DigestUtils.sha256Hex("Random")); } @@ -436,7 +437,8 @@ public class TestEndPoint { reportsBuilder = StorageContainerDatanodeProtocolProtos .ContainerReportsRequestProto.newBuilder(); for (int x = 0; x < count; x++) { - ContainerReport report = new ContainerReport(UUID.randomUUID().toString(), + long containerID = RandomUtils.nextLong(); + ContainerReport report = new ContainerReport(containerID, DigestUtils.sha256Hex("Simulated")); report.setKeyCount(1000); report.setSize(OzoneConsts.GB * 5); @@ -445,7 +447,6 @@ public class TestEndPoint { report.setReadBytes(OzoneConsts.GB * 1); report.setWriteCount(50); report.setWriteBytes(OzoneConsts.GB * 2); - report.setContainerID(1); reportsBuilder.addReports(report.getProtoBufMessage()); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java index 8eb07e61c2..01f70b1628 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java @@ -149,17 +149,17 @@ public class TestContainerSupervisor { */ public void testDetectSingleContainerReplica() throws TimeoutException, InterruptedException { - String singleNodeContainer = "SingleNodeContainer"; - String threeNodeContainer = "ThreeNodeContainer"; + long singleNodeContainerID = 9001; + long threeNodeContainerID = 9003; InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0); // Only single datanode reporting that "SingleNodeContainer" exists. List<ContainerReportsRequestProto> clist = - datanodeStateManager.getContainerReport(singleNodeContainer, + datanodeStateManager.getContainerReport(singleNodeContainerID, ppool.getPool().getPoolName(), 1); ppool.handleContainerReport(clist.get(0)); // Three nodes are going to report that ThreeNodeContainer exists. - clist = datanodeStateManager.getContainerReport(threeNodeContainer, + clist = datanodeStateManager.getContainerReport(threeNodeContainerID, ppool.getPool().getPoolName(), 3); for (ContainerReportsRequestProto reportsProto : clist) { @@ -169,9 +169,10 @@ public class TestContainerSupervisor { 200, 1000); ppool.setDoneProcessing(); - List<Map.Entry<String, Integer>> containers = ppool.filterContainer(p -> p + List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p .getValue() == 1); - Assert.assertEquals(singleNodeContainer, containers.get(0).getKey()); + Assert.assertEquals(singleNodeContainerID, + containers.get(0).getKey().longValue()); int count = containers.get(0).getValue(); Assert.assertEquals(1L, count); } @@ -184,24 +185,24 @@ public class TestContainerSupervisor { */ public void testDetectOverReplica() throws TimeoutException, InterruptedException { - String normalContainer = "NormalContainer"; - String overReplicated = "OverReplicatedContainer"; - String wayOverReplicated = "WayOverReplicated"; + long normalContainerID = 9000; + long overReplicatedContainerID = 9001; + long wayOverReplicatedContainerID = 9002; InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0); List<ContainerReportsRequestProto> clist = - datanodeStateManager.getContainerReport(normalContainer, + datanodeStateManager.getContainerReport(normalContainerID, ppool.getPool().getPoolName(), 3); ppool.handleContainerReport(clist.get(0)); - clist = datanodeStateManager.getContainerReport(overReplicated, + clist = datanodeStateManager.getContainerReport(overReplicatedContainerID, ppool.getPool().getPoolName(), 4); for (ContainerReportsRequestProto reportsProto : clist) { ppool.handleContainerReport(reportsProto); } - clist = datanodeStateManager.getContainerReport(wayOverReplicated, + clist = datanodeStateManager.getContainerReport(wayOverReplicatedContainerID, ppool.getPool().getPoolName(), 7); for (ContainerReportsRequestProto reportsProto : clist) { @@ -215,7 +216,7 @@ public class TestContainerSupervisor { 200, 1000); ppool.setDoneProcessing(); - List<Map.Entry<String, Integer>> containers = ppool.filterContainer(p -> p + List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p .getValue() > 3); Assert.assertEquals(2, containers.size()); } @@ -255,14 +256,15 @@ public class TestContainerSupervisor { logCapturer.getOutput().contains("PoolNew"), 200, 15 * 1000); + long newContainerID = 7001; // Assert that we are able to send a container report to this new // pool and datanode. List<ContainerReportsRequestProto> clist = - datanodeStateManager.getContainerReport("NewContainer1", + datanodeStateManager.getContainerReport(newContainerID, "PoolNew", 1); containerSupervisor.handleContainerReport(clist.get(0)); GenericTestUtils.waitFor(() -> - inProgressLog.getOutput().contains("NewContainer1") && inProgressLog + inProgressLog.getOutput().contains(Long.toString(newContainerID)) && inProgressLog .getOutput().contains(id.getUuidString()), 200, 10 * 1000); } finally { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java index 26f3514497..50fd18f565 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java @@ -16,6 +16,7 @@ */ package org.apache.hadoop.ozone.container.testutils; +import com.google.common.primitives.Longs; import org.apache.commons.codec.digest.DigestUtils; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.NodePoolManager; @@ -56,13 +57,13 @@ public class ReplicationDatanodeStateManager { /** * Get Container Report as if it is from a datanode in the cluster. - * @param containerName - Container Name. + * @param containerID - Container ID. * @param poolName - Pool Name. * @param dataNodeCount - Datanode Count. * @return List of Container Reports. */ public List<ContainerReportsRequestProto> getContainerReport( - String containerName, String poolName, int dataNodeCount) { + long containerID, String poolName, int dataNodeCount) { List<ContainerReportsRequestProto> containerList = new LinkedList<>(); List<DatanodeDetails> nodesInPool = poolManager.getNodes(poolName); @@ -75,7 +76,6 @@ public class ReplicationDatanodeStateManager { "required container reports"); } - int containerID = 1; while (containerList.size() < dataNodeCount && nodesInPool.size() > 0) { DatanodeDetails id = nodesInPool.get(r.nextInt(nodesInPool.size())); nodesInPool.remove(id); @@ -83,8 +83,9 @@ public class ReplicationDatanodeStateManager { // We return container reports only for nodes that are healthy. if (nodeManager.getNodeState(id) == HEALTHY) { ContainerInfo info = ContainerInfo.newBuilder() - .setContainerName(containerName) - .setFinalhash(DigestUtils.sha256Hex(containerName)) + .setContainerID(containerID) + .setFinalhash(DigestUtils.sha256Hex( + Longs.toByteArray(containerID))) .setContainerID(containerID) .build(); ContainerReportsRequestProto containerReport = diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java index 9a44525461..4f3b14385c 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java @@ -24,7 +24,7 @@ import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.cli.SCMCLI; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import java.io.IOException; @@ -34,30 +34,32 @@ import java.io.IOException; public class CloseContainerHandler extends OzoneCommandHandler { public static final String CONTAINER_CLOSE = "close"; - public static final String OPT_CONTAINER_NAME = "c"; + public static final String OPT_CONTAINER_ID = "c"; @Override public void execute(CommandLine cmd) throws IOException { if (!cmd.hasOption(CONTAINER_CLOSE)) { throw new IOException("Expecting container close"); } - if (!cmd.hasOption(OPT_CONTAINER_NAME)) { + if (!cmd.hasOption(OPT_CONTAINER_ID)) { displayHelp(); if (!cmd.hasOption(SCMCLI.HELP_OP)) { - throw new IOException("Expecting container name"); + throw new IOException("Expecting container id"); } else { return; } } - String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME); + String containerID = cmd.getOptionValue(OPT_CONTAINER_ID); - Pipeline pipeline = getScmClient().getContainer(containerName); - if (pipeline == null) { + ContainerInfo container = getScmClient(). + getContainer(Long.parseLong(containerID)); + if (container == null) { throw new IOException("Cannot close an non-exist container " - + containerName); + + containerID); } - logOut("Closing container : %s.", containerName); - getScmClient().closeContainer(pipeline); + logOut("Closing container : %s.", containerID); + getScmClient().closeContainer(container.getContainerID(), + container.getPipeline()); logOut("Container closed."); } @@ -72,8 +74,8 @@ public class CloseContainerHandler extends OzoneCommandHandler { } public static void addOptions(Options options) { - Option containerNameOpt = new Option(OPT_CONTAINER_NAME, - true, "Specify container name"); + Option containerNameOpt = new Option(OPT_CONTAINER_ID, + true, "Specify container ID"); options.addOption(containerNameOpt); } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java index 980388f28c..428f179932 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java @@ -119,7 +119,6 @@ public class ContainerCommandHandler extends OzoneCommandHandler { public static void addOptions(Options options) { addCommandsOption(options); // for create container options. - CreateContainerHandler.addOptions(options); DeleteContainerHandler.addOptions(options); InfoContainerHandler.addOptions(options); ListContainerHandler.addOptions(options); diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java index 2961831e67..c0ff1f7a6d 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hdds.scm.cli.container; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.client.ScmClient; @@ -35,7 +34,6 @@ import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP; public class CreateContainerHandler extends OzoneCommandHandler { public static final String CONTAINER_CREATE = "create"; - public static final String OPT_CONTAINER_NAME = "c"; public static final String CONTAINER_OWNER = "OZONE"; // TODO Support an optional -p <pipelineID> option to create // container on given datanodes. @@ -49,33 +47,17 @@ public class CreateContainerHandler extends OzoneCommandHandler { if (!cmd.hasOption(CONTAINER_CREATE)) { throw new IOException("Expecting container create"); } - if (!cmd.hasOption(OPT_CONTAINER_NAME)) { - displayHelp(); - if (!cmd.hasOption(HELP_OP)) { - throw new IOException("Expecting container name"); - } else { - return; - } - } - String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME); - logOut("Creating container : %s.", containerName); - getScmClient().createContainer(containerName, CONTAINER_OWNER); + logOut("Creating container..."); + getScmClient().createContainer(CONTAINER_OWNER); logOut("Container created."); } @Override public void displayHelp() { Options options = new Options(); - addOptions(options); HelpFormatter helpFormatter = new HelpFormatter(); helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -create <option>", "where <option> is", options, ""); } - - public static void addOptions(Options options) { - Option containerNameOpt = new Option(OPT_CONTAINER_NAME, - true, "Specify container name"); - options.addOption(containerNameOpt); - } } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java index a5b625a5e6..20a6d9eb27 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java @@ -25,7 +25,7 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import java.io.IOException; @@ -39,7 +39,7 @@ public class DeleteContainerHandler extends OzoneCommandHandler { protected static final String CONTAINER_DELETE = "delete"; protected static final String OPT_FORCE = "f"; - protected static final String OPT_CONTAINER_NAME = "c"; + protected static final String OPT_CONTAINER_ID = "c"; public DeleteContainerHandler(ScmClient scmClient) { super(scmClient); @@ -49,7 +49,7 @@ public class DeleteContainerHandler extends OzoneCommandHandler { public void execute(CommandLine cmd) throws IOException { Preconditions.checkArgument(cmd.hasOption(CONTAINER_DELETE), "Expecting command delete"); - if (!cmd.hasOption(OPT_CONTAINER_NAME)) { + if (!cmd.hasOption(OPT_CONTAINER_ID)) { displayHelp(); if (!cmd.hasOption(HELP_OP)) { throw new IOException("Expecting container name"); @@ -58,17 +58,19 @@ public class DeleteContainerHandler extends OzoneCommandHandler { } } - String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME); + String containerID = cmd.getOptionValue(OPT_CONTAINER_ID); - Pipeline pipeline = getScmClient().getContainer(containerName); - if (pipeline == null) { + ContainerInfo container = getScmClient().getContainer( + Long.parseLong(containerID)); + if (container == null) { throw new IOException("Cannot delete an non-exist container " - + containerName); + + containerID); } - logOut("Deleting container : %s.", containerName); - getScmClient().deleteContainer(pipeline, cmd.hasOption(OPT_FORCE)); - logOut("Container %s deleted.", containerName); + logOut("Deleting container : %s.", containerID); + getScmClient().deleteContainer(container.getContainerID(), + container.getPipeline(), cmd.hasOption(OPT_FORCE)); + logOut("Container %s deleted.", containerID); } @Override @@ -85,8 +87,8 @@ public class DeleteContainerHandler extends OzoneCommandHandler { false, "forcibly delete a container"); options.addOption(forceOpt); - Option containerNameOpt = new Option(OPT_CONTAINER_NAME, - true, "Specify container name"); + Option containerNameOpt = new Option(OPT_CONTAINER_ID, + true, "Specify container id"); options.addOption(containerNameOpt); } } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java index c609915117..36d46c0b73 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java @@ -24,7 +24,7 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ContainerData; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -41,7 +41,7 @@ import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP; public class InfoContainerHandler extends OzoneCommandHandler { public static final String CONTAINER_INFO = "info"; - protected static final String OPT_CONTAINER_NAME = "c"; + protected static final String OPT_CONTAINER_ID = "c"; /** * Constructs a handler object. @@ -57,7 +57,7 @@ public class InfoContainerHandler extends OzoneCommandHandler { if (!cmd.hasOption(CONTAINER_INFO)) { throw new IOException("Expecting container info"); } - if (!cmd.hasOption(OPT_CONTAINER_NAME)) { + if (!cmd.hasOption(OPT_CONTAINER_ID)) { displayHelp(); if (!cmd.hasOption(HELP_OP)) { throw new IOException("Expecting container name"); @@ -65,16 +65,17 @@ public class InfoContainerHandler extends OzoneCommandHandler { return; } } - String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME); - Pipeline pipeline = getScmClient().getContainer(containerName); - Preconditions.checkNotNull(pipeline, "Pipeline cannot be null"); + String containerID = cmd.getOptionValue(OPT_CONTAINER_ID); + ContainerInfo container = getScmClient(). + getContainer(Long.parseLong(containerID)); + Preconditions.checkNotNull(container, "Container cannot be null"); ContainerData containerData = - getScmClient().readContainer(pipeline); + getScmClient().readContainer(container.getContainerID(), + container.getPipeline()); // Print container report info. - logOut("Container Name: %s", - containerData.getName()); + logOut("Container id: %s", containerID); String openStatus = containerData.getState() == HddsProtos.LifeCycleState.OPEN ? "OPEN" : "CLOSED"; @@ -91,8 +92,10 @@ public class InfoContainerHandler extends OzoneCommandHandler { logOut("Container Metadata: {%s}", metadataStr); // Print pipeline of an existing container. - logOut("LeaderID: %s", pipeline.getLeader().getHostName()); - String machinesStr = pipeline.getMachines().stream().map( + logOut("LeaderID: %s", container.getPipeline() + .getLeader().getHostName()); + String machinesStr = container.getPipeline() + .getMachines().stream().map( DatanodeDetails::getHostName).collect(Collectors.joining(",")); logOut("Datanodes: [%s]", machinesStr); } @@ -107,8 +110,8 @@ public class InfoContainerHandler extends OzoneCommandHandler { } public static void addOptions(Options options) { - Option containerNameOpt = new Option(OPT_CONTAINER_NAME, - true, "Specify container name"); - options.addOption(containerNameOpt); + Option containerIdOpt = new Option(OPT_CONTAINER_ID, + true, "Specify container id"); + options.addOption(containerIdOpt); } } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java index 0c7e79037e..42dae65ade 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java @@ -40,7 +40,6 @@ public class ListContainerHandler extends OzoneCommandHandler { public static final String CONTAINER_LIST = "list"; public static final String OPT_START_CONTAINER = "start"; - public static final String OPT_PREFIX_CONTAINER = "prefix"; public static final String OPT_COUNT = "count"; /** @@ -71,8 +70,7 @@ public class ListContainerHandler extends OzoneCommandHandler { } } - String startName = cmd.getOptionValue(OPT_START_CONTAINER); - String prefixName = cmd.getOptionValue(OPT_PREFIX_CONTAINER); + String startID = cmd.getOptionValue(OPT_START_CONTAINER); int count = 0; if (cmd.hasOption(OPT_COUNT)) { @@ -84,7 +82,8 @@ public class ListContainerHandler extends OzoneCommandHandler { } List<ContainerInfo> containerList = - getScmClient().listContainer(startName, prefixName, count); + getScmClient().listContainer( + Long.parseLong(startID), count); // Output data list for (ContainerInfo container : containerList) { @@ -109,13 +108,10 @@ public class ListContainerHandler extends OzoneCommandHandler { public static void addOptions(Options options) { Option startContainerOpt = new Option(OPT_START_CONTAINER, - true, "Specify start container name"); - Option endContainerOpt = new Option(OPT_PREFIX_CONTAINER, - true, "Specify prefix container name"); + true, "Specify start container id"); Option countOpt = new Option(OPT_COUNT, true, "Specify count number, required"); options.addOption(countOpt); options.addOption(startContainerOpt); - options.addOption(endContainerOpt); } } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java index b82ed2576b..ccc59116be 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java @@ -21,11 +21,12 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; @@ -260,7 +261,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { storageContainerLocationClient, String requestId) throws IOException { long length = 0; - String containerKey; + long containerKey; ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream(); groupInputStream.key = keyInfo.getKeyName(); List<KsmKeyLocationInfo> keyLocationInfos = @@ -268,20 +269,20 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { groupInputStream.streamOffset = new long[keyLocationInfos.size()]; for (int i = 0; i < keyLocationInfos.size(); i++) { KsmKeyLocationInfo ksmKeyLocationInfo = keyLocationInfos.get(i); - String containerName = ksmKeyLocationInfo.getContainerName(); - Pipeline pipeline = - storageContainerLocationClient.getContainer(containerName); + BlockID blockID = ksmKeyLocationInfo.getBlockID(); + long containerID = blockID.getContainerID(); + ContainerInfo container = + storageContainerLocationClient.getContainer(containerID); XceiverClientSpi xceiverClient = - xceiverClientManager.acquireClient(pipeline); + xceiverClientManager.acquireClient(container.getPipeline(), containerID); boolean success = false; - containerKey = ksmKeyLocationInfo.getBlockID(); + containerKey = ksmKeyLocationInfo.getLocalID(); try { LOG.debug("get key accessing {} {}", - xceiverClient.getPipeline().getContainerName(), containerKey); + containerID, containerKey); groupInputStream.streamOffset[i] = length; - ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation - .containerKeyDataForRead( - xceiverClient.getPipeline().getContainerName(), containerKey); + ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation + .containerKeyDataForRead(blockID); ContainerProtos.GetKeyResponseProto response = ContainerProtocolCalls .getKey(xceiverClient, containerKeyData, requestId); List<ContainerProtos.ChunkInfo> chunks = @@ -291,7 +292,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { } success = true; ChunkInputStream inputStream = new ChunkInputStream( - containerKey, xceiverClientManager, xceiverClient, + ksmKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient, chunks, requestId); groupInputStream.addStream(inputStream, ksmKeyLocationInfo.getLength()); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java index 4c465d36b8..41ceee41f0 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java @@ -21,6 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; @@ -32,7 +34,6 @@ import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers .StorageContainerException; import org.apache.hadoop.hdds.scm.protocolPB @@ -162,30 +163,34 @@ public class ChunkGroupOutputStream extends OutputStream { private void checkKeyLocationInfo(KsmKeyLocationInfo subKeyInfo) throws IOException { - String containerKey = subKeyInfo.getBlockID(); - String containerName = subKeyInfo.getContainerName(); - Pipeline pipeline = scmClient.getContainer(containerName); + ContainerInfo container = scmClient.getContainer( + subKeyInfo.getContainerID()); XceiverClientSpi xceiverClient = - xceiverClientManager.acquireClient(pipeline); + xceiverClientManager.acquireClient(container.getPipeline(), + container.getContainerID()); // create container if needed if (subKeyInfo.getShouldCreateContainer()) { try { - ContainerProtocolCalls.createContainer(xceiverClient, requestID); + ContainerProtocolCalls.createContainer(xceiverClient, + container.getContainerID(), requestID); scmClient.notifyObjectStageChange( ObjectStageChangeRequestProto.Type.container, - containerName, ObjectStageChangeRequestProto.Op.create, + subKeyInfo.getContainerID(), + ObjectStageChangeRequestProto.Op.create, ObjectStageChangeRequestProto.Stage.complete); } catch (StorageContainerException ex) { if (ex.getResult().equals(Result.CONTAINER_EXISTS)) { //container already exist, this should never happen - LOG.debug("Container {} already exists.", containerName); + LOG.debug("Container {} already exists.", + container.getContainerID()); } else { - LOG.error("Container creation failed for {}.", containerName, ex); + LOG.error("Container creation failed for {}.", + container.getContainerID(), ex); throw ex; } } } - streamEntries.add(new ChunkOutputStreamEntry(containerKey, + streamEntries.add(new ChunkOutputStreamEntry(subKeyInfo.getBlockID(), keyArgs.getKeyName(), xceiverClientManager, xceiverClient, requestID, chunkSize, subKeyInfo.getLength())); } @@ -390,7 +395,7 @@ public class ChunkGroupOutputStream extends OutputStream { private static class ChunkOutputStreamEntry extends OutputStream { private OutputStream outputStream; - private final String containerKey; + private final BlockID blockID; private final String key; private final XceiverClientManager xceiverClientManager; private final XceiverClientSpi xceiverClient; @@ -401,12 +406,12 @@ public class ChunkGroupOutputStream extends OutputStream { // the current position of this stream 0 <= currentPosition < length private long currentPosition; - ChunkOutputStreamEntry(String containerKey, String key, + ChunkOutputStreamEntry(BlockID blockID, String key, XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient, String requestId, int chunkSize, long length) { this.outputStream = null; - this.containerKey = containerKey; + this.blockID = blockID; this.key = key; this.xceiverClientManager = xceiverClientManager; this.xceiverClient = xceiverClient; @@ -424,7 +429,7 @@ public class ChunkGroupOutputStream extends OutputStream { */ ChunkOutputStreamEntry(OutputStream outputStream, long length) { this.outputStream = outputStream; - this.containerKey = null; + this.blockID = null; this.key = null; this.xceiverClientManager = null; this.xceiverClient = null; @@ -445,7 +450,7 @@ public class ChunkGroupOutputStream extends OutputStream { private synchronized void checkStream() { if (this.outputStream == null) { - this.outputStream = new ChunkOutputStream(containerKey, + this.outputStream = new ChunkOutputStream(blockID, key, xceiverClientManager, xceiverClient, requestId, chunkSize); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java index bf9e80f4ab..2132bc89a6 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java @@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.client.io; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.KeyData; +import org.apache.hadoop.hdds.client.BlockID; + /** * This class contains methods that define the translation between the Ozone @@ -30,16 +32,13 @@ final class OzoneContainerTranslation { /** * Creates key data intended for reading a container key. * - * @param containerName container name - * @param containerKey container key + * @param blockID - ID of the block. * @return KeyData intended for reading the container key */ - public static KeyData containerKeyDataForRead(String containerName, - String containerKey) { + public static KeyData containerKeyDataForRead(BlockID blockID) { return KeyData .newBuilder() - .setContainerName(containerName) - .setName(containerKey) + .setBlockID(blockID.getProtobuf()) .build(); } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java index 9d24b30659..45feda0b7c 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java @@ -16,6 +16,7 @@ */ package org.apache.hadoop.ozone.ksm.helpers; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocation; /** @@ -23,9 +24,7 @@ import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyL * into a number of subkeys. This class represents one such subkey instance. */ public final class KsmKeyLocationInfo { - private final String containerName; - // name of the block id SCM assigned for the key - private final String blockID; + private final BlockID blockID; private final boolean shouldCreateContainer; // the id of this subkey in all the subkeys. private final long length; @@ -33,10 +32,8 @@ public final class KsmKeyLocationInfo { // the version number indicating when this block was added private long createVersion; - private KsmKeyLocationInfo(String containerName, - String blockID, boolean shouldCreateContainer, + private KsmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer, long length, long offset) { - this.containerName = containerName; this.blockID = blockID; this.shouldCreateContainer = shouldCreateContainer; this.length = length; @@ -51,12 +48,16 @@ public final class KsmKeyLocationInfo { return createVersion; } - public String getContainerName() { - return containerName; + public BlockID getBlockID() { + return blockID; } - public String getBlockID() { - return blockID; + public long getContainerID() { + return blockID.getContainerID(); + } + + public long getLocalID() { + return blockID.getLocalID(); } public boolean getShouldCreateContainer() { @@ -75,19 +76,13 @@ public final class KsmKeyLocationInfo { * Builder of KsmKeyLocationInfo. */ public static class Builder { - private String containerName; - private String blockID; + private BlockID blockID; private boolean shouldCreateContainer; private long length; private long offset; - public Builder setContainerName(String container) { - this.containerName = container; - return this; - } - - public Builder setBlockID(String block) { - this.blockID = block; + public Builder setBlockID(BlockID blockId) { + this.blockID = blockId; return this; } @@ -107,15 +102,14 @@ public final class KsmKeyLocationInfo { } public KsmKeyLocationInfo build() { - return new KsmKeyLocationInfo(containerName, blockID, + return new KsmKeyLocationInfo(blockID, shouldCreateContainer, length, offset); } } public KeyLocation getProtobuf() { return KeyLocation.newBuilder() - .setContainerName(containerName) - .setBlockID(blockID) + .setBlockID(blockID.getProtobuf()) .setShouldCreateContainer(shouldCreateContainer) .setLength(length) .setOffset(offset) @@ -125,8 +119,7 @@ public final class KsmKeyLocationInfo { public static KsmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) { KsmKeyLocationInfo info = new KsmKeyLocationInfo( - keyLocation.getContainerName(), - keyLocation.getBlockID(), + BlockID.getFromProtobuf(keyLocation.getBlockID()), keyLocation.getShouldCreateContainer(), keyLocation.getLength(), keyLocation.getOffset()); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java index bef65ece33..0facf3ca41 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java @@ -111,7 +111,7 @@ public class KsmKeyLocationInfoGroup { StringBuilder sb = new StringBuilder(); sb.append("version:").append(version).append(" "); for (KsmKeyLocationInfo kli : locationList) { - sb.append(kli.getBlockID()).append(" || "); + sb.append(kli.getLocalID()).append(" || "); } return sb.toString(); } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java index 07856d02e0..8a75928a1c 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Request; import javax.ws.rs.core.UriInfo; +import java.util.Arrays; /** * UserArgs is used to package caller info @@ -118,7 +119,8 @@ public class UserArgs { * @return String[] */ public String[] getGroups() { - return this.groups; + return groups != null ? + Arrays.copyOf(groups, groups.length) : null; } /** @@ -127,7 +129,9 @@ public class UserArgs { * @param groups list of groups */ public void setGroups(String[] groups) { - this.groups = groups; + if (groups != null) { + this.groups = Arrays.copyOf(groups, groups.length); + } } /** diff --git a/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto index a6026f1f99..405c5b0617 100644 --- a/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto +++ b/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto @@ -230,13 +230,12 @@ message KeyArgs { } message KeyLocation { - required string blockID = 1; - required string containerName = 2; - required bool shouldCreateContainer = 3; - required uint64 offset = 4; - required uint64 length = 5; + required hadoop.hdds.BlockID blockID = 1; + required bool shouldCreateContainer = 2; + required uint64 offset = 3; + required uint64 length = 4; // indicated at which version this block gets created. - optional uint64 createVersion = 6; + optional uint64 createVersion = 5; } message KeyLocationList { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java index f74578889d..bedd5c453b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java @@ -16,7 +16,7 @@ */ package org.apache.hadoop.hdds.scm.container; -import org.apache.commons.lang.RandomStringUtils; +import com.google.common.primitives.Longs; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConsts; @@ -31,6 +31,8 @@ import org.junit.Test; import java.io.IOException; import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; import java.util.NavigableSet; import java.util.Random; @@ -69,15 +71,14 @@ public class TestContainerStateManager { @Test public void testAllocateContainer() throws IOException { // Allocate a container and verify the container info - String container1 = "container" + RandomStringUtils.randomNumeric(5); - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container1, containerOwner); + xceiverClientManager.getFactor(), containerOwner); ContainerInfo info = containerStateManager .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED); - Assert.assertEquals(container1, info.getContainerName()); + Assert.assertEquals(container1.getContainerID(), info.getContainerID()); Assert.assertEquals(OzoneConsts.GB * 3, info.getAllocatedBytes()); Assert.assertEquals(containerOwner, info.getOwner()); Assert.assertEquals(xceiverClientManager.getType(), @@ -87,28 +88,31 @@ public class TestContainerStateManager { Assert.assertEquals(HddsProtos.LifeCycleState.ALLOCATED, info.getState()); // Check there are two containers in ALLOCATED state after allocation - String container2 = "container" + RandomStringUtils.randomNumeric(5); - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container2 = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container2, containerOwner); + xceiverClientManager.getFactor(), containerOwner); int numContainers = containerStateManager .getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED).size(); + Assert.assertNotEquals(container1.getContainerID(), container2.getContainerID()); Assert.assertEquals(2, numContainers); } @Test public void testContainerStateManagerRestart() throws IOException { // Allocate 5 containers in ALLOCATED state and 5 in CREATING state - String cname = "container" + RandomStringUtils.randomNumeric(5); + + List<ContainerInfo> containers = new ArrayList<>(); for (int i = 0; i < 10; i++) { - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), cname + i, containerOwner); + xceiverClientManager.getFactor(), containerOwner); + containers.add(container); if (i >= 5) { scm.getScmContainerManager() - .updateContainerState(cname + i, HddsProtos.LifeCycleEvent.CREATE); + .updateContainerState(container.getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); } } @@ -117,48 +121,46 @@ public class TestContainerStateManager { ContainerStateManager stateManager = new ContainerStateManager(conf, scmContainerMapping ); - int containers = stateManager + int matchCount = stateManager .getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED).size(); - Assert.assertEquals(5, containers); - containers = stateManager.getMatchingContainerIDs(containerOwner, + Assert.assertEquals(5, matchCount); + matchCount = stateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CREATING).size(); - Assert.assertEquals(5, containers); + Assert.assertEquals(5, matchCount); } @Test public void testGetMatchingContainer() throws IOException { - String container1 = "container-01234"; - scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container1, containerOwner); - scmContainerMapping.updateContainerState(container1, + ContainerInfo container1 = scm.getClientProtocolServer(). + allocateContainer(xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); + scmContainerMapping.updateContainerState(container1.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container1, + scmContainerMapping.updateContainerState(container1.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); - String container2 = "container-56789"; - scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container2, containerOwner); + ContainerInfo container2 = scm.getClientProtocolServer(). + allocateContainer(xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); ContainerInfo info = containerStateManager .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN); - Assert.assertEquals(container1, info.getContainerName()); + Assert.assertEquals(container1.getContainerID(), info.getContainerID()); info = containerStateManager .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED); - Assert.assertEquals(container2, info.getContainerName()); + Assert.assertEquals(container2.getContainerID(), info.getContainerID()); - scmContainerMapping.updateContainerState(container2, + scmContainerMapping.updateContainerState(container2.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container2, + scmContainerMapping.updateContainerState(container2.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); // space has already been allocated in container1, now container 2 should @@ -167,7 +169,7 @@ public class TestContainerStateManager { .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN); - Assert.assertEquals(container2, info.getContainerName()); + Assert.assertEquals(container2.getContainerID(), info.getContainerID()); } @Test @@ -181,23 +183,22 @@ public class TestContainerStateManager { // Allocate container1 and update its state from ALLOCATED -> CREATING -> // OPEN -> CLOSING -> CLOSED -> DELETING -> DELETED - String container1 = "container" + RandomStringUtils.randomNumeric(5); - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container1, containerOwner); + xceiverClientManager.getFactor(), containerOwner); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED).size(); Assert.assertEquals(1, containers); - scmContainerMapping.updateContainerState(container1, + scmContainerMapping.updateContainerState(container1.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CREATING).size(); Assert.assertEquals(1, containers); - scmContainerMapping.updateContainerState(container1, + scmContainerMapping.updateContainerState(container1.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -205,28 +206,32 @@ public class TestContainerStateManager { Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1, HddsProtos.LifeCycleEvent.FINALIZE); + .updateContainerState(container1.getContainerID(), + HddsProtos.LifeCycleEvent.FINALIZE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CLOSING).size(); Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1, HddsProtos.LifeCycleEvent.CLOSE); + .updateContainerState(container1.getContainerID(), + HddsProtos.LifeCycleEvent.CLOSE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CLOSED).size(); Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1, HddsProtos.LifeCycleEvent.DELETE); + .updateContainerState(container1.getContainerID(), + HddsProtos.LifeCycleEvent.DELETE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.DELETING).size(); Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1, HddsProtos.LifeCycleEvent.CLEANUP); + .updateContainerState(container1.getContainerID(), + HddsProtos.LifeCycleEvent.CLEANUP); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.DELETED).size(); @@ -234,14 +239,14 @@ public class TestContainerStateManager { // Allocate container1 and update its state from ALLOCATED -> CREATING -> // DELETING - String container2 = "container" + RandomStringUtils.randomNumeric(5); - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container2 = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container2, containerOwner); - scmContainerMapping.updateContainerState(container2, + xceiverClientManager.getFactor(), containerOwner); + scmContainerMapping.updateContainerState(container2.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); scmContainerMapping - .updateContainerState(container2, HddsProtos.LifeCycleEvent.TIMEOUT); + .updateContainerState(container2.getContainerID(), + HddsProtos.LifeCycleEvent.TIMEOUT); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.DELETING).size(); @@ -249,18 +254,18 @@ public class TestContainerStateManager { // Allocate container1 and update its state from ALLOCATED -> CREATING -> // OPEN -> CLOSING -> CLOSED - String container3 = "container" + RandomStringUtils.randomNumeric(5); - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container3 = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container3, containerOwner); - scmContainerMapping.updateContainerState(container3, + xceiverClientManager.getFactor(), containerOwner); + scmContainerMapping.updateContainerState(container3.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container3, + scmContainerMapping.updateContainerState(container3.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); - scmContainerMapping.updateContainerState(container3, + scmContainerMapping.updateContainerState(container3.getContainerID(), HddsProtos.LifeCycleEvent.FINALIZE); scmContainerMapping - .updateContainerState(container3, HddsProtos.LifeCycleEvent.CLOSE); + .updateContainerState(container3.getContainerID(), + HddsProtos.LifeCycleEvent.CLOSE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CLOSED).size(); @@ -269,13 +274,12 @@ public class TestContainerStateManager { @Test public void testUpdatingAllocatedBytes() throws Exception { - String container1 = "container" + RandomStringUtils.randomNumeric(5); - scm.getClientProtocolServer().allocateContainer( + ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), container1, containerOwner); - scmContainerMapping.updateContainerState(container1, + xceiverClientManager.getFactor(), containerOwner); + scmContainerMapping.updateContainerState(container1.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container1, + scmContainerMapping.updateContainerState(container1.getContainerID(), HddsProtos.LifeCycleEvent.CREATED); Random ran = new Random(); @@ -288,7 +292,7 @@ public class TestContainerStateManager { .getMatchingContainer(size, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN); - Assert.assertEquals(container1, info.getContainerName()); + Assert.assertEquals(container1.getContainerID(), info.getContainerID()); ContainerMapping containerMapping = (ContainerMapping)scmContainerMapping; @@ -296,10 +300,10 @@ public class TestContainerStateManager { // to disk containerMapping.flushContainerInfo(); - Charset utf8 = Charset.forName("UTF-8"); // the persisted value should always be equal to allocated size. byte[] containerBytes = - containerMapping.getContainerStore().get(container1.getBytes(utf8)); + containerMapping.getContainerStore().get( + Longs.toByteArray(container1.getContainerID())); HddsProtos.SCMContainerInfo infoProto = HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes); ContainerInfo currentInfo = ContainerInfo.fromProtobuf(infoProto); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java index 20579fd0a7..d4c9d4f713 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -77,11 +78,12 @@ public class TestContainerOperations { */ @Test public void testCreate() throws Exception { - Pipeline pipeline0 = storageClient.createContainer(HddsProtos + ContainerInfo container = storageClient.createContainer(HddsProtos .ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor - .ONE, "container0", "OZONE"); - assertEquals("container0", pipeline0.getContainerName()); - + .ONE, "OZONE"); + assertEquals(container.getContainerID(), + storageClient.getContainer(container.getContainerID()). + getContainerID()); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java index 6755e34a13..29238cf536 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java @@ -91,7 +91,6 @@ public class TestMiniOzoneCluster { assertEquals(numberOfNodes, datanodes.size()); for(HddsDatanodeService dn : datanodes) { // Create a single member pipe line - String containerName = OzoneUtils.getRequestID(); DatanodeDetails datanodeDetails = dn.getDatanodeDetails(); final PipelineChannel pipelineChannel = new PipelineChannel(datanodeDetails.getUuidString(), @@ -99,7 +98,7 @@ public class TestMiniOzoneCluster { HddsProtos.ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor.ONE, "test"); pipelineChannel.addMember(datanodeDetails); - Pipeline pipeline = new Pipeline(containerName, pipelineChannel); + Pipeline pipeline = new Pipeline(pipelineChannel); // Verify client is able to connect to the container try (XceiverClient client = new XceiverClient(pipeline, conf)){ diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java index 5a5a08b0cc..78ea5e148c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java @@ -20,10 +20,13 @@ package org.apache.hadoop.ozone; import static org.junit.Assert.fail; import java.io.IOException; -import org.apache.commons.lang.RandomStringUtils; +import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer; import org.apache.hadoop.hdds.scm.server.SCMStorage; import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -38,7 +41,6 @@ import org.apache.hadoop.hdds.scm.block.DeletedBlockLog; import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.XceiverClientManager; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.ScmInfo; import org.junit.Rule; import org.junit.Assert; @@ -113,7 +115,8 @@ public class TestStorageContainerManager { .thenReturn(fakeUser); try { - mockScm.getClientProtocolServer().deleteContainer("container1"); + mockScm.getClientProtocolServer().deleteContainer( + ContainerTestHelper.getTestContainerID()); fail("Operation should fail, expecting an IOException here."); } catch (Exception e) { if (expectPermissionDenied) { @@ -127,35 +130,34 @@ public class TestStorageContainerManager { } try { - Pipeline pipeLine2 = mockScm.getClientProtocolServer() + ContainerInfo container2 = mockScm.getClientProtocolServer() .allocateContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, "container2", "OZONE"); + HddsProtos.ReplicationFactor.ONE, "OZONE"); if (expectPermissionDenied) { fail("Operation should fail, expecting an IOException here."); } else { - Assert.assertEquals("container2", pipeLine2.getContainerName()); + Assert.assertEquals(1, container2.getPipeline().getMachines().size()); } } catch (Exception e) { verifyPermissionDeniedException(e, fakeUser); } try { - Pipeline pipeLine3 = mockScm.getClientProtocolServer() + ContainerInfo container3 = mockScm.getClientProtocolServer() .allocateContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, "container3", "OZONE"); - + HddsProtos.ReplicationFactor.ONE, "OZONE"); if (expectPermissionDenied) { fail("Operation should fail, expecting an IOException here."); } else { - Assert.assertEquals("container3", pipeLine3.getContainerName()); - Assert.assertEquals(1, pipeLine3.getMachines().size()); + Assert.assertEquals(1, container3.getPipeline().getMachines().size()); } } catch (Exception e) { verifyPermissionDeniedException(e, fakeUser); } try { - mockScm.getClientProtocolServer().getContainer("container4"); + mockScm.getClientProtocolServer().getContainer( + ContainerTestHelper.getTestContainerID()); fail("Operation should fail, expecting an IOException here."); } catch (Exception e) { if (expectPermissionDenied) { @@ -210,9 +212,9 @@ public class TestStorageContainerManager { new TestStorageContainerManagerHelper(cluster, conf); Map<String, KsmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096); - Map<String, List<String>> containerBlocks = createDeleteTXLog(delLog, + Map<Long, List<Long>> containerBlocks = createDeleteTXLog(delLog, keyLocations, helper); - Set<String> containerNames = containerBlocks.keySet(); + Set<Long> containerIDs = containerBlocks.keySet(); // Verify a few TX gets created in the TX log. Assert.assertTrue(delLog.getNumOfValidTransactions() > 0); @@ -229,16 +231,16 @@ public class TestStorageContainerManager { return false; } }, 1000, 10000); - Assert.assertTrue(helper.getAllBlocks(containerNames).isEmpty()); + Assert.assertTrue(helper.getAllBlocks(containerIDs).isEmpty()); // Continue the work, add some TXs that with known container names, // but unknown block IDs. - for (String containerName : containerBlocks.keySet()) { + for (Long containerID : containerBlocks.keySet()) { // Add 2 TXs per container. - delLog.addTransaction(containerName, - Collections.singletonList(RandomStringUtils.randomAlphabetic(5))); - delLog.addTransaction(containerName, - Collections.singletonList(RandomStringUtils.randomAlphabetic(5))); + delLog.addTransaction(containerID, + Collections.singletonList(RandomUtils.nextLong())); + delLog.addTransaction(containerID, + Collections.singletonList(RandomUtils.nextLong())); } // Verify a few TX gets created in the TX log. @@ -319,16 +321,16 @@ public class TestStorageContainerManager { }, 500, 10000); } - private Map<String, List<String>> createDeleteTXLog(DeletedBlockLog delLog, + private Map<Long, List<Long>> createDeleteTXLog(DeletedBlockLog delLog, Map<String, KsmKeyInfo> keyLocations, TestStorageContainerManagerHelper helper) throws IOException { // These keys will be written into a bunch of containers, // gets a set of container names, verify container containerBlocks // on datanodes. - Set<String> containerNames = new HashSet<>(); + Set<Long> containerNames = new HashSet<>(); for (Map.Entry<String, KsmKeyInfo> entry : keyLocations.entrySet()) { entry.getValue().getLatestVersionLocations().getLocationList() - .forEach(loc -> containerNames.add(loc.getContainerName())); + .forEach(loc -> containerNames.add(loc.getContainerID())); } // Total number of containerBlocks of these containers should be equal to @@ -342,22 +344,22 @@ public class TestStorageContainerManager { helper.getAllBlocks(containerNames).size()); // Create a deletion TX for each key. - Map<String, List<String>> containerBlocks = Maps.newHashMap(); + Map<Long, List<Long>> containerBlocks = Maps.newHashMap(); for (KsmKeyInfo info : keyLocations.values()) { List<KsmKeyLocationInfo> list = info.getLatestVersionLocations().getLocationList(); list.forEach(location -> { - if (containerBlocks.containsKey(location.getContainerName())) { - containerBlocks.get(location.getContainerName()) - .add(location.getBlockID()); + if (containerBlocks.containsKey(location.getContainerID())) { + containerBlocks.get(location.getContainerID()) + .add(location.getBlockID().getLocalID()); } else { - List<String> blks = Lists.newArrayList(); - blks.add(location.getBlockID()); - containerBlocks.put(location.getContainerName(), blks); + List<Long> blks = Lists.newArrayList(); + blks.add(location.getBlockID().getLocalID()); + containerBlocks.put(location.getContainerID(), blks); } }); } - for (Map.Entry<String, List<String>> tx : containerBlocks.entrySet()) { + for (Map.Entry<Long, List<Long>> tx : containerBlocks.entrySet()) { delLog.addTransaction(tx.getKey(), tx.getValue()); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java index 991701886f..da87d7a0f9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java @@ -22,6 +22,7 @@ import com.google.common.collect.Sets; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -36,7 +37,6 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs; import org.apache.hadoop.ozone.web.handlers.VolumeArgs; import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.utils.OzoneUtils; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter; import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter; import org.apache.hadoop.utils.MetadataStore; @@ -114,10 +114,10 @@ public class TestStorageContainerManagerHelper { return keyLocationMap; } - public List<String> getPendingDeletionBlocks(String containerName) + public List<String> getPendingDeletionBlocks(Long containerID) throws IOException { List<String> pendingDeletionBlocks = Lists.newArrayList(); - MetadataStore meta = getContainerMetadata(containerName); + MetadataStore meta = getContainerMetadata(containerID); KeyPrefixFilter filter = new KeyPrefixFilter(OzoneConsts.DELETING_KEY_PREFIX); List<Map.Entry<byte[], byte[]>> kvs = meta @@ -130,18 +130,18 @@ public class TestStorageContainerManagerHelper { return pendingDeletionBlocks; } - public List<String> getAllBlocks(Set<String> containerNames) + public List<Long> getAllBlocks(Set<Long> containerIDs) throws IOException { - List<String> allBlocks = Lists.newArrayList(); - for (String containerName : containerNames) { - allBlocks.addAll(getAllBlocks(containerName)); + List<Long> allBlocks = Lists.newArrayList(); + for (Long containerID : containerIDs) { + allBlocks.addAll(getAllBlocks(containerID)); } return allBlocks; } - public List<String> getAllBlocks(String containerName) throws IOException { - List<String> allBlocks = Lists.newArrayList(); - MetadataStore meta = getContainerMetadata(containerName); + public List<Long> getAllBlocks(Long containeID) throws IOException { + List<Long> allBlocks = Lists.newArrayList(); + MetadataStore meta = getContainerMetadata(containeID); MetadataKeyFilter filter = (preKey, currentKey, nextKey) -> !DFSUtil.bytes2String(currentKey) .startsWith(OzoneConsts.DELETING_KEY_PREFIX); @@ -149,20 +149,21 @@ public class TestStorageContainerManagerHelper { meta.getRangeKVs(null, Integer.MAX_VALUE, filter); kvs.forEach(entry -> { String key = DFSUtil.bytes2String(entry.getKey()); - allBlocks.add(key.replace(OzoneConsts.DELETING_KEY_PREFIX, "")); + key.replace(OzoneConsts.DELETING_KEY_PREFIX, ""); + allBlocks.add(Long.parseLong(key)); }); return allBlocks; } - private MetadataStore getContainerMetadata(String containerName) + private MetadataStore getContainerMetadata(Long containerID) throws IOException { - Pipeline pipeline = cluster.getStorageContainerManager() - .getClientProtocolServer().getContainer(containerName); - DatanodeDetails leadDN = pipeline.getLeader(); + ContainerInfo container = cluster.getStorageContainerManager() + .getClientProtocolServer().getContainer(containerID); + DatanodeDetails leadDN = container.getPipeline().getLeader(); OzoneContainer containerServer = getContainerServerByDatanodeUuid(leadDN.getUuidString()); ContainerData containerData = containerServer.getContainerManager() - .readContainer(containerName); + .readContainer(containerID); return KeyUtils.getDB(containerData, conf); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java index 32a70a25cd..c9a25e55a5 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java @@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.client.rpc; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConfigKeys; @@ -45,7 +46,6 @@ import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB. StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.util.Time; @@ -388,10 +388,10 @@ public class TestOzoneRpcClient { KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs); for (KsmKeyLocationInfo info: keyInfo.getLatestVersionLocations().getLocationList()) { - Pipeline pipeline = - storageContainerLocationClient.getContainer(info.getContainerName()); - if ((pipeline.getFactor() != replicationFactor) || - (pipeline.getType() != replicationType)) { + ContainerInfo container = + storageContainerLocationClient.getContainer(info.getContainerID()); + if ((container.getPipeline().getFactor() != replicationFactor) || + (container.getPipeline().getType() != replicationType)) { return false; } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index acab0b2e0e..bcd08d70bc 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -21,12 +21,14 @@ package org.apache.hadoop.ozone.container; import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import org.apache.commons.codec.binary.Hex; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos .ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos .ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; @@ -39,6 +41,7 @@ import org.apache.hadoop.ozone.container.common.helpers.KeyData; import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.Time; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -79,9 +82,9 @@ public final class ContainerTestHelper { * @return Pipeline with single node in it. * @throws IOException */ - public static Pipeline createSingleNodePipeline(String containerName) throws + public static Pipeline createSingleNodePipeline() throws IOException { - return createPipeline(containerName, 1); + return createPipeline(1); } public static String createLocalAddress() throws IOException { @@ -111,19 +114,18 @@ public final class ContainerTestHelper { * @return Pipeline with single node in it. * @throws IOException */ - public static Pipeline createPipeline(String containerName, int numNodes) + public static Pipeline createPipeline(int numNodes) throws IOException { Preconditions.checkArgument(numNodes >= 1); final List<DatanodeDetails> ids = new ArrayList<>(numNodes); for(int i = 0; i < numNodes; i++) { ids.add(createDatanodeDetails()); } - return createPipeline(containerName, ids); + return createPipeline(ids); } public static Pipeline createPipeline( - String containerName, Iterable<DatanodeDetails> ids) - throws IOException { + Iterable<DatanodeDetails> ids) throws IOException { Objects.requireNonNull(ids, "ids == null"); final Iterator<DatanodeDetails> i = ids.iterator(); Preconditions.checkArgument(i.hasNext()); @@ -136,21 +138,21 @@ public final class ContainerTestHelper { for(; i.hasNext();) { pipelineChannel.addMember(i.next()); } - return new Pipeline(containerName, pipelineChannel); + return new Pipeline(pipelineChannel); } /** * Creates a ChunkInfo for testing. * - * @param keyName - Name of the key + * @param keyID - ID of the key * @param seqNo - Chunk number. * @return ChunkInfo * @throws IOException */ - public static ChunkInfo getChunk(String keyName, int seqNo, long offset, + public static ChunkInfo getChunk(long keyID, int seqNo, long offset, long len) throws IOException { - ChunkInfo info = new ChunkInfo(String.format("%s.data.%d", keyName, + ChunkInfo info = new ChunkInfo(String.format("%d.data.%d", keyID, seqNo), offset, len); return info; } @@ -185,29 +187,27 @@ public final class ContainerTestHelper { * Returns a writeChunk Request. * * @param pipeline - A set of machines where this container lives. - * @param containerName - Name of the container. - * @param keyName - Name of the Key this chunk is part of. + * @param blockID - Block ID of the chunk. * @param datalen - Length of data. * @return ContainerCommandRequestProto * @throws IOException * @throws NoSuchAlgorithmException */ public static ContainerCommandRequestProto getWriteChunkRequest( - Pipeline pipeline, String containerName, String keyName, int datalen) + Pipeline pipeline, BlockID blockID, int datalen) throws IOException, NoSuchAlgorithmException { - LOG.trace("writeChunk {} (key={}) to pipeline=", - datalen, keyName, pipeline); + LOG.trace("writeChunk {} (blockID={}) to pipeline=", + datalen, blockID, pipeline); ContainerProtos.WriteChunkRequestProto.Builder writeRequest = ContainerProtos.WriteChunkRequestProto .newBuilder(); Pipeline newPipeline = - new Pipeline(containerName, pipeline.getPipelineChannel()); - writeRequest.setPipeline(newPipeline.getProtobufMessage()); - writeRequest.setKeyName(keyName); + new Pipeline(pipeline.getPipelineChannel()); + writeRequest.setBlockID(blockID.getProtobuf()); byte[] data = getData(datalen); - ChunkInfo info = getChunk(keyName, 0, 0, datalen); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen); setDataChecksum(info, data); writeRequest.setChunkData(info.getProtoBufMessage()); @@ -227,29 +227,26 @@ public final class ContainerTestHelper { * Returns PutSmallFile Request that we can send to the container. * * @param pipeline - Pipeline - * @param containerName - ContainerName. - * @param keyName - KeyName + * @param blockID - Block ID of the small file. * @param dataLen - Number of bytes in the data * @return ContainerCommandRequestProto */ public static ContainerCommandRequestProto getWriteSmallFileRequest( - Pipeline pipeline, String containerName, String keyName, int dataLen) + Pipeline pipeline, BlockID blockID, int dataLen) throws Exception { ContainerProtos.PutSmallFileRequestProto.Builder smallFileRequest = ContainerProtos.PutSmallFileRequestProto.newBuilder(); Pipeline newPipeline = - new Pipeline(containerName, pipeline.getPipelineChannel()); + new Pipeline(pipeline.getPipelineChannel()); byte[] data = getData(dataLen); - ChunkInfo info = getChunk(keyName, 0, 0, dataLen); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, dataLen); setDataChecksum(info, data); ContainerProtos.PutKeyRequestProto.Builder putRequest = ContainerProtos.PutKeyRequestProto.newBuilder(); - putRequest.setPipeline(newPipeline.getProtobufMessage()); - KeyData keyData = new KeyData(containerName, keyName); - + KeyData keyData = new KeyData(blockID); List<ContainerProtos.ChunkInfo> newList = new LinkedList<>(); newList.add(info.getProtoBufMessage()); keyData.setChunks(newList); @@ -270,12 +267,11 @@ public final class ContainerTestHelper { public static ContainerCommandRequestProto getReadSmallFileRequest( - ContainerProtos.PutKeyRequestProto putKey) + Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKey) throws Exception { ContainerProtos.GetSmallFileRequestProto.Builder smallFileRequest = ContainerProtos.GetSmallFileRequestProto.newBuilder(); - Pipeline pipeline = Pipeline.getFromProtoBuf(putKey.getPipeline()); - ContainerCommandRequestProto getKey = getKeyRequest(putKey); + ContainerCommandRequestProto getKey = getKeyRequest(pipeline, putKey); smallFileRequest.setKey(getKey.getGetKey()); ContainerCommandRequestProto.Builder request = @@ -290,23 +286,21 @@ public final class ContainerTestHelper { /** * Returns a read Request. * + * @param pipeline pipeline. * @param request writeChunkRequest. * @return Request. * @throws IOException * @throws NoSuchAlgorithmException */ public static ContainerCommandRequestProto getReadChunkRequest( - ContainerProtos.WriteChunkRequestProto request) + Pipeline pipeline, ContainerProtos.WriteChunkRequestProto request) throws IOException, NoSuchAlgorithmException { - LOG.trace("readChunk key={} from pipeline={}", - request.getKeyName(), request.getPipeline()); + LOG.trace("readChunk blockID={} from pipeline={}", + request.getBlockID(), pipeline); ContainerProtos.ReadChunkRequestProto.Builder readRequest = ContainerProtos.ReadChunkRequestProto.newBuilder(); - Pipeline pipeline = Pipeline.getFromProtoBuf(request.getPipeline()); - readRequest.setPipeline(request.getPipeline()); - - readRequest.setKeyName(request.getKeyName()); + readRequest.setBlockID(request.getBlockID()); readRequest.setChunkData(request.getChunkData()); ContainerCommandRequestProto.Builder newRequest = @@ -321,25 +315,25 @@ public final class ContainerTestHelper { /** * Returns a delete Request. * + * @param pipeline pipeline. * @param writeRequest - write request * @return request * @throws IOException * @throws NoSuchAlgorithmException */ public static ContainerCommandRequestProto getDeleteChunkRequest( - ContainerProtos.WriteChunkRequestProto writeRequest) + Pipeline pipeline, ContainerProtos.WriteChunkRequestProto writeRequest) throws IOException, NoSuchAlgorithmException { - LOG.trace("deleteChunk key={} from pipeline={}", - writeRequest.getKeyName(), writeRequest.getPipeline()); - Pipeline pipeline = Pipeline.getFromProtoBuf(writeRequest.getPipeline()); + LOG.trace("deleteChunk blockID={} from pipeline={}", + writeRequest.getBlockID(), pipeline); + ContainerProtos.DeleteChunkRequestProto.Builder deleteRequest = ContainerProtos.DeleteChunkRequestProto .newBuilder(); - deleteRequest.setPipeline(writeRequest.getPipeline()); deleteRequest.setChunkData(writeRequest.getChunkData()); - deleteRequest.setKeyName(writeRequest.getKeyName()); + deleteRequest.setBlockID(writeRequest.getBlockID()); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder(); @@ -357,19 +351,17 @@ public final class ContainerTestHelper { * @return ContainerCommandRequestProto. */ public static ContainerCommandRequestProto getCreateContainerRequest( - String containerName, Pipeline pipeline) throws IOException { - LOG.trace("addContainer: {}", containerName); + long containerID, Pipeline pipeline) throws IOException { + LOG.trace("addContainer: {}", containerID); ContainerProtos.CreateContainerRequestProto.Builder createRequest = ContainerProtos.CreateContainerRequestProto .newBuilder(); ContainerProtos.ContainerData.Builder containerData = ContainerProtos .ContainerData.newBuilder(); - containerData.setName(containerName); - createRequest.setPipeline( - ContainerTestHelper.createSingleNodePipeline(containerName) - .getProtobufMessage()); + containerData.setContainerID(containerID); createRequest.setContainerData(containerData.build()); + createRequest.setPipeline(pipeline.getProtobufMessage()); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder(); @@ -386,18 +378,18 @@ public final class ContainerTestHelper { * Creates a container data based on the given meta data, * and request to update an existing container with it. * - * @param containerName + * @param containerID * @param metaData * @return * @throws IOException */ public static ContainerCommandRequestProto getUpdateContainerRequest( - String containerName, Map<String, String> metaData) throws IOException { + long containerID, Map<String, String> metaData) throws IOException { ContainerProtos.UpdateContainerRequestProto.Builder updateRequestBuilder = ContainerProtos.UpdateContainerRequestProto.newBuilder(); ContainerProtos.ContainerData.Builder containerData = ContainerProtos .ContainerData.newBuilder(); - containerData.setName(containerName); + containerData.setContainerID(containerID); String[] keys = metaData.keySet().toArray(new String[]{}); for(int i=0; i<keys.length; i++) { KeyValue.Builder kvBuilder = KeyValue.newBuilder(); @@ -406,7 +398,7 @@ public final class ContainerTestHelper { containerData.addMetadata(i, kvBuilder.build()); } Pipeline pipeline = - ContainerTestHelper.createSingleNodePipeline(containerName); + ContainerTestHelper.createSingleNodePipeline(); updateRequestBuilder.setPipeline(pipeline.getProtobufMessage()); updateRequestBuilder.setContainerData(containerData.build()); @@ -440,22 +432,20 @@ public final class ContainerTestHelper { /** * Returns the PutKeyRequest for test purpose. - * + * @param pipeline - pipeline. * @param writeRequest - Write Chunk Request. * @return - Request */ public static ContainerCommandRequestProto getPutKeyRequest( - ContainerProtos.WriteChunkRequestProto writeRequest) { + Pipeline pipeline, ContainerProtos.WriteChunkRequestProto writeRequest) { LOG.trace("putKey: {} to pipeline={}", - writeRequest.getKeyName(), writeRequest.getPipeline()); + writeRequest.getBlockID()); - Pipeline pipeline = Pipeline.getFromProtoBuf(writeRequest.getPipeline()); ContainerProtos.PutKeyRequestProto.Builder putRequest = ContainerProtos.PutKeyRequestProto.newBuilder(); - putRequest.setPipeline(writeRequest.getPipeline()); - KeyData keyData = new KeyData(writeRequest.getPipeline().getContainerName(), - writeRequest.getKeyName()); + KeyData keyData = new KeyData( + BlockID.getFromProtobuf(writeRequest.getBlockID())); List<ContainerProtos.ChunkInfo> newList = new LinkedList<>(); newList.add(writeRequest.getChunkData()); keyData.setChunks(newList); @@ -472,24 +462,22 @@ public final class ContainerTestHelper { /** * Gets a GetKeyRequest for test purpose. - * + * @param pipeline - pipeline * @param putKeyRequest - putKeyRequest. * @return - Request + * immediately. */ public static ContainerCommandRequestProto getKeyRequest( - ContainerProtos.PutKeyRequestProto putKeyRequest) { - LOG.trace("getKey: name={} from pipeline={}", - putKeyRequest.getKeyData().getName(), putKeyRequest.getPipeline()); - Pipeline pipeline = Pipeline.getFromProtoBuf(putKeyRequest.getPipeline()); + Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKeyRequest) { + HddsProtos.BlockID blockID = putKeyRequest.getKeyData().getBlockID(); + LOG.trace("getKey: blockID={}", blockID); ContainerProtos.GetKeyRequestProto.Builder getRequest = ContainerProtos.GetKeyRequestProto.newBuilder(); ContainerProtos.KeyData.Builder keyData = ContainerProtos.KeyData .newBuilder(); - keyData.setContainerName(putKeyRequest.getPipeline().getContainerName()); - keyData.setName(putKeyRequest.getKeyData().getName()); + keyData.setBlockID(blockID); getRequest.setKeyData(keyData); - getRequest.setPipeline(putKeyRequest.getPipeline()); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder(); @@ -517,18 +505,17 @@ public final class ContainerTestHelper { } /** + * @param pipeline - pipeline. * @param putKeyRequest - putKeyRequest. * @return - Request */ public static ContainerCommandRequestProto getDeleteKeyRequest( - ContainerProtos.PutKeyRequestProto putKeyRequest) { - LOG.trace("deleteKey: name={} from pipeline={}", - putKeyRequest.getKeyData().getName(), putKeyRequest.getPipeline()); - Pipeline pipeline = Pipeline.getFromProtoBuf(putKeyRequest.getPipeline()); + Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKeyRequest) { + LOG.trace("deleteKey: name={}", + putKeyRequest.getKeyData().getBlockID()); ContainerProtos.DeleteKeyRequestProto.Builder delRequest = ContainerProtos.DeleteKeyRequestProto.newBuilder(); - delRequest.setPipeline(putKeyRequest.getPipeline()); - delRequest.setName(putKeyRequest.getKeyData().getName()); + delRequest.setBlockID(putKeyRequest.getKeyData().getBlockID()); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.DeleteKey); @@ -541,14 +528,14 @@ public final class ContainerTestHelper { /** * Returns a close container request. * @param pipeline - pipeline + * @param containerID - ID of the container. * @return ContainerCommandRequestProto. */ public static ContainerCommandRequestProto getCloseContainer( - Pipeline pipeline) { - Preconditions.checkNotNull(pipeline); + Pipeline pipeline, long containerID) { ContainerProtos.CloseContainerRequestProto closeRequest = - ContainerProtos.CloseContainerRequestProto.newBuilder().setPipeline( - pipeline.getProtobufMessage()).build(); + ContainerProtos.CloseContainerRequestProto.newBuilder(). + setContainerID(containerID).build(); ContainerProtos.ContainerCommandRequestProto cmd = ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos .Type.CloseContainer).setCloseContainer(closeRequest) @@ -562,19 +549,19 @@ public final class ContainerTestHelper { /** * Returns a simple request without traceId. * @param pipeline - pipeline + * @param containerID - ID of the container. * @return ContainerCommandRequestProto without traceId. */ public static ContainerCommandRequestProto getRequestWithoutTraceId( - Pipeline pipeline) { + Pipeline pipeline, long containerID) { Preconditions.checkNotNull(pipeline); ContainerProtos.CloseContainerRequestProto closeRequest = - ContainerProtos.CloseContainerRequestProto.newBuilder().setPipeline( - pipeline.getProtobufMessage()).build(); + ContainerProtos.CloseContainerRequestProto.newBuilder(). + setContainerID(containerID).build(); ContainerProtos.ContainerCommandRequestProto cmd = ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos .Type.CloseContainer).setCloseContainer(closeRequest) - .setDatanodeUuid( - pipeline.getLeader().getUuidString()) + .setDatanodeUuid(pipeline.getLeader().getUuidString()) .build(); return cmd; } @@ -585,12 +572,12 @@ public final class ContainerTestHelper { * @return ContainerCommandRequestProto. */ public static ContainerCommandRequestProto getDeleteContainer( - Pipeline pipeline, boolean forceDelete) { + Pipeline pipeline, long containerID, boolean forceDelete) { Preconditions.checkNotNull(pipeline); ContainerProtos.DeleteContainerRequestProto deleteRequest = - ContainerProtos.DeleteContainerRequestProto.newBuilder().setName( - pipeline.getContainerName()).setPipeline( - pipeline.getProtobufMessage()).setForceDelete(forceDelete).build(); + ContainerProtos.DeleteContainerRequestProto.newBuilder(). + setContainerID(containerID). + setForceDelete(forceDelete).build(); return ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.DeleteContainer) .setDeleteContainer(deleteRequest) @@ -598,4 +585,23 @@ public final class ContainerTestHelper { .setDatanodeUuid(pipeline.getLeader().getUuidString()) .build(); } + + private static void sleep(long milliseconds) { + try { + Thread.sleep(milliseconds); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + public static BlockID getTestBlockID(long containerID) { + // Add 2ms delay so that localID based on UtcTime + // won't collide. + sleep(2); + return new BlockID(containerID, Time.getUtcTime()); + } + + public static long getTestContainerID() { + return Time.getUtcTime(); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java index 0f8c4578ff..a60da21808 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java @@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.common; import com.google.common.collect.Lists; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; @@ -27,6 +28,7 @@ import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.testutils.BlockDeletingServiceTestImpl; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.KeyData; @@ -35,7 +37,6 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl; import org.apache.hadoop.ozone.container.common.impl.RandomContainerDeletionChoosingPolicy; import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager; import org.apache.hadoop.ozone.container.common.statemachine.background.BlockDeletingService; -import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils.LogCapturer; @@ -128,19 +129,21 @@ public class TestBlockDeletingService { Configuration conf, int numOfContainers, int numOfBlocksPerContainer, int numOfChunksPerBlock, File chunkDir) throws IOException { for (int x = 0; x < numOfContainers; x++) { - String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, new Long(x), conf); - mgr.createContainer(createSingleNodePipeline(containerName), data); - data = mgr.readContainer(containerName); + long containerID = ContainerTestHelper.getTestContainerID(); + ContainerData data = new ContainerData(containerID, conf); + mgr.createContainer(data); + data = mgr.readContainer(containerID); MetadataStore metadata = KeyUtils.getDB(data, conf); for (int j = 0; j<numOfBlocksPerContainer; j++) { - String blockName = containerName + "b" + j; - String deleteStateName = OzoneConsts.DELETING_KEY_PREFIX + blockName; - KeyData kd = new KeyData(containerName, deleteStateName); + BlockID blockID = + ContainerTestHelper.getTestBlockID(containerID); + String deleteStateName = OzoneConsts.DELETING_KEY_PREFIX + + blockID.getLocalID(); + KeyData kd = new KeyData(blockID); List<ContainerProtos.ChunkInfo> chunks = Lists.newArrayList(); for (int k = 0; k<numOfChunksPerBlock; k++) { // offset doesn't matter here - String chunkName = blockName + "_chunk_" + k; + String chunkName = blockID.getLocalID() + "_chunk_" + k; File chunk = new File(chunkDir, chunkName); FileUtils.writeStringToFile(chunk, "a chunk", Charset.defaultCharset()); @@ -200,7 +203,7 @@ public class TestBlockDeletingService { // Ensure 1 container was created List<ContainerData> containerData = Lists.newArrayList(); - containerManager.listContainer(null, 1, "", containerData); + containerManager.listContainer(0L, 1, containerData); Assert.assertEquals(1, containerData.size()); MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf); @@ -286,7 +289,7 @@ public class TestBlockDeletingService { // get container meta data List<ContainerData> containerData = Lists.newArrayList(); - containerManager.listContainer(null, 1, "", containerData); + containerManager.listContainer(0L, 1, containerData); MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf); LogCapturer newLog = LogCapturer.captureLogs(BackgroundService.LOG); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java index 893f2f69dc..331db40af6 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java @@ -93,12 +93,10 @@ public class TestContainerDeletionChoosingPolicy { int numContainers = 10; for (int i = 0; i < numContainers; i++) { - String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, new Long(i), conf); - containerManager.createContainer(createSingleNodePipeline(containerName), - data); + ContainerData data = new ContainerData(new Long(i), conf); + containerManager.createContainer(data); Assert.assertTrue( - containerManager.getContainerMap().containsKey(containerName)); + containerManager.getContainerMap().containsKey(data.getContainerID())); } List<ContainerData> result0 = containerManager @@ -113,8 +111,8 @@ public class TestContainerDeletionChoosingPolicy { boolean hasShuffled = false; for (int i = 0; i < numContainers; i++) { - if (!result1.get(i).getContainerName() - .equals(result2.get(i).getContainerName())) { + if (result1.get(i).getContainerID() + != result2.get(i).getContainerID()) { hasShuffled = true; break; } @@ -144,9 +142,8 @@ public class TestContainerDeletionChoosingPolicy { // create [numContainers + 1] containers for (int i = 0; i <= numContainers; i++) { String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, new Long(i), conf); - containerManager.createContainer(createSingleNodePipeline(containerName), - data); + ContainerData data = new ContainerData(new Long(i), conf); + containerManager.createContainer(data); Assert.assertTrue( containerManager.getContainerMap().containsKey(containerName)); @@ -186,7 +183,7 @@ public class TestContainerDeletionChoosingPolicy { // verify the order of return list int lastCount = Integer.MAX_VALUE; for (ContainerData data : result1) { - int currentCount = name2Count.remove(data.getContainerName()); + int currentCount = name2Count.remove(data.getContainerID()); // previous count should not smaller than next one Assert.assertTrue(currentCount > 0 && currentCount <= lastCount); lastCount = currentCount; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java index fae4c491b0..e51c3f75af 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java @@ -19,8 +19,9 @@ package org.apache.hadoop.ozone.container.common.impl; import org.apache.commons.codec.binary.Hex; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.ozone.OzoneConfigKeys; @@ -28,6 +29,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; @@ -36,6 +38,7 @@ import org.apache.hadoop.ozone.container.common.helpers.KeyData; import org.apache.hadoop.ozone.container.common.helpers.KeyUtils; import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; +import org.apache.hadoop.util.Time; import org.apache.hadoop.utils.MetadataStore; import org.junit.After; import org.junit.AfterClass; @@ -177,19 +180,21 @@ public class TestContainerPersistence { } } + private long getTestContainerID() { + return ContainerTestHelper.getTestContainerID(); + } + @Test public void testCreateContainer() throws Exception { - - String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, containerID++, conf); + long testContainerID = getTestContainerID(); + ContainerData data = new ContainerData(testContainerID, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner)", "bilbo"); - containerManager.createContainer(createSingleNodePipeline(containerName), - data); + containerManager.createContainer(data); Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName)); + .containsKey(testContainerID)); ContainerStatus status = containerManager - .getContainerMap().get(containerName); + .getContainerMap().get(testContainerID); Assert.assertNotNull(status.getContainer()); Assert.assertNotNull(status.getContainer().getContainerPath()); @@ -215,16 +220,14 @@ public class TestContainerPersistence { @Test public void testCreateDuplicateContainer() throws Exception { - String containerName = OzoneUtils.getRequestID(); + long testContainerID = getTestContainerID(); - ContainerData data = new ContainerData(containerName, containerID++, conf); + ContainerData data = new ContainerData(testContainerID, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner)", "bilbo"); - containerManager.createContainer(createSingleNodePipeline(containerName), - data); + containerManager.createContainer(data); try { - containerManager.createContainer(createSingleNodePipeline( - containerName), data); + containerManager.createContainer(data); fail("Expected Exception not thrown."); } catch (IOException ex) { Assert.assertNotNull(ex); @@ -233,85 +236,76 @@ public class TestContainerPersistence { @Test public void testDeleteContainer() throws Exception { - String containerName1 = OzoneUtils.getRequestID(); - String containerName2 = OzoneUtils.getRequestID(); - + long testContainerID1 = getTestContainerID(); + Thread.sleep(100); + long testContainerID2 = getTestContainerID(); - ContainerData data = new ContainerData(containerName1, containerID++, conf); + ContainerData data = new ContainerData(testContainerID1, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner)", "bilbo"); - containerManager.createContainer(createSingleNodePipeline(containerName1), - data); - containerManager.closeContainer(containerName1); + containerManager.createContainer(data); + containerManager.closeContainer(testContainerID1); - data = new ContainerData(containerName2, containerID++, conf); + data = new ContainerData(testContainerID2, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner)", "bilbo"); - containerManager.createContainer(createSingleNodePipeline(containerName2), - data); - containerManager.closeContainer(containerName2); + containerManager.createContainer(data); + containerManager.closeContainer(testContainerID2); Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName1)); + .containsKey(testContainerID1)); Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName2)); + .containsKey(testContainerID2)); - containerManager.deleteContainer(createSingleNodePipeline(containerName1), - containerName1, false); + containerManager.deleteContainer(testContainerID1, false); Assert.assertFalse(containerManager.getContainerMap() - .containsKey(containerName1)); + .containsKey(testContainerID1)); // Let us make sure that we are able to re-use a container name after // delete. - data = new ContainerData(containerName1, containerID++, conf); + data = new ContainerData(testContainerID1, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner)", "bilbo"); - containerManager.createContainer(createSingleNodePipeline(containerName1), - data); - containerManager.closeContainer(containerName1); + containerManager.createContainer(data); + containerManager.closeContainer(testContainerID1); // Assert we still have both containers. Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName1)); + .containsKey(testContainerID1)); Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName2)); + .containsKey(testContainerID2)); // Add some key to a container and then delete. // Delete should fail because the container is no longer empty. - KeyData someKey = new KeyData(containerName1, "someKey"); + BlockID blockID1 = ContainerTestHelper.getTestBlockID(testContainerID1); + KeyData someKey = new KeyData(blockID1); someKey.setChunks(new LinkedList<ContainerProtos.ChunkInfo>()); - keyManager.putKey( - createSingleNodePipeline(containerName1), - someKey); + keyManager.putKey(someKey); exception.expect(StorageContainerException.class); exception.expectMessage( "Container cannot be deleted because it is not empty."); - containerManager.deleteContainer( - createSingleNodePipeline(containerName1), - containerName1, false); + containerManager.deleteContainer(testContainerID1, false); Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName1)); + .containsKey(testContainerID1)); } @Test public void testGetContainerReports() throws Exception{ final int count = 10; - List<String> containerNames = new ArrayList<String>(); + List<Long> containerIDs = new ArrayList<>(); for (int i = 0; i < count; i++) { - String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, containerID++, - conf); - containerManager.createContainer(createSingleNodePipeline(containerName), - data); + long testContainerID = getTestContainerID(); + ContainerData data = new ContainerData(testContainerID, conf); + containerManager.createContainer(data); // Close a bunch of containers. // Put closed container names to a list. if (i%3 == 0) { - containerManager.closeContainer(containerName); - containerNames.add(containerName); + containerManager.closeContainer(testContainerID); + containerIDs.add(testContainerID); } } @@ -319,10 +313,10 @@ public class TestContainerPersistence { List<ContainerData> reports = containerManager.getContainerReports(); Assert.assertEquals(4, reports.size()); for(ContainerData report : reports) { - String actualName = report.getContainerName(); - Assert.assertTrue(containerNames.remove(actualName)); + long actualContainerID = report.getContainerID(); + Assert.assertTrue(containerIDs.remove(actualContainerID)); } - Assert.assertTrue(containerNames.isEmpty()); + Assert.assertTrue(containerIDs.isEmpty()); } /** @@ -336,31 +330,29 @@ public class TestContainerPersistence { final int count = 50; final int step = 5; - Map<String, ContainerData> testMap = new HashMap<>(); + Map<Long, ContainerData> testMap = new HashMap<>(); for (int x = 0; x < count; x++) { - String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, containerID++, - conf); + long testContainerID = getTestContainerID(); + ContainerData data = new ContainerData(testContainerID, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner)", "bilbo"); - containerManager.createContainer(createSingleNodePipeline(containerName), - data); - testMap.put(containerName, data); + containerManager.createContainer(data); + testMap.put(testContainerID, data); } int counter = 0; - String prevKey = ""; + long prevKey = 0; List<ContainerData> results = new LinkedList<>(); while (counter < count) { - containerManager.listContainer(null, step, prevKey, results); + containerManager.listContainer(prevKey, step, results); for (int y = 0; y < results.size(); y++) { - testMap.remove(results.get(y).getContainerName()); + testMap.remove(results.get(y).getContainerID()); } counter += step; - String nextKey = results.get(results.size() - 1).getContainerName(); + long nextKey = results.get(results.size() - 1).getContainerID(); //Assert that container is returning results in a sorted fashion. - Assert.assertTrue(prevKey.compareTo(nextKey) < 0); + Assert.assertTrue(prevKey < nextKey); prevKey = nextKey; results.clear(); } @@ -369,23 +361,22 @@ public class TestContainerPersistence { Assert.assertTrue(testMap.isEmpty()); } - private ChunkInfo writeChunkHelper(String containerName, String keyName, + private ChunkInfo writeChunkHelper(BlockID blockID, Pipeline pipeline) throws IOException, NoSuchAlgorithmException { final int datalen = 1024; - Pipeline newPipeline = - new Pipeline(containerName, pipeline.getPipelineChannel()); - ContainerData cData = new ContainerData(containerName, containerID++, conf); + long testContainerID = blockID.getContainerID(); + ContainerData cData = new ContainerData(testContainerID, conf); cData.addMetadata("VOLUME", "shire"); cData.addMetadata("owner", "bilbo"); if(!containerManager.getContainerMap() - .containsKey(containerName)) { - containerManager.createContainer(newPipeline, cData); + .containsKey(testContainerID)) { + containerManager.createContainer(cData); } - ChunkInfo info = getChunk(keyName, 0, 0, datalen); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen); byte[] data = getData(datalen); setDataChecksum(info, data); - chunkManager.writeChunk(newPipeline, keyName, info, data, COMBINED); + chunkManager.writeChunk(blockID, info, data, COMBINED); return info; } @@ -399,10 +390,10 @@ public class TestContainerPersistence { @Test public void testWriteChunk() throws IOException, NoSuchAlgorithmException { - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); - writeChunkHelper(containerName, keyName, pipeline); + BlockID blockID = ContainerTestHelper. + getTestBlockID(getTestContainerID()); + Pipeline pipeline = createSingleNodePipeline(); + writeChunkHelper(blockID, pipeline); } /** @@ -418,29 +409,30 @@ public class TestContainerPersistence { final int datalen = 1024; final int chunkCount = 1024; - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); + long testContainerID = getTestContainerID(); Map<String, ChunkInfo> fileHashMap = new HashMap<>(); - ContainerData cData = new ContainerData(containerName, containerID++, conf); + ContainerData cData = new ContainerData(testContainerID, conf); cData.addMetadata("VOLUME", "shire"); cData.addMetadata("owner)", "bilbo"); - containerManager.createContainer(pipeline, cData); + containerManager.createContainer(cData); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); + for (int x = 0; x < chunkCount; x++) { - ChunkInfo info = getChunk(keyName, x, 0, datalen); + ChunkInfo info = getChunk(blockID.getLocalID(), x, 0, datalen); byte[] data = getData(datalen); setDataChecksum(info, data); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); - String fileName = String.format("%s.data.%d", keyName, x); + chunkManager.writeChunk(blockID, info, data, COMBINED); + String fileName = String.format("%s.data.%d", blockID.getLocalID(), x); fileHashMap.put(fileName, info); } - ContainerData cNewData = containerManager.readContainer(containerName); + ContainerData cNewData = containerManager.readContainer(testContainerID); Assert.assertNotNull(cNewData); Path dataDir = ContainerUtils.getDataDirectory(cNewData); - String globFormat = String.format("%s.data.*", keyName); + String globFormat = String.format("%s.data.*", blockID.getLocalID()); MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH); // Read chunk via file system and verify. @@ -460,9 +452,9 @@ public class TestContainerPersistence { // Read chunk via ReadChunk call. sha.reset(); for (int x = 0; x < chunkCount; x++) { - String fileName = String.format("%s.data.%d", keyName, x); + String fileName = String.format("%s.data.%d", blockID.getLocalID(), x); ChunkInfo info = fileHashMap.get(fileName); - byte[] data = chunkManager.readChunk(pipeline, keyName, info); + byte[] data = chunkManager.readChunk(blockID, info); sha.update(data); Assert.assertEquals(Hex.encodeHexString(sha.digest()), info.getChecksum()); @@ -482,24 +474,24 @@ public class TestContainerPersistence { final int start = datalen/4; final int length = datalen/2; - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); - ContainerData cData = new ContainerData(containerName, containerID++, conf); + ContainerData cData = new ContainerData(testContainerID, conf); cData.addMetadata("VOLUME", "shire"); cData.addMetadata("owner)", "bilbo"); - containerManager.createContainer(pipeline, cData); - ChunkInfo info = getChunk(keyName, 0, 0, datalen); + containerManager.createContainer(cData); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen); byte[] data = getData(datalen); setDataChecksum(info, data); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); + chunkManager.writeChunk(blockID, info, data, COMBINED); - byte[] readData = chunkManager.readChunk(pipeline, keyName, info); + byte[] readData = chunkManager.readChunk(blockID, info); assertTrue(Arrays.equals(data, readData)); - ChunkInfo info2 = getChunk(keyName, 0, start, length); - byte[] readData2 = chunkManager.readChunk(pipeline, keyName, info2); + ChunkInfo info2 = getChunk(blockID.getLocalID(), 0, start, length); + byte[] readData2 = chunkManager.readChunk(blockID, info2); assertEquals(length, readData2.length); assertTrue(Arrays.equals( Arrays.copyOfRange(data, start, start + length), readData2)); @@ -516,20 +508,21 @@ public class TestContainerPersistence { public void testOverWrite() throws IOException, NoSuchAlgorithmException { final int datalen = 1024; - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); - ContainerData cData = new ContainerData(containerName, containerID++, conf); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); + + ContainerData cData = new ContainerData(testContainerID, conf); cData.addMetadata("VOLUME", "shire"); cData.addMetadata("owner)", "bilbo"); - containerManager.createContainer(pipeline, cData); - ChunkInfo info = getChunk(keyName, 0, 0, datalen); + containerManager.createContainer(cData); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen); byte[] data = getData(datalen); setDataChecksum(info, data); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); + chunkManager.writeChunk(blockID, info, data, COMBINED); try { - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); + chunkManager.writeChunk(blockID, info, data, COMBINED); } catch (IOException ex) { Assert.assertTrue(ex.getCause().getMessage().contains( "Rejecting write chunk request. OverWrite flag required")); @@ -537,11 +530,11 @@ public class TestContainerPersistence { // With the overwrite flag it should work now. info.addMetadata(OzoneConsts.CHUNK_OVERWRITE, "true"); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); - long bytesUsed = containerManager.getBytesUsed(containerName); + chunkManager.writeChunk(blockID, info, data, COMBINED); + long bytesUsed = containerManager.getBytesUsed(testContainerID); Assert.assertEquals(datalen, bytesUsed); - long bytesWrite = containerManager.getWriteBytes(containerName); + long bytesWrite = containerManager.getWriteBytes(testContainerID); Assert.assertEquals(datalen * 2, bytesWrite); } @@ -558,28 +551,28 @@ public class TestContainerPersistence { final int datalen = 1024; final int chunkCount = 1024; - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); - ContainerData cData = new ContainerData(containerName, containerID++, conf); + ContainerData cData = new ContainerData(testContainerID, conf); cData.addMetadata("VOLUME", "shire"); cData.addMetadata("owner)", "bilbo"); - containerManager.createContainer(pipeline, cData); + containerManager.createContainer(cData); MessageDigest oldSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH); for (int x = 0; x < chunkCount; x++) { // we are writing to the same chunk file but at different offsets. long offset = x * datalen; - ChunkInfo info = getChunk(keyName, 0, offset, datalen); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, offset, datalen); byte[] data = getData(datalen); oldSha.update(data); setDataChecksum(info, data); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); + chunkManager.writeChunk(blockID, info, data, COMBINED); } // Request to read the whole data in a single go. - ChunkInfo largeChunk = getChunk(keyName, 0, 0, datalen * chunkCount); - byte[] newdata = chunkManager.readChunk(pipeline, keyName, largeChunk); + ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0, datalen * chunkCount); + byte[] newdata = chunkManager.readChunk(blockID, largeChunk); MessageDigest newSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH); newSha.update(newdata); Assert.assertEquals(Hex.encodeHexString(oldSha.digest()), @@ -596,22 +589,22 @@ public class TestContainerPersistence { public void testDeleteChunk() throws IOException, NoSuchAlgorithmException { final int datalen = 1024; - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); - ContainerData cData = new ContainerData(containerName, containerID++, conf); + ContainerData cData = new ContainerData(testContainerID, conf); cData.addMetadata("VOLUME", "shire"); cData.addMetadata("owner)", "bilbo"); - containerManager.createContainer(pipeline, cData); - ChunkInfo info = getChunk(keyName, 0, 0, datalen); + containerManager.createContainer(cData); + ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen); byte[] data = getData(datalen); setDataChecksum(info, data); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); - chunkManager.deleteChunk(pipeline, keyName, info); + chunkManager.writeChunk(blockID, info, data, COMBINED); + chunkManager.deleteChunk(blockID, info); exception.expect(StorageContainerException.class); exception.expectMessage("Unable to find the chunk file."); - chunkManager.readChunk(pipeline, keyName, info); + chunkManager.readChunk(blockID, info); } /** @@ -622,15 +615,16 @@ public class TestContainerPersistence { */ @Test public void testPutKey() throws IOException, NoSuchAlgorithmException { - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); - ChunkInfo info = writeChunkHelper(containerName, keyName, pipeline); - KeyData keyData = new KeyData(containerName, keyName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); + Pipeline pipeline = createSingleNodePipeline(); + ChunkInfo info = writeChunkHelper(blockID, pipeline); + KeyData keyData = new KeyData(blockID); List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>(); chunkList.add(info.getProtoBufMessage()); keyData.setChunks(chunkList); - keyManager.putKey(pipeline, keyData); + keyManager.putKey(keyData); KeyData readKeyData = keyManager.getKey(keyData); ChunkInfo readChunk = ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(0)); @@ -649,39 +643,40 @@ public class TestContainerPersistence { final int chunkCount = 2; final int datalen = 1024; long totalSize = 0L; - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); + Pipeline pipeline = createSingleNodePipeline(); List<ChunkInfo> chunkList = new LinkedList<>(); - ChunkInfo info = writeChunkHelper(containerName, keyName, pipeline); + ChunkInfo info = writeChunkHelper(blockID, pipeline); totalSize += datalen; chunkList.add(info); for (int x = 1; x < chunkCount; x++) { // with holes in the front (before x * datalen) - info = getChunk(keyName, x, x * datalen, datalen); + info = getChunk(blockID.getLocalID(), x, x * datalen, datalen); byte[] data = getData(datalen); setDataChecksum(info, data); - chunkManager.writeChunk(pipeline, keyName, info, data, COMBINED); + chunkManager.writeChunk(blockID, info, data, COMBINED); totalSize += datalen * (x + 1); chunkList.add(info); } - long bytesUsed = containerManager.getBytesUsed(containerName); + long bytesUsed = containerManager.getBytesUsed(testContainerID); Assert.assertEquals(totalSize, bytesUsed); - long writeBytes = containerManager.getWriteBytes(containerName); + long writeBytes = containerManager.getWriteBytes(testContainerID); Assert.assertEquals(chunkCount * datalen, writeBytes); - long readCount = containerManager.getReadCount(containerName); + long readCount = containerManager.getReadCount(testContainerID); Assert.assertEquals(0, readCount); - long writeCount = containerManager.getWriteCount(containerName); + long writeCount = containerManager.getWriteCount(testContainerID); Assert.assertEquals(chunkCount, writeCount); - KeyData keyData = new KeyData(containerName, keyName); + KeyData keyData = new KeyData(blockID); List<ContainerProtos.ChunkInfo> chunkProtoList = new LinkedList<>(); for (ChunkInfo i : chunkList) { chunkProtoList.add(i.getProtoBufMessage()); } keyData.setChunks(chunkProtoList); - keyManager.putKey(pipeline, keyData); + keyManager.putKey(keyData); KeyData readKeyData = keyManager.getKey(keyData); ChunkInfo lastChunk = chunkList.get(chunkList.size() - 1); ChunkInfo readChunk = @@ -698,16 +693,16 @@ public class TestContainerPersistence { */ @Test public void testDeleteKey() throws IOException, NoSuchAlgorithmException { - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); - ChunkInfo info = writeChunkHelper(containerName, keyName, pipeline); - KeyData keyData = new KeyData(containerName, keyName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID); + Pipeline pipeline = createSingleNodePipeline(); + ChunkInfo info = writeChunkHelper(blockID, pipeline); + KeyData keyData = new KeyData(blockID); List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>(); chunkList.add(info.getProtoBufMessage()); keyData.setChunks(chunkList); - keyManager.putKey(pipeline, keyData); - keyManager.deleteKey(pipeline, keyName); + keyManager.putKey(keyData); + keyManager.deleteKey(blockID); exception.expect(StorageContainerException.class); exception.expectMessage("Unable to find the key."); keyManager.getKey(keyData); @@ -722,19 +717,20 @@ public class TestContainerPersistence { @Test public void testDeleteKeyTwice() throws IOException, NoSuchAlgorithmException { - String containerName = OzoneUtils.getRequestID(); - String keyName = OzoneUtils.getRequestID(); - Pipeline pipeline = createSingleNodePipeline(containerName); - ChunkInfo info = writeChunkHelper(containerName, keyName, pipeline); - KeyData keyData = new KeyData(containerName, keyName); + long testContainerID = getTestContainerID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(testContainerID); + Pipeline pipeline = createSingleNodePipeline(); + ChunkInfo info = writeChunkHelper(blockID, pipeline); + KeyData keyData = new KeyData(blockID); List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>(); chunkList.add(info.getProtoBufMessage()); keyData.setChunks(chunkList); - keyManager.putKey(pipeline, keyData); - keyManager.deleteKey(pipeline, keyName); + keyManager.putKey(keyData); + keyManager.deleteKey(blockID); exception.expect(StorageContainerException.class); exception.expectMessage("Unable to find the key."); - keyManager.deleteKey(pipeline, keyName); + keyManager.deleteKey(blockID); } /** @@ -745,35 +741,30 @@ public class TestContainerPersistence { */ @Test public void testUpdateContainer() throws IOException { - String containerName = OzoneUtils.getRequestID(); - ContainerData data = new ContainerData(containerName, containerID++, conf); + long testContainerID = ContainerTestHelper. + getTestContainerID(); + ContainerData data = new ContainerData(testContainerID, conf); data.addMetadata("VOLUME", "shire"); data.addMetadata("owner", "bilbo"); - containerManager.createContainer( - createSingleNodePipeline(containerName), - data); + containerManager.createContainer(data); File orgContainerFile = containerManager.getContainerFile(data); Assert.assertTrue(orgContainerFile.exists()); - ContainerData newData = new ContainerData(containerName, containerID++, - conf); + ContainerData newData = new ContainerData(testContainerID, conf); newData.addMetadata("VOLUME", "shire_new"); newData.addMetadata("owner", "bilbo_new"); - containerManager.updateContainer( - createSingleNodePipeline(containerName), - containerName, - newData, false); + containerManager.updateContainer(testContainerID, newData, false); Assert.assertEquals(1, containerManager.getContainerMap().size()); Assert.assertTrue(containerManager.getContainerMap() - .containsKey(containerName)); + .containsKey(testContainerID)); // Verify in-memory map ContainerData actualNewData = containerManager.getContainerMap() - .get(containerName).getContainer(); + .get(testContainerID).getContainer(); Assert.assertEquals("shire_new", actualNewData.getAllMetadata().get("VOLUME")); Assert.assertEquals("bilbo_new", @@ -802,23 +793,21 @@ public class TestContainerPersistence { // Delete container file then try to update without force update flag. FileUtil.fullyDelete(newContainerFile); try { - containerManager.updateContainer(createSingleNodePipeline(containerName), - containerName, newData, false); + containerManager.updateContainer(testContainerID, newData, false); } catch (StorageContainerException ex) { Assert.assertEquals("Container file not exists or " - + "corrupted. Name: " + containerName, ex.getMessage()); + + "corrupted. ID: " + testContainerID, ex.getMessage()); } // Update with force flag, it should be success. - newData = new ContainerData(containerName, containerID++, conf); + newData = new ContainerData(testContainerID, conf); newData.addMetadata("VOLUME", "shire_new_1"); newData.addMetadata("owner", "bilbo_new_1"); - containerManager.updateContainer(createSingleNodePipeline(containerName), - containerName, newData, true); + containerManager.updateContainer(testContainerID, newData, true); // Verify in-memory map actualNewData = containerManager.getContainerMap() - .get(containerName).getContainer(); + .get(testContainerID).getContainer(); Assert.assertEquals("shire_new_1", actualNewData.getAllMetadata().get("VOLUME")); Assert.assertEquals("bilbo_new_1", @@ -827,16 +816,14 @@ public class TestContainerPersistence { // Update a non-existing container exception.expect(StorageContainerException.class); exception.expectMessage("Container doesn't exist."); - containerManager.updateContainer( - createSingleNodePipeline("non_exist_container"), - "non_exist_container", newData, false); + containerManager.updateContainer(RandomUtils.nextLong(), + newData, false); } - private KeyData writeKeyHelper(Pipeline pipeline, - String containerName, String keyName) + private KeyData writeKeyHelper(Pipeline pipeline, BlockID blockID) throws IOException, NoSuchAlgorithmException { - ChunkInfo info = writeChunkHelper(containerName, keyName, pipeline); - KeyData keyData = new KeyData(containerName, keyName); + ChunkInfo info = writeChunkHelper(blockID, pipeline); + KeyData keyData = new KeyData(blockID); List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>(); chunkList.add(info.getProtoBufMessage()); keyData.setChunks(chunkList); @@ -845,61 +832,43 @@ public class TestContainerPersistence { @Test public void testListKey() throws Exception { - String containerName = "c0" + RandomStringUtils.randomAlphanumeric(10); - Pipeline pipeline = createSingleNodePipeline(containerName); - List<String> expectedKeys = new ArrayList<String>(); + + long testContainerID = getTestContainerID(); + Pipeline pipeline = createSingleNodePipeline(); + List<BlockID> expectedKeys = new ArrayList<>(); for (int i = 0; i < 10; i++) { - String keyName = "k" + i + "-" + UUID.randomUUID(); - expectedKeys.add(keyName); - KeyData kd = writeKeyHelper(pipeline, containerName, keyName); - keyManager.putKey(pipeline, kd); + BlockID blockID = new BlockID( + testContainerID, i); + expectedKeys.add(blockID); + KeyData kd = writeKeyHelper(pipeline, blockID); + keyManager.putKey(kd); } // List all keys - List<KeyData> result = keyManager.listKey(pipeline, null, null, 100); + List<KeyData> result = keyManager.listKey(testContainerID, 0, 100); Assert.assertEquals(10, result.size()); int index = 0; for (int i = index; i < result.size(); i++) { KeyData data = result.get(i); - Assert.assertEquals(containerName, data.getContainerName()); - Assert.assertEquals(expectedKeys.get(i), data.getKeyName()); + Assert.assertEquals(testContainerID, data.getContainerID()); + Assert.assertEquals(expectedKeys.get(i).getLocalID(), data.getLocalID()); index++; } - // List key with prefix - result = keyManager.listKey(pipeline, "k1", null, 100); - // There is only one key with prefix k1 - Assert.assertEquals(1, result.size()); - Assert.assertEquals(expectedKeys.get(1), result.get(0).getKeyName()); - - // List key with startKey filter - String k6 = expectedKeys.get(6); - result = keyManager.listKey(pipeline, null, k6, 100); + long k6 = expectedKeys.get(6).getLocalID(); + result = keyManager.listKey(testContainerID, k6, 100); Assert.assertEquals(4, result.size()); for (int i = 6; i < 10; i++) { - Assert.assertEquals(expectedKeys.get(i), - result.get(i - 6).getKeyName()); - } - - // List key with both prefix and startKey filter - String k7 = expectedKeys.get(7); - result = keyManager.listKey(pipeline, "k3", k7, 100); - // k3 is after k7, enhance we get an empty result - Assert.assertTrue(result.isEmpty()); - - // Set a pretty small cap for the key count - result = keyManager.listKey(pipeline, null, null, 3); - Assert.assertEquals(3, result.size()); - for (int i = 0; i < 3; i++) { - Assert.assertEquals(expectedKeys.get(i), result.get(i).getKeyName()); + Assert.assertEquals(expectedKeys.get(i).getLocalID(), + result.get(i - 6).getLocalID()); } // Count must be >0 exception.expect(IllegalArgumentException.class); exception.expectMessage("Count must be a positive number."); - keyManager.listKey(pipeline, null, null, -1); + keyManager.listKey(testContainerID, 0, -1); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java index 0034e8e26a..fbe43d72f6 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java @@ -79,32 +79,32 @@ public class TestCloseContainerHandler { cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); - String containerName = ksmKeyLocationInfo.getContainerName(); + long containerID = ksmKeyLocationInfo.getContainerID(); - Assert.assertFalse(isContainerClosed(cluster, containerName)); + Assert.assertFalse(isContainerClosed(cluster, containerID)); DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0) .getDatanodeDetails(); //send the order to close the container cluster.getStorageContainerManager().getScmNodeManager() .addDatanodeCommand(datanodeDetails.getUuid(), - new CloseContainerCommand(containerName)); + new CloseContainerCommand(containerID)); - GenericTestUtils.waitFor(() -> isContainerClosed(cluster, containerName), + GenericTestUtils.waitFor(() -> isContainerClosed(cluster, containerID), 500, 5 * 1000); //double check if it's really closed (waitFor also throws an exception) - Assert.assertTrue(isContainerClosed(cluster, containerName)); + Assert.assertTrue(isContainerClosed(cluster, containerID)); } private Boolean isContainerClosed(MiniOzoneCluster cluster, - String containerName) { + long containerID) { ContainerData containerData; try { containerData = cluster.getHddsDatanodes().get(0) .getDatanodeStateMachine().getContainer().getContainerManager() - .readContainer(containerName); + .readContainer(containerID); return !containerData.isOpen(); } catch (StorageContainerException e) { throw new AssertionError(e); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java index 1565fbc1c3..0bba5c1c63 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.mockito.Mockito.mock; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -53,13 +54,13 @@ public class TestContainerMetrics { public void testContainerMetrics() throws Exception { XceiverServer server = null; XceiverClient client = null; - String containerName = OzoneUtils.getRequestID(); + long containerID = ContainerTestHelper.getTestContainerID(); String keyName = OzoneUtils.getRequestID(); try { final int interval = 1; Pipeline pipeline = ContainerTestHelper - .createSingleNodePipeline(containerName); + .createSingleNodePipeline(); OzoneConfiguration conf = new OzoneConfiguration(); conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader().getContainerPort()); @@ -72,12 +73,12 @@ public class TestContainerMetrics { ContainerManager containerManager = mock(ContainerManager.class); ChunkManager chunkManager = mock(ChunkManager.class); Mockito.doNothing().when(chunkManager).writeChunk( - Mockito.any(Pipeline.class), Mockito.anyString(), + Mockito.any(BlockID.class), Mockito.any(ChunkInfo.class), Mockito.any(byte[].class), Mockito.any(ContainerProtos.Stage.class)); Mockito.doReturn(chunkManager).when(containerManager).getChunkManager(); - Mockito.doReturn(true).when(containerManager).isOpen(containerName); + Mockito.doReturn(true).when(containerManager).isOpen(containerID); Dispatcher dispatcher = new Dispatcher(containerManager, conf); dispatcher.init(); @@ -90,16 +91,17 @@ public class TestContainerMetrics { // Create container ContainerCommandRequestProto request = ContainerTestHelper - .getCreateContainerRequest(containerName, pipeline); + .getCreateContainerRequest(containerID, pipeline); ContainerCommandResponseProto response = client.sendCommand(request); Assert.assertTrue(request.getTraceID().equals(response.getTraceID())); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); // Write Chunk + BlockID blockID = ContainerTestHelper.getTestBlockID(containerID); ContainerProtos.ContainerCommandRequestProto writeChunkRequest = ContainerTestHelper.getWriteChunkRequest( - pipeline, containerName, keyName, 1024); + pipeline, blockID, 1024); response = client.sendCommand(writeChunkRequest); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java index 4a6ca1d3c7..4e1d14be6d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java @@ -18,17 +18,18 @@ package org.apache.hadoop.ozone.container.ozoneimpl; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.hdds.scm.TestUtils; -import org.apache.hadoop.ozone.web.utils.OzoneUtils; import org.apache.hadoop.hdds.scm.XceiverClient; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.Time; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -52,7 +53,7 @@ public class TestOzoneContainer { @Test public void testCreateOzoneContainer() throws Exception { - String containerName = OzoneUtils.getRequestID(); + long containerID = ContainerTestHelper.getTestContainerID(); OzoneConfiguration conf = newOzoneConfiguration(); OzoneContainer container = null; MiniOzoneCluster cluster = null; @@ -61,8 +62,7 @@ public class TestOzoneContainer { cluster.waitForClusterToBeReady(); // We don't start Ozone Container via data node, we will do it // independently in our test path. - Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline( - containerName); + Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(); conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader().getContainerPort()); conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false); @@ -71,7 +71,7 @@ public class TestOzoneContainer { XceiverClient client = new XceiverClient(pipeline, conf); client.connect(); - createContainerForTesting(client, containerName); + createContainerForTesting(client, containerID); } finally { if (container != null) { container.stop(); @@ -93,13 +93,14 @@ public class TestOzoneContainer { public void testOzoneContainerViaDataNode() throws Exception { MiniOzoneCluster cluster = null; try { - String containerName = OzoneUtils.getRequestID(); + long containerID = + ContainerTestHelper.getTestContainerID(); OzoneConfiguration conf = newOzoneConfiguration(); // Start ozone container Via Datanode create. Pipeline pipeline = - ContainerTestHelper.createSingleNodePipeline(containerName); + ContainerTestHelper.createSingleNodePipeline(); conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader().getContainerPort()); @@ -111,7 +112,7 @@ public class TestOzoneContainer { // This client talks to ozone container via datanode. XceiverClient client = new XceiverClient(pipeline, conf); - runTestOzoneContainerViaDataNode(containerName, client); + runTestOzoneContainerViaDataNode(containerID, client); } finally { if (cluster != null) { cluster.shutdown(); @@ -120,7 +121,7 @@ public class TestOzoneContainer { } static void runTestOzoneContainerViaDataNode( - String containerName, XceiverClientSpi client) throws Exception { + long testContainerID, XceiverClientSpi client) throws Exception { ContainerProtos.ContainerCommandRequestProto request, writeChunkRequest, putKeyRequest, updateRequest1, updateRequest2; @@ -129,12 +130,12 @@ public class TestOzoneContainer { try { client.connect(); - // Create container - createContainerForTesting(client, containerName); - writeChunkRequest = writeChunkForContainer(client, containerName, 1024); + Pipeline pipeline = client.getPipeline(); + createContainerForTesting(client, testContainerID); + writeChunkRequest = writeChunkForContainer(client, testContainerID, 1024); // Read Chunk - request = ContainerTestHelper.getReadChunkRequest(writeChunkRequest + request = ContainerTestHelper.getReadChunkRequest(pipeline, writeChunkRequest .getWriteChunk()); response = client.sendCommand(request); @@ -143,7 +144,7 @@ public class TestOzoneContainer { Assert.assertTrue(request.getTraceID().equals(response.getTraceID())); // Put Key - putKeyRequest = ContainerTestHelper.getPutKeyRequest(writeChunkRequest + putKeyRequest = ContainerTestHelper.getPutKeyRequest(pipeline, writeChunkRequest .getWriteChunk()); @@ -154,21 +155,21 @@ public class TestOzoneContainer { .assertTrue(putKeyRequest.getTraceID().equals(response.getTraceID())); // Get Key - request = ContainerTestHelper.getKeyRequest(putKeyRequest.getPutKey()); + request = ContainerTestHelper.getKeyRequest(pipeline, putKeyRequest.getPutKey()); response = client.sendCommand(request); ContainerTestHelper.verifyGetKey(request, response); // Delete Key request = - ContainerTestHelper.getDeleteKeyRequest(putKeyRequest.getPutKey()); + ContainerTestHelper.getDeleteKeyRequest(pipeline, putKeyRequest.getPutKey()); response = client.sendCommand(request); Assert.assertNotNull(response); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); Assert.assertTrue(request.getTraceID().equals(response.getTraceID())); //Delete Chunk - request = ContainerTestHelper.getDeleteChunkRequest(writeChunkRequest + request = ContainerTestHelper.getDeleteChunkRequest(pipeline, writeChunkRequest .getWriteChunk()); response = client.sendCommand(request); @@ -180,15 +181,17 @@ public class TestOzoneContainer { Map<String, String> containerUpdate = new HashMap<String, String>(); containerUpdate.put("container_updated_key", "container_updated_value"); updateRequest1 = ContainerTestHelper.getUpdateContainerRequest( - containerName, containerUpdate); + testContainerID, containerUpdate); updateResponse1 = client.sendCommand(updateRequest1); Assert.assertNotNull(updateResponse1); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); //Update an non-existing container + long nonExistingContinerID = + ContainerTestHelper.getTestContainerID(); updateRequest2 = ContainerTestHelper.getUpdateContainerRequest( - "non_exist_container", containerUpdate); + nonExistingContinerID, containerUpdate); updateResponse2 = client.sendCommand(updateRequest2); Assert.assertEquals(ContainerProtos.Result.CONTAINER_NOT_FOUND, updateResponse2.getResult()); @@ -211,9 +214,8 @@ public class TestOzoneContainer { .setRandomContainerPort(false) .build(); cluster.waitForClusterToBeReady(); - String containerName = client.getPipeline().getContainerName(); - - runTestBothGetandPutSmallFile(containerName, client); + long containerID = ContainerTestHelper.getTestContainerID(); + runTestBothGetandPutSmallFile(containerID, client); } finally { if (cluster != null) { cluster.shutdown(); @@ -222,16 +224,16 @@ public class TestOzoneContainer { } static void runTestBothGetandPutSmallFile( - String containerName, XceiverClientSpi client) throws Exception { + long containerID, XceiverClientSpi client) throws Exception { try { client.connect(); - createContainerForTesting(client, containerName); + createContainerForTesting(client, containerID); - String keyName = OzoneUtils.getRequestID(); + BlockID blockId = ContainerTestHelper.getTestBlockID(containerID); final ContainerProtos.ContainerCommandRequestProto smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest( - client.getPipeline(), containerName, keyName, 1024); + client.getPipeline(), blockId, 1024); ContainerProtos.ContainerCommandResponseProto response = client.sendCommand(smallFileRequest); Assert.assertNotNull(response); @@ -239,7 +241,7 @@ public class TestOzoneContainer { .equals(response.getTraceID())); final ContainerProtos.ContainerCommandRequestProto getSmallFileRequest - = ContainerTestHelper.getReadSmallFileRequest( + = ContainerTestHelper.getReadSmallFileRequest(client.getPipeline(), smallFileRequest.getPutSmallFile().getKey()); response = client.sendCommand(getSmallFileRequest); Assert.assertArrayEquals( @@ -272,13 +274,13 @@ public class TestOzoneContainer { cluster.waitForClusterToBeReady(); client.connect(); - String containerName = client.getPipeline().getContainerName(); - createContainerForTesting(client, containerName); - writeChunkRequest = writeChunkForContainer(client, containerName, 1024); + long containerID = ContainerTestHelper.getTestContainerID(); + createContainerForTesting(client, containerID); + writeChunkRequest = writeChunkForContainer(client, containerID, 1024); - putKeyRequest = ContainerTestHelper.getPutKeyRequest(writeChunkRequest - .getWriteChunk()); + putKeyRequest = ContainerTestHelper.getPutKeyRequest(client.getPipeline(), + writeChunkRequest.getWriteChunk()); // Put key before closing. response = client.sendCommand(putKeyRequest); Assert.assertNotNull(response); @@ -288,7 +290,8 @@ public class TestOzoneContainer { putKeyRequest.getTraceID().equals(response.getTraceID())); // Close the contianer. - request = ContainerTestHelper.getCloseContainer(client.getPipeline()); + request = ContainerTestHelper.getCloseContainer( + client.getPipeline(), containerID); response = client.sendCommand(request); Assert.assertNotNull(response); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); @@ -307,8 +310,8 @@ public class TestOzoneContainer { writeChunkRequest.getTraceID().equals(response.getTraceID())); // Read chunk must work on a closed container. - request = ContainerTestHelper.getReadChunkRequest(writeChunkRequest - .getWriteChunk()); + request = ContainerTestHelper.getReadChunkRequest(client.getPipeline(), + writeChunkRequest.getWriteChunk()); response = client.sendCommand(request); Assert.assertNotNull(response); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); @@ -324,13 +327,15 @@ public class TestOzoneContainer { .assertTrue(putKeyRequest.getTraceID().equals(response.getTraceID())); // Get key must work on the closed container. - request = ContainerTestHelper.getKeyRequest(putKeyRequest.getPutKey()); + request = ContainerTestHelper.getKeyRequest(client.getPipeline(), + putKeyRequest.getPutKey()); response = client.sendCommand(request); ContainerTestHelper.verifyGetKey(request, response); // Delete Key must fail on a closed container. request = - ContainerTestHelper.getDeleteKeyRequest(putKeyRequest.getPutKey()); + ContainerTestHelper.getDeleteKeyRequest(client.getPipeline(), + putKeyRequest.getPutKey()); response = client.sendCommand(request); Assert.assertNotNull(response); Assert.assertEquals(ContainerProtos.Result.CLOSED_CONTAINER_IO, @@ -363,12 +368,12 @@ public class TestOzoneContainer { cluster.waitForClusterToBeReady(); client.connect(); - String containerName = client.getPipeline().getContainerName(); - createContainerForTesting(client, containerName); - writeChunkRequest = writeChunkForContainer(client, containerName, 1024); + long containerID = ContainerTestHelper.getTestContainerID(); + createContainerForTesting(client, containerID); + writeChunkRequest = writeChunkForContainer(client, containerID, 1024); - putKeyRequest = ContainerTestHelper.getPutKeyRequest(writeChunkRequest - .getWriteChunk()); + putKeyRequest = ContainerTestHelper.getPutKeyRequest(client.getPipeline(), + writeChunkRequest.getWriteChunk()); // Put key before deleting. response = client.sendCommand(putKeyRequest); Assert.assertNotNull(response); @@ -380,7 +385,7 @@ public class TestOzoneContainer { // Container cannot be deleted forcibly because // the container is not closed. request = ContainerTestHelper.getDeleteContainer( - client.getPipeline(), true); + client.getPipeline(), containerID, true); response = client.sendCommand(request); Assert.assertNotNull(response); @@ -389,7 +394,8 @@ public class TestOzoneContainer { Assert.assertTrue(request.getTraceID().equals(response.getTraceID())); // Close the container. - request = ContainerTestHelper.getCloseContainer(client.getPipeline()); + request = ContainerTestHelper.getCloseContainer( + client.getPipeline(), containerID); response = client.sendCommand(request); Assert.assertNotNull(response); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); @@ -397,7 +403,7 @@ public class TestOzoneContainer { // Container cannot be deleted because the container is not empty. request = ContainerTestHelper.getDeleteContainer( - client.getPipeline(), false); + client.getPipeline(), containerID, false); response = client.sendCommand(request); Assert.assertNotNull(response); @@ -408,7 +414,7 @@ public class TestOzoneContainer { // Container can be deleted forcibly because // it is closed and non-empty. request = ContainerTestHelper.getDeleteContainer( - client.getPipeline(), true); + client.getPipeline(), containerID, true); response = client.sendCommand(request); Assert.assertNotNull(response); @@ -430,19 +436,19 @@ public class TestOzoneContainer { // Runs a set of commands as Async calls and verifies that calls indeed worked // as expected. static void runAsyncTests( - String containerName, XceiverClientSpi client) throws Exception { + long containerID, XceiverClientSpi client) throws Exception { try { client.connect(); - createContainerForTesting(client, containerName); + createContainerForTesting(client, containerID); final List<CompletableFuture> computeResults = new LinkedList<>(); int requestCount = 1000; // Create a bunch of Async calls from this test. for(int x = 0; x <requestCount; x++) { - String keyName = OzoneUtils.getRequestID(); + BlockID blockID = ContainerTestHelper.getTestBlockID(containerID); final ContainerProtos.ContainerCommandRequestProto smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest( - client.getPipeline(), containerName, keyName, 1024); + client.getPipeline(), blockID, 1024); CompletableFuture<ContainerProtos.ContainerCommandResponseProto> response = client.sendCommandAsync(smallFileRequest); @@ -477,8 +483,8 @@ public class TestOzoneContainer { .setRandomContainerPort(false) .build(); cluster.waitForClusterToBeReady(); - String containerName = client.getPipeline().getContainerName(); - runAsyncTests(containerName, client); + long containerID = ContainerTestHelper.getTestContainerID(); + runAsyncTests(containerID, client); } finally { if (cluster != null) { cluster.shutdown(); @@ -502,8 +508,9 @@ public class TestOzoneContainer { client.connect(); // Send a request without traceId. + long containerID = ContainerTestHelper.getTestContainerID(); request = ContainerTestHelper - .getRequestWithoutTraceId(client.getPipeline()); + .getRequestWithoutTraceId(client.getPipeline(), containerID); client.sendCommand(request); Assert.fail("IllegalArgumentException expected"); } catch(IllegalArgumentException iae){ @@ -515,13 +522,11 @@ public class TestOzoneContainer { } } - private static XceiverClient createClientForTesting(OzoneConfiguration conf) throws Exception { - String containerName = OzoneUtils.getRequestID(); // Start ozone container Via Datanode create. Pipeline pipeline = - ContainerTestHelper.createSingleNodePipeline(containerName); + ContainerTestHelper.createSingleNodePipeline(); conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader().getContainerPort()); @@ -530,11 +535,11 @@ public class TestOzoneContainer { } private static void createContainerForTesting(XceiverClientSpi client, - String containerName) throws Exception { + long containerID) throws Exception { // Create container ContainerProtos.ContainerCommandRequestProto request = - ContainerTestHelper.getCreateContainerRequest(containerName, - client.getPipeline()); + ContainerTestHelper.getCreateContainerRequest( + containerID, client.getPipeline()); ContainerProtos.ContainerCommandResponseProto response = client.sendCommand(request); Assert.assertNotNull(response); @@ -543,13 +548,12 @@ public class TestOzoneContainer { private static ContainerProtos.ContainerCommandRequestProto writeChunkForContainer(XceiverClientSpi client, - String containerName, int dataLen) throws Exception { + long containerID, int dataLen) throws Exception { // Write Chunk - final String keyName = OzoneUtils.getRequestID(); + BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);; ContainerProtos.ContainerCommandRequestProto writeChunkRequest = ContainerTestHelper.getWriteChunkRequest(client.getPipeline(), - containerName, keyName, dataLen); - + blockID, dataLen); ContainerProtos.ContainerCommandResponseProto response = client.sendCommand(writeChunkRequest); Assert.assertNotNull(response); @@ -559,24 +563,20 @@ public class TestOzoneContainer { } static void runRequestWithoutTraceId( - String containerName, XceiverClientSpi client) throws Exception { + long containerID, XceiverClientSpi client) throws Exception { try { client.connect(); - - createContainerForTesting(client, containerName); - - String keyName = OzoneUtils.getRequestID(); + createContainerForTesting(client, containerID); + BlockID blockID = ContainerTestHelper.getTestBlockID(containerID); final ContainerProtos.ContainerCommandRequestProto smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest( - client.getPipeline(), containerName, keyName, 1024); + client.getPipeline(), blockID, 1024); ContainerProtos.ContainerCommandResponseProto response = client.sendCommand(smallFileRequest); Assert.assertNotNull(response); Assert.assertTrue(smallFileRequest.getTraceID() .equals(response.getTraceID())); - - } finally { if (client != null) { client.close(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java index 9c10b280fc..c686b0b22f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java @@ -67,7 +67,7 @@ public class TestOzoneContainerRatis { private static void runTest( String testName, RpcType rpc, int numNodes, - CheckedBiConsumer<String, XceiverClientSpi, Exception> test) + CheckedBiConsumer<Long, XceiverClientSpi, Exception> test) throws Exception { LOG.info(testName + "(rpc=" + rpc + ", numNodes=" + numNodes); @@ -84,7 +84,6 @@ public class TestOzoneContainerRatis { final String containerName = OzoneUtils.getRequestID(); final List<HddsDatanodeService> datanodes = cluster.getHddsDatanodes(); final Pipeline pipeline = ContainerTestHelper.createPipeline( - containerName, CollectionUtils.as(datanodes, HddsDatanodeService::getDatanodeDetails)); LOG.info("pipeline=" + pipeline); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java index 651b10f1cd..b207914475 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java @@ -77,8 +77,9 @@ public class TestContainerServer { channel = new EmbeddedChannel(new XceiverServerHandler( new TestContainerDispatcher())); ContainerCommandRequestProto request = - ContainerTestHelper.getCreateContainerRequest(containerName, - ContainerTestHelper.createSingleNodePipeline(containerName)); + ContainerTestHelper.getCreateContainerRequest( + ContainerTestHelper.getTestContainerID(), + ContainerTestHelper.createSingleNodePipeline()); channel.writeInbound(request); Assert.assertTrue(channel.finish()); @@ -165,8 +166,7 @@ public class TestContainerServer { XceiverClientSpi client = null; String containerName = OzoneUtils.getRequestID(); try { - final Pipeline pipeline = ContainerTestHelper.createPipeline( - containerName, numDatanodes); + final Pipeline pipeline = ContainerTestHelper.createPipeline(numDatanodes); final OzoneConfiguration conf = new OzoneConfiguration(); initConf.accept(pipeline, conf); @@ -182,7 +182,8 @@ public class TestContainerServer { final ContainerCommandRequestProto request = ContainerTestHelper - .getCreateContainerRequest(containerName, pipeline); + .getCreateContainerRequest( + ContainerTestHelper.getTestContainerID(), pipeline); Assert.assertNotNull(request.getTraceID()); ContainerCommandResponseProto response = client.sendCommand(request); @@ -202,8 +203,7 @@ public class TestContainerServer { String containerName = OzoneUtils.getRequestID(); try { - Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline( - containerName); + Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(); OzoneConfiguration conf = new OzoneConfiguration(); conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader().getContainerPort()); @@ -219,8 +219,8 @@ public class TestContainerServer { client.connect(); ContainerCommandRequestProto request = - ContainerTestHelper.getCreateContainerRequest(containerName, - pipeline); + ContainerTestHelper.getCreateContainerRequest( + ContainerTestHelper.getTestContainerID(), pipeline); ContainerCommandResponseProto response = client.sendCommand(request); Assert.assertTrue(request.getTraceID().equals(response.getTraceID())); Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java index a2a04e017e..18482d1866 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java @@ -117,24 +117,24 @@ public class TestContainerReportWithKeys { cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); - ContainerData cd = getContainerData(keyInfo.getContainerName()); + ContainerData cd = getContainerData(keyInfo.getContainerID()); LOG.info("DN Container Data: keyCount: {} used: {} ", cd.getKeyCount(), cd.getBytesUsed()); - ContainerInfo cinfo = scm.getContainerInfo(keyInfo.getContainerName()); + ContainerInfo cinfo = scm.getContainerInfo(keyInfo.getContainerID()); LOG.info("SCM Container Info keyCount: {} usedBytes: {}", cinfo.getNumberOfKeys(), cinfo.getUsedBytes()); } - private static ContainerData getContainerData(String containerName) { + private static ContainerData getContainerData(long containerID) { ContainerData containerData; try { ContainerManager containerManager = cluster.getHddsDatanodes().get(0) .getDatanodeStateMachine().getContainer().getContainerManager(); - containerData = containerManager.readContainer(containerName); + containerData = containerManager.readContainer(containerID); } catch (StorageContainerException e) { throw new AssertionError(e); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java index ae0ffa07ae..6478e888bf 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java @@ -641,14 +641,6 @@ public class TestKeySpaceManager { new MetadataKeyFilters.KeyPrefixFilter(DELETING_KEY_PREFIX)); Assert.assertEquals(1, list.size()); - // Check the block key in SCM, make sure it's deleted. - Set<String> keys = new HashSet<>(); - keys.add(keyArgs.getResourceName()); - exception.expect(IOException.class); - exception.expectMessage("Specified block key does not exist"); - cluster.getStorageContainerManager().getBlockProtocolServer() - .getBlockLocations(keys); - // Delete the key again to test deleting non-existing key. exception.expect(IOException.class); exception.expectMessage("KEY_NOT_FOUND"); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java index 34bbaf6795..addd87b185 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java @@ -164,7 +164,7 @@ public class TestKsmBlockVersioning { // all the blocks from the previous version must present in the next // version for (KsmKeyLocationInfo info2 : version.getLocationList()) { - if (info.getBlockID().equals(info2.getBlockID())) { + if (info.getLocalID() == info2.getLocalID()) { found = true; break; } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java index 275fadb007..ef6fd5f3c1 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.scm; import org.apache.commons.lang.RandomStringUtils; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -67,12 +68,12 @@ public class TestAllocateContainer { @Test public void testAllocate() throws Exception { - Pipeline pipeline = storageContainerLocationClient.allocateContainer( + ContainerInfo container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), - "container0", containerOwner); - Assert.assertNotNull(pipeline); - Assert.assertNotNull(pipeline.getLeader()); + containerOwner); + Assert.assertNotNull(container); + Assert.assertNotNull(container.getPipeline().getLeader()); } @@ -81,19 +82,6 @@ public class TestAllocateContainer { thrown.expect(NullPointerException.class); storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), - xceiverClientManager.getFactor(), null, containerOwner); - } - - @Test - public void testAllocateDuplicate() throws Exception { - String containerName = RandomStringUtils.randomAlphanumeric(10); - thrown.expect(IOException.class); - thrown.expectMessage("Specified container already exists"); - storageContainerLocationClient.allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerName, containerOwner); - storageContainerLocationClient.allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerName, containerOwner); + xceiverClientManager.getFactor(), null); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java index a5d0eacfd7..dabe90394c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java @@ -88,10 +88,7 @@ public class TestContainerSQLCli { private NodeManager nodeManager; private BlockManagerImpl blockManager; - private Pipeline pipeline1; - private Pipeline pipeline2; - - private HashMap<String, String> blockContainerMap; + private HashMap<Long, Long> blockContainerMap; private final static long DEFAULT_BLOCK_SIZE = 4 * KB; private static HddsProtos.ReplicationFactor factor; @@ -124,7 +121,7 @@ public class TestContainerSQLCli { nodeManager = cluster.getStorageContainerManager().getScmNodeManager(); mapping = new ContainerMapping(conf, nodeManager, 128); - blockManager = new BlockManagerImpl(conf, nodeManager, mapping, 128); + blockManager = new BlockManagerImpl(conf, nodeManager, mapping); // blockManager.allocateBlock() will create containers if there is none // stored in levelDB. The number of containers to create is the value of @@ -142,8 +139,8 @@ public class TestContainerSQLCli { assertEquals(2, nodeManager.getAllNodes().size()); AllocatedBlock ab1 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, CONTAINER_OWNER); - pipeline1 = ab1.getPipeline(); - blockContainerMap.put(ab1.getKey(), pipeline1.getContainerName()); + blockContainerMap.put(ab1.getBlockID().getLocalID(), + ab1.getBlockID().getContainerID()); AllocatedBlock ab2; // we want the two blocks on the two provisioned containers respectively, @@ -155,9 +152,10 @@ public class TestContainerSQLCli { while (true) { ab2 = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, CONTAINER_OWNER); - pipeline2 = ab2.getPipeline(); - blockContainerMap.put(ab2.getKey(), pipeline2.getContainerName()); - if (!pipeline1.getContainerName().equals(pipeline2.getContainerName())) { + blockContainerMap.put(ab2.getBlockID().getLocalID(), + ab2.getBlockID().getContainerID()); + if (ab1.getBlockID().getContainerID() != + ab2.getBlockID().getContainerID()) { break; } } @@ -250,25 +248,26 @@ public class TestContainerSQLCli { conn = connectDB(dbOutPath); sql = "SELECT * FROM containerInfo"; rs = executeQuery(conn, sql); - ArrayList<String> containerNames = new ArrayList<>(); + ArrayList<Long> containerIDs = new ArrayList<>(); while (rs.next()) { - containerNames.add(rs.getString("containerName")); + containerIDs.add(rs.getLong("containerID")); //assertEquals(dnUUID, rs.getString("leaderUUID")); } - assertTrue(containerNames.size() == 2 && - containerNames.contains(pipeline1.getContainerName()) && - containerNames.contains(pipeline2.getContainerName())); + /* TODO: fix this later when the SQLCLI is fixed. + assertTrue(containerIDs.size() == 2 && + containerIDs.contains(pipeline1.getContainerName()) && + containerIDs.contains(pipeline2.getContainerName())); sql = "SELECT * FROM containerMembers"; rs = executeQuery(conn, sql); - containerNames = new ArrayList<>(); + containerIDs = new ArrayList<>(); while (rs.next()) { - containerNames.add(rs.getString("containerName")); + containerIDs.add(rs.getLong("containerID")); //assertEquals(dnUUID, rs.getString("datanodeUUID")); } - assertTrue(containerNames.size() == 2 && - containerNames.contains(pipeline1.getContainerName()) && - containerNames.contains(pipeline2.getContainerName())); + assertTrue(containerIDs.size() == 2 && + containerIDs.contains(pipeline1.getContainerName()) && + containerIDs.contains(pipeline2.getContainerName())); sql = "SELECT * FROM datanodeInfo"; rs = executeQuery(conn, sql); @@ -282,6 +281,7 @@ public class TestContainerSQLCli { int expected = pipeline1.getLeader().getUuid().equals( pipeline2.getLeader().getUuid())? 1 : 2; assertEquals(expected, count); + */ Files.delete(Paths.get(dbOutPath)); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java index d75b66c6bc..f56d78c8d6 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.ozone.scm; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -29,9 +31,9 @@ import org.apache.hadoop.hdds.scm.protocolPB .StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; +import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -79,19 +81,21 @@ public class TestContainerSmallFile { @Test public void testAllocateWrite() throws Exception { String traceID = UUID.randomUUID().toString(); - String containerName = "container0"; - Pipeline pipeline = + ContainerInfo container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); - XceiverClientSpi client = xceiverClientManager.acquireClient(pipeline); - ContainerProtocolCalls.createContainer(client, traceID); - - ContainerProtocolCalls.writeSmallFile(client, containerName, - "key", "data123".getBytes(), traceID); + HddsProtos.ReplicationFactor.ONE, containerOwner); + XceiverClientSpi client = xceiverClientManager.acquireClient( + container.getPipeline(), container.getContainerID()); + ContainerProtocolCalls.createContainer(client, + container.getContainerID(), traceID); + + BlockID blockID = ContainerTestHelper.getTestBlockID( + container.getContainerID()); + ContainerProtocolCalls.writeSmallFile(client, blockID, + "data123".getBytes(), traceID); ContainerProtos.GetSmallFileResponseProto response = - ContainerProtocolCalls.readSmallFile(client, containerName, "key", - traceID); + ContainerProtocolCalls.readSmallFile(client, blockID, traceID); String readData = response.getData().getData().toStringUtf8(); Assert.assertEquals("data123", readData); xceiverClientManager.releaseClient(client); @@ -100,37 +104,42 @@ public class TestContainerSmallFile { @Test public void testInvalidKeyRead() throws Exception { String traceID = UUID.randomUUID().toString(); - String containerName = "container1"; - Pipeline pipeline = + ContainerInfo container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); - XceiverClientSpi client = xceiverClientManager.acquireClient(pipeline); - ContainerProtocolCalls.createContainer(client, traceID); + HddsProtos.ReplicationFactor.ONE, containerOwner); + XceiverClientSpi client = xceiverClientManager.acquireClient( + container.getPipeline(), container.getContainerID()); + ContainerProtocolCalls.createContainer(client, + container.getContainerID(), traceID); thrown.expect(StorageContainerException.class); thrown.expectMessage("Unable to find the key"); + BlockID blockID = ContainerTestHelper.getTestBlockID( + container.getContainerID()); // Try to read a Key Container Name ContainerProtos.GetSmallFileResponseProto response = - ContainerProtocolCalls.readSmallFile(client, containerName, "key", - traceID); + ContainerProtocolCalls.readSmallFile(client, blockID, traceID); xceiverClientManager.releaseClient(client); } @Test public void testInvalidContainerRead() throws Exception { String traceID = UUID.randomUUID().toString(); - String invalidName = "invalidName"; - String containerName = "container2"; - Pipeline pipeline = + long nonExistContainerID = 8888L; + ContainerInfo container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); - XceiverClientSpi client = xceiverClientManager.acquireClient(pipeline); - ContainerProtocolCalls.createContainer(client, traceID); - ContainerProtocolCalls.writeSmallFile(client, containerName, - "key", "data123".getBytes(), traceID); + HddsProtos.ReplicationFactor.ONE, containerOwner); + XceiverClientSpi client = xceiverClientManager. + acquireClient(container.getPipeline(), container.getContainerID()); + ContainerProtocolCalls.createContainer(client, + container.getContainerID(), traceID); + BlockID blockID = ContainerTestHelper.getTestBlockID( + container.getContainerID()); + ContainerProtocolCalls.writeSmallFile(client, blockID, + "data123".getBytes(), traceID); thrown.expect(StorageContainerException.class); @@ -138,10 +147,13 @@ public class TestContainerSmallFile { // Try to read a invalid key ContainerProtos.GetSmallFileResponseProto response = - ContainerProtocolCalls.readSmallFile(client, invalidName, "key", - traceID); + ContainerProtocolCalls.readSmallFile(client, + ContainerTestHelper.getTestBlockID( + nonExistContainerID), traceID); xceiverClientManager.releaseClient(client); } + + } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java index 04473d1a37..888b72e61f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java @@ -17,11 +17,13 @@ */ package org.apache.hadoop.ozone.scm; +import com.google.common.primitives.Longs; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.KeyUtils; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -43,6 +45,8 @@ import org.junit.rules.Timeout; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.OPEN; @@ -122,28 +126,29 @@ public class TestSCMCli { @Test public void testCreateContainer() throws Exception { - String containerName = "containerTestCreate"; + long containerID = ContainerTestHelper.getTestContainerID(); try { - scm.getClientProtocolServer().getContainer(containerName); + scm.getClientProtocolServer().getContainer(containerID); fail("should not be able to get the container"); } catch (IOException ioe) { assertTrue(ioe.getMessage().contains( - "Specified key does not exist. key : " + containerName)); + "Specified key does not exist. key : " + containerID)); } - String[] args = {"-container", "-create", "-c", containerName}; + String[] args = {"-container", "-create", "-c", + Long.toString(containerID)}; assertEquals(ResultCode.SUCCESS, cli.run(args)); - Pipeline container = scm.getClientProtocolServer() - .getContainer(containerName); + ContainerInfo container = scm.getClientProtocolServer() + .getContainer(containerID); assertNotNull(container); - assertEquals(containerName, container.getContainerName()); + assertEquals(containerID, container.containerID()); } - private boolean containerExist(String containerName) { + private boolean containerExist(long containerID) { try { - Pipeline scmPipeline = scm.getClientProtocolServer() - .getContainer(containerName); - return scmPipeline != null - && containerName.equals(scmPipeline.getContainerName()); + ContainerInfo container = scm.getClientProtocolServer() + .getContainer(containerID); + return container != null + && containerID == container.getContainerID(); } catch (IOException e) { return false; } @@ -162,29 +167,31 @@ public class TestSCMCli { // 1. Test to delete a non-empty container. // **************************************** // Create an non-empty container - containerName = "non-empty-container"; - pipeline = containerOperationClient + ContainerInfo container = containerOperationClient .createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); + HddsProtos.ReplicationFactor.ONE, containerOwner); ContainerData cdata = ContainerData - .getFromProtBuf(containerOperationClient.readContainer(pipeline), conf); - KeyUtils.getDB(cdata, conf).put(containerName.getBytes(), + .getFromProtBuf(containerOperationClient.readContainer( + container.getContainerID(), container.getPipeline()), conf); + KeyUtils.getDB(cdata, conf).put(Longs.toByteArray(container.getContainerID()), "someKey".getBytes()); - Assert.assertTrue(containerExist(containerName)); + Assert.assertTrue(containerExist(container.getContainerID())); // Gracefully delete a container should fail because it is open. - delCmd = new String[] {"-container", "-delete", "-c", containerName}; + delCmd = new String[] {"-container", "-delete", "-c", + Long.toString(container.getContainerID())}; testErr = new ByteArrayOutputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream(); exitCode = runCommandAndGetOutput(delCmd, out, testErr); assertEquals(EXECUTION_ERROR, exitCode); assertTrue(testErr.toString() .contains("Deleting an open container is not allowed.")); - Assert.assertTrue(containerExist(containerName)); + Assert.assertTrue(containerExist(container.getContainerID())); // Close the container - containerOperationClient.closeContainer(pipeline); + containerOperationClient.closeContainer( + container.getContainerID(), container.getPipeline()); // Gracefully delete a container should fail because it is not empty. testErr = new ByteArrayOutputStream(); @@ -192,42 +199,46 @@ public class TestSCMCli { assertEquals(EXECUTION_ERROR, exitCode2); assertTrue(testErr.toString() .contains("Container cannot be deleted because it is not empty.")); - Assert.assertTrue(containerExist(containerName)); + Assert.assertTrue(containerExist(container.getContainerID())); // Try force delete again. - delCmd = new String[] {"-container", "-delete", "-c", containerName, "-f"}; + delCmd = new String[] {"-container", "-delete", "-c", + Long.toString(container.getContainerID()), "-f"}; exitCode = runCommandAndGetOutput(delCmd, out, null); assertEquals("Expected success, found:", ResultCode.SUCCESS, exitCode); - assertFalse(containerExist(containerName)); + assertFalse(containerExist(container.getContainerID())); // **************************************** // 2. Test to delete an empty container. // **************************************** // Create an empty container - containerName = "empty-container"; - pipeline = containerOperationClient + ContainerInfo emptyContainer = containerOperationClient .createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); - containerOperationClient.closeContainer(pipeline); - Assert.assertTrue(containerExist(containerName)); + HddsProtos.ReplicationFactor.ONE, containerOwner); + containerOperationClient.closeContainer(container.getContainerID(), + container.getPipeline()); + Assert.assertTrue(containerExist(container.getContainerID())); // Successfully delete an empty container. - delCmd = new String[] {"-container", "-delete", "-c", containerName}; + delCmd = new String[] {"-container", "-delete", "-c", + Long.toString(emptyContainer.getContainerID())}; exitCode = runCommandAndGetOutput(delCmd, out, null); assertEquals(ResultCode.SUCCESS, exitCode); - assertFalse(containerExist(containerName)); + assertFalse(containerExist(emptyContainer.getContainerID())); // After the container is deleted, - // a same name container can now be recreated. - containerOperationClient.createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); - Assert.assertTrue(containerExist(containerName)); + // another container can now be recreated. + ContainerInfo newContainer = containerOperationClient. + createContainer(xceiverClientManager.getType(), + HddsProtos.ReplicationFactor.ONE, containerOwner); + Assert.assertTrue(containerExist(newContainer.getContainerID())); // **************************************** // 3. Test to delete a non-exist container. // **************************************** - containerName = "non-exist-container"; - delCmd = new String[] {"-container", "-delete", "-c", containerName}; + long nonExistContainerID = ContainerTestHelper.getTestContainerID(); + delCmd = new String[] {"-container", "-delete", "-c", + Long.toString(nonExistContainerID)}; testErr = new ByteArrayOutputStream(); exitCode = runCommandAndGetOutput(delCmd, out, testErr); assertEquals(EXECUTION_ERROR, exitCode); @@ -267,12 +278,13 @@ public class TestSCMCli { EXECUTION_ERROR, exitCode); // Create an empty container. - cname = "ContainerTestInfo1"; - Pipeline pipeline = containerOperationClient + ContainerInfo container = containerOperationClient .createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, cname, containerOwner); + HddsProtos.ReplicationFactor.ONE, containerOwner); ContainerData data = ContainerData - .getFromProtBuf(containerOperationClient.readContainer(pipeline), conf); + .getFromProtBuf(containerOperationClient. + readContainer(container.getContainerID(), + container.getPipeline()), conf); info = new String[]{"-container", "-info", "-c", cname}; ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -289,12 +301,12 @@ public class TestSCMCli { out.reset(); // Create an non-empty container - cname = "ContainerTestInfo2"; - pipeline = containerOperationClient + container = containerOperationClient .createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, cname, containerOwner); + HddsProtos.ReplicationFactor.ONE, containerOwner); data = ContainerData - .getFromProtBuf(containerOperationClient.readContainer(pipeline), conf); + .getFromProtBuf(containerOperationClient.readContainer( + container.getContainerID(), container.getPipeline()), conf); KeyUtils.getDB(data, conf).put(cname.getBytes(), "someKey".getBytes()); info = new String[]{"-container", "-info", "-c", cname}; @@ -311,13 +323,15 @@ public class TestSCMCli { // Close last container and test info again. - containerOperationClient.closeContainer(pipeline); + containerOperationClient.closeContainer( + container.getContainerID(), container.getPipeline()); info = new String[] {"-container", "-info", "-c", cname}; exitCode = runCommandAndGetOutput(info, out, null); assertEquals(ResultCode.SUCCESS, exitCode); data = ContainerData - .getFromProtBuf(containerOperationClient.readContainer(pipeline), conf); + .getFromProtBuf(containerOperationClient.readContainer( + container.getContainerID(), container.getPipeline()), conf); openStatus = data.isOpen() ? "OPEN" : "CLOSED"; expected = String.format(formatStrWithHash, cname, openStatus, @@ -347,11 +361,12 @@ public class TestSCMCli { @Test public void testListContainerCommand() throws Exception { // Create 20 containers for testing. - String prefix = "ContainerForTesting"; + List<ContainerInfo> containers = new ArrayList<>(); for (int index = 0; index < 20; index++) { - String containerName = String.format("%s%02d", prefix, index); - containerOperationClient.createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName, containerOwner); + ContainerInfo container = containerOperationClient.createContainer( + xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, + containerOwner); + containers.add(container); } ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -367,9 +382,11 @@ public class TestSCMCli { out.reset(); err.reset(); + long startContainerID = containers.get(0).getContainerID(); + String startContainerIDStr = Long.toString(startContainerID); // Test with -start and -count, the value of -count is negative. args = new String[] {"-container", "-list", - "-start", prefix + 0, "-count", "-1"}; + "-start", startContainerIDStr, "-count", "-1"}; exitCode = runCommandAndGetOutput(args, out, err); assertEquals(EXECUTION_ERROR, exitCode); assertTrue(err.toString() @@ -378,67 +395,23 @@ public class TestSCMCli { out.reset(); err.reset(); - String startName = String.format("%s%02d", prefix, 0); - // Test with -start and -count. args = new String[] {"-container", "-list", "-start", - startName, "-count", "10"}; - exitCode = runCommandAndGetOutput(args, out, err); - assertEquals(ResultCode.SUCCESS, exitCode); - for (int index = 0; index < 10; index++) { - String containerName = String.format("%s%02d", prefix, index); - assertTrue(out.toString().contains(containerName)); - } - - out.reset(); - err.reset(); - - // Test with -start, -prefix and -count. - startName = String.format("%s%02d", prefix, 0); - String prefixName = String.format("%s0", prefix); - args = new String[] {"-container", "-list", "-start", - startName, "-prefix", prefixName, "-count", "20"}; - exitCode = runCommandAndGetOutput(args, out, err); - assertEquals(ResultCode.SUCCESS, exitCode); - for (int index = 0; index < 10; index++) { - String containerName = String.format("%s%02d", prefix, index); - assertTrue(out.toString().contains(containerName)); - } - - out.reset(); - err.reset(); - - startName = String.format("%s%02d", prefix, 0); - prefixName = String.format("%s0", prefix); - args = new String[] {"-container", "-list", "-start", - startName, "-prefix", prefixName, "-count", "4"}; + startContainerIDStr, "-count", "10"}; exitCode = runCommandAndGetOutput(args, out, err); assertEquals(ResultCode.SUCCESS, exitCode); - for (int index = 0; index < 4; index++) { - String containerName = String.format("%s%02d", prefix, index); - assertTrue(out.toString().contains(containerName)); + for (int index = 1; index < 10; index++) { + String containerID = Long.toString( + containers.get(index).getContainerID()); + assertTrue(out.toString().contains(containerID)); } out.reset(); err.reset(); - prefixName = String.format("%s0", prefix); - args = new String[] {"-container", "-list", - "-prefix", prefixName, "-count", "6"}; - exitCode = runCommandAndGetOutput(args, out, err); - assertEquals(ResultCode.SUCCESS, exitCode); - for (int index = 0; index < 6; index++) { - String containerName = String.format("%s%02d", prefix, index); - assertTrue(out.toString().contains(containerName)); - } - - out.reset(); - err.reset(); - - // Test with -start and -prefix, while -count doesn't exist. - prefixName = String.format("%s%02d", prefix, 20); + // Test with -start, while -count doesn't exist. args = new String[] {"-container", "-list", "-start", - startName, "-prefix", prefixName, "-count", "10"}; + startContainerIDStr}; exitCode = runCommandAndGetOutput(args, out, err); assertEquals(ResultCode.SUCCESS, exitCode); assertTrue(out.toString().isEmpty()); @@ -446,21 +419,23 @@ public class TestSCMCli { @Test public void testCloseContainer() throws Exception { - String containerName = "containerTestClose"; - String[] args = {"-container", "-create", "-c", containerName}; + long containerID = ContainerTestHelper.getTestContainerID(); + String[] args = {"-container", "-create", "-c", + Long.toString(containerID)}; assertEquals(ResultCode.SUCCESS, cli.run(args)); - Pipeline container = scm.getClientProtocolServer() - .getContainer(containerName); + ContainerInfo container = scm.getClientProtocolServer() + .getContainer(containerID); assertNotNull(container); - assertEquals(containerName, container.getContainerName()); + assertEquals(containerID, container.getContainerID()); - ContainerInfo containerInfo = scm.getContainerInfo(containerName); + ContainerInfo containerInfo = scm.getContainerInfo(containerID); assertEquals(OPEN, containerInfo.getState()); - String[] args1 = {"-container", "-close", "-c", containerName}; + String[] args1 = {"-container", "-close", "-c", + Long.toString(containerID)}; assertEquals(ResultCode.SUCCESS, cli.run(args1)); - containerInfo = scm.getContainerInfo(containerName); + containerInfo = scm.getContainerInfo(containerID); assertEquals(CLOSED, containerInfo.getState()); // closing this container again will trigger an error. @@ -502,9 +477,7 @@ public class TestSCMCli { String[] args2 = {"-container", "-create", "-help"}; assertEquals(ResultCode.SUCCESS, cli.run(args2)); String expected2 = - "usage: hdfs scm -container -create <option>\n" + - "where <option> is\n" + - " -c <arg> Specify container name\n"; + "usage: hdfs scm -container -create\n"; assertEquals(expected2, testContent.toString()); testContent.reset(); @@ -513,7 +486,7 @@ public class TestSCMCli { String expected3 = "usage: hdfs scm -container -delete <option>\n" + "where <option> is\n" + - " -c <arg> Specify container name\n" + + " -c <arg> Specify container id\n" + " -f forcibly delete a container\n"; assertEquals(expected3, testContent.toString()); testContent.reset(); @@ -523,7 +496,7 @@ public class TestSCMCli { String expected4 = "usage: hdfs scm -container -info <option>\n" + "where <option> is\n" + - " -c <arg> Specify container name\n"; + " -c <arg> Specify container id\n"; assertEquals(expected4, testContent.toString()); testContent.reset(); @@ -532,9 +505,8 @@ public class TestSCMCli { String expected5 = "usage: hdfs scm -container -list <option>\n" + "where <option> can be the following\n" + - " -count <arg> Specify count number, required\n" + - " -prefix <arg> Specify prefix container name\n" + - " -start <arg> Specify start container name\n"; + " -start <arg> Specify start container id, required\n" + + " -count <arg> Specify count number name\n"; assertEquals(expected5, testContent.toString()); testContent.reset(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java index 332e6790f6..1d19bb3c00 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java @@ -22,9 +22,8 @@ import static org.apache.hadoop.test.MetricsAsserts.getLongGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.junit.Assert.assertEquals; -import java.util.UUID; - import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.TestUtils; @@ -225,7 +224,7 @@ public class TestSCMMetrics { for (int i = 0; i < numReport; i++) { ContainerReport report = new ContainerReport( - UUID.randomUUID().toString(), DigestUtils.sha256Hex("Simulated")); + RandomUtils.nextLong(), DigestUtils.sha256Hex("Simulated")); report.setSize(stat.getSize().get()); report.setBytesUsed(stat.getUsed().get()); report.setReadCount(stat.getReadCount().get()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java index 85403a2e7c..07ad6ef610 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java @@ -19,13 +19,13 @@ package org.apache.hadoop.ozone.scm; import com.google.common.cache.Cache; import org.apache.commons.lang.RandomStringUtils; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientManager; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB .StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; @@ -78,29 +78,24 @@ public class TestXceiverClientManager { OzoneConfiguration conf = new OzoneConfiguration(); XceiverClientManager clientManager = new XceiverClientManager(conf); - String containerName1 = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline1 = storageContainerLocationClient + ContainerInfo container1 = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), - containerName1, containerOwner); - XceiverClientSpi client1 = clientManager.acquireClient(pipeline1); + containerOwner); + XceiverClientSpi client1 = clientManager.acquireClient(container1.getPipeline(), + container1.getContainerID()); Assert.assertEquals(1, client1.getRefcount()); - Assert.assertEquals(containerName1, - client1.getPipeline().getContainerName()); - String containerName2 = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline2 = storageContainerLocationClient + ContainerInfo container2 = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), - containerName2, containerOwner); - XceiverClientSpi client2 = clientManager.acquireClient(pipeline2); + containerOwner); + XceiverClientSpi client2 = clientManager.acquireClient(container2.getPipeline(), + container2.getContainerID()); Assert.assertEquals(1, client2.getRefcount()); - Assert.assertEquals(containerName2, - client2.getPipeline().getContainerName()); - XceiverClientSpi client3 = clientManager.acquireClient(pipeline1); + XceiverClientSpi client3 = clientManager.acquireClient(container1.getPipeline(), + container1.getContainerID()); Assert.assertEquals(2, client3.getRefcount()); Assert.assertEquals(2, client1.getRefcount()); - Assert.assertEquals(containerName1, - client3.getPipeline().getContainerName()); Assert.assertEquals(client1, client3); clientManager.releaseClient(client1); clientManager.releaseClient(client2); @@ -112,43 +107,43 @@ public class TestXceiverClientManager { OzoneConfiguration conf = new OzoneConfiguration(); conf.setInt(SCM_CONTAINER_CLIENT_MAX_SIZE_KEY, 1); XceiverClientManager clientManager = new XceiverClientManager(conf); - Cache<String, XceiverClientSpi> cache = + Cache<Long, XceiverClientSpi> cache = clientManager.getClientCache(); - String containerName1 = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline1 = + ContainerInfo container1 = storageContainerLocationClient.allocateContainer( clientManager.getType(), HddsProtos.ReplicationFactor.ONE, - containerName1, containerOwner); - XceiverClientSpi client1 = clientManager.acquireClient(pipeline1); + containerOwner); + XceiverClientSpi client1 = clientManager.acquireClient(container1.getPipeline(), + container1.getContainerID()); Assert.assertEquals(1, client1.getRefcount()); - Assert.assertEquals(containerName1, - client1.getPipeline().getContainerName()); + Assert.assertEquals(container1.getPipeline(), + client1.getPipeline()); - String containerName2 = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline2 = + ContainerInfo container2 = storageContainerLocationClient.allocateContainer( clientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerName2, containerOwner); - XceiverClientSpi client2 = clientManager.acquireClient(pipeline2); + HddsProtos.ReplicationFactor.ONE, containerOwner); + XceiverClientSpi client2 = clientManager.acquireClient(container2.getPipeline(), + container2.getContainerID()); Assert.assertEquals(1, client2.getRefcount()); - Assert.assertEquals(containerName2, - client2.getPipeline().getContainerName()); Assert.assertNotEquals(client1, client2); // least recent container (i.e containerName1) is evicted - XceiverClientSpi nonExistent1 = cache.getIfPresent(containerName1); + XceiverClientSpi nonExistent1 = cache.getIfPresent(container1.getContainerID()); Assert.assertEquals(null, nonExistent1); // However container call should succeed because of refcount on the client. String traceID1 = "trace" + RandomStringUtils.randomNumeric(4); - ContainerProtocolCalls.createContainer(client1, traceID1); + ContainerProtocolCalls.createContainer(client1, + container1.getContainerID(), traceID1); // After releasing the client, this connection should be closed // and any container operations should fail clientManager.releaseClient(client1); exception.expect(IOException.class); exception.expectMessage("This channel is not connected."); - ContainerProtocolCalls.createContainer(client1, traceID1); + ContainerProtocolCalls.createContainer(client1, + container1.getContainerID(), traceID1); clientManager.releaseClient(client2); } @@ -157,42 +152,39 @@ public class TestXceiverClientManager { OzoneConfiguration conf = new OzoneConfiguration(); conf.setInt(SCM_CONTAINER_CLIENT_MAX_SIZE_KEY, 1); XceiverClientManager clientManager = new XceiverClientManager(conf); - Cache<String, XceiverClientSpi> cache = + Cache<Long, XceiverClientSpi> cache = clientManager.getClientCache(); - String containerName1 = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline1 = + ContainerInfo container1 = storageContainerLocationClient.allocateContainer( clientManager.getType(), - clientManager.getFactor(), containerName1, containerOwner); - XceiverClientSpi client1 = clientManager.acquireClient(pipeline1); + clientManager.getFactor(), containerOwner); + XceiverClientSpi client1 = clientManager.acquireClient(container1.getPipeline(), + container1.getContainerID()); Assert.assertEquals(1, client1.getRefcount()); - Assert.assertEquals(containerName1, - client1.getPipeline().getContainerName()); clientManager.releaseClient(client1); Assert.assertEquals(0, client1.getRefcount()); - String containerName2 = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline2 = storageContainerLocationClient + ContainerInfo container2 = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), - containerName2, containerOwner); - XceiverClientSpi client2 = clientManager.acquireClient(pipeline2); + containerOwner); + XceiverClientSpi client2 = clientManager.acquireClient(container2.getPipeline(), + container2.getContainerID()); Assert.assertEquals(1, client2.getRefcount()); - Assert.assertEquals(containerName2, - client2.getPipeline().getContainerName()); Assert.assertNotEquals(client1, client2); // now client 1 should be evicted - XceiverClientSpi nonExistent = cache.getIfPresent(containerName1); + XceiverClientSpi nonExistent = cache.getIfPresent(container1.getContainerID()); Assert.assertEquals(null, nonExistent); // Any container operation should now fail String traceID2 = "trace" + RandomStringUtils.randomNumeric(4); exception.expect(IOException.class); exception.expectMessage("This channel is not connected."); - ContainerProtocolCalls.createContainer(client1, traceID2); + ContainerProtocolCalls.createContainer(client1, + container1.getContainerID(), traceID2); clientManager.releaseClient(client2); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java index 1403f895bb..99742c2024 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java @@ -27,9 +27,11 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import org.apache.commons.lang.RandomStringUtils; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -41,6 +43,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.Time; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -78,14 +81,15 @@ public class TestXceiverClientMetrics { OzoneConfiguration conf = new OzoneConfiguration(); XceiverClientManager clientManager = new XceiverClientManager(conf); - String containerName = "container" + RandomStringUtils.randomNumeric(10); - Pipeline pipeline = storageContainerLocationClient + ContainerInfo container = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), - containerName, containerOwner); - XceiverClientSpi client = clientManager.acquireClient(pipeline); + containerOwner); + XceiverClientSpi client = clientManager.acquireClient( + container.getPipeline(), container.getContainerID()); ContainerCommandRequestProto request = ContainerTestHelper - .getCreateContainerRequest(containerName, pipeline); + .getCreateContainerRequest(container.getContainerID(), + container.getPipeline()); client.sendCommand(request); MetricsRecordBuilder containerMetrics = getMetrics( @@ -109,11 +113,12 @@ public class TestXceiverClientMetrics { try { // use async interface for testing pending metrics for (int i = 0; i < numRequest; i++) { - String keyName = OzoneUtils.getRequestID(); + BlockID blockID = ContainerTestHelper. + getTestBlockID(container.getContainerID()); ContainerProtos.ContainerCommandRequestProto smallFileRequest; smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest( - client.getPipeline(), containerName, keyName, 1024); + client.getPipeline(), blockID, 1024); CompletableFuture<ContainerProtos.ContainerCommandResponseProto> response = client.sendCommandAsync(smallFileRequest); computeResults.add(response); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java index ae30fb30a6..b621a08d2b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java @@ -623,12 +623,11 @@ public class TestKeys { List<KsmKeyLocationInfo> locations = keyInfo.getLatestVersionLocations().getLocationList(); for (KsmKeyLocationInfo location : locations) { - String containerName = location.getContainerName(); - KeyData keyData = new KeyData(containerName, location.getBlockID()); + KeyData keyData = new KeyData(location.getBlockID()); KeyData blockInfo = cm.getContainerManager() .getKeyManager().getKey(keyData); ContainerData containerData = cm.getContainerManager() - .readContainer(containerName); + .readContainer(keyData.getContainerID()); File dataDir = ContainerUtils .getDataDirectory(containerData).toFile(); for (ContainerProtos.ChunkInfo chunkInfo : blockInfo.getChunks()) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java index fa0eaa2acb..13cc40bb34 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java @@ -21,11 +21,11 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; @@ -480,8 +480,8 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { if (latest == null) { return Collections.emptyList(); } - List<String> item = latest.getLocationList().stream() - .map(KsmKeyLocationInfo::getBlockID) + List<BlockID> item = latest.getLocationList().stream() + .map(b->new BlockID(b.getContainerID(), b.getLocalID())) .collect(Collectors.toList()); BlockGroup keyBlocks = BlockGroup.newBuilder() .setKeyName(DFSUtil.bytes2String(entry.getKey())) @@ -510,9 +510,9 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { continue; } // Get block keys as a list. - List<String> item = info.getLatestVersionLocations() + List<BlockID> item = info.getLatestVersionLocations() .getBlocksLatestVersionOnly().stream() - .map(KsmKeyLocationInfo::getBlockID) + .map(b->new BlockID(b.getContainerID(), b.getLocalID())) .collect(Collectors.toList()); BlockGroup keyBlocks = BlockGroup.newBuilder() .setKeyName(DFSUtil.bytes2String(entry.getKey())) diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java index 14fb69c35f..e51ab28d02 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.utils.BackgroundService; import org.apache.hadoop.utils.BackgroundTask; @@ -117,7 +118,7 @@ public class KeyDeletingService extends BackgroundService { LOG.warn("Key {} deletion failed because some of the blocks" + " were failed to delete, failed blocks: {}", result.getObjectKey(), - String.join(",", result.getFailedBlocks())); + StringUtils.join(",", result.getFailedBlocks())); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java index 70ba178ba3..8ee7d25530 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java @@ -203,8 +203,7 @@ public class KeyManagerImpl implements KeyManager { KsmKeyInfo keyInfo = KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(keyData)); KsmKeyLocationInfo info = new KsmKeyLocationInfo.Builder() - .setContainerName(allocatedBlock.getPipeline().getContainerName()) - .setBlockID(allocatedBlock.getKey()) + .setBlockID(allocatedBlock.getBlockID()) .setShouldCreateContainer(allocatedBlock.getCreateContainer()) .setLength(scmBlockSize) .setOffset(0) @@ -256,8 +255,7 @@ public class KeyManagerImpl implements KeyManager { AllocatedBlock allocatedBlock = scmBlockClient.allocateBlock(allocateSize, type, factor, ksmId); KsmKeyLocationInfo subKeyInfo = new KsmKeyLocationInfo.Builder() - .setContainerName(allocatedBlock.getPipeline().getContainerName()) - .setBlockID(allocatedBlock.getKey()) + .setBlockID(allocatedBlock.getBlockID()) .setShouldCreateContainer(allocatedBlock.getCreateContainer()) .setLength(allocateSize) .setOffset(0) diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java index 76312e7304..120eb0609e 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java @@ -715,7 +715,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } @Override - public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) + public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) throws IOException { try { metrics.incNumBlockAllocateCalls(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java index 7a2d7cc9ca..8e2540a627 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java @@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.ksm; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.utils.BackgroundService; import org.apache.hadoop.utils.BackgroundTask; import org.apache.hadoop.utils.BackgroundTaskQueue; @@ -97,7 +98,7 @@ public class OpenKeyCleanupService extends BackgroundService { LOG.warn("Deleting open Key {} failed because some of the blocks" + " were failed to delete, failed blocks: {}", result.getObjectKey(), - String.join(",", result.getFailedBlocks())); + StringUtils.join(",", result.getFailedBlocks())); } } LOG.info("Found {} expired open key entries, successfully " + diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java index 70d80d5cda..e3f6cc95d8 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java @@ -60,7 +60,7 @@ public class BenchMarkContainerStateMap { for (int x = 1; x < 1000; x++) { try { ContainerInfo containerInfo = new ContainerInfo.Builder() - .setContainerName(pipeline.getContainerName()).setState(CLOSED) + .setState(CLOSED) .setPipeline(pipeline) // This is bytes allocated for blocks inside container, not the // container size @@ -76,7 +76,7 @@ public class BenchMarkContainerStateMap { for (int y = currentCount; y < 2000; y++) { try { ContainerInfo containerInfo = new ContainerInfo.Builder() - .setContainerName(pipeline.getContainerName()).setState(OPEN) + .setState(OPEN) .setPipeline(pipeline) // This is bytes allocated for blocks inside container, not the // container size @@ -91,7 +91,7 @@ public class BenchMarkContainerStateMap { } try { ContainerInfo containerInfo = new ContainerInfo.Builder() - .setContainerName(pipeline.getContainerName()).setState(OPEN) + .setState(OPEN) .setPipeline(pipeline) // This is bytes allocated for blocks inside container, not the // container size @@ -142,7 +142,7 @@ public class BenchMarkContainerStateMap { for (; i.hasNext();) { pipelineChannel.addMember(i.next()); } - return new Pipeline(containerName, pipelineChannel); + return new Pipeline(pipelineChannel); } @Benchmark @@ -151,7 +151,7 @@ public class BenchMarkContainerStateMap { Pipeline pipeline = createSingleNodePipeline(UUID.randomUUID().toString()); int cid = state.containerID.incrementAndGet(); ContainerInfo containerInfo = new ContainerInfo.Builder() - .setContainerName(pipeline.getContainerName()).setState(CLOSED) + .setState(CLOSED) .setPipeline(pipeline) // This is bytes allocated for blocks inside container, not the // container size 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 468fee55ea..b73f108f48 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 @@ -21,7 +21,9 @@ import com.google.protobuf.ByteString; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.io.FileUtils; import org.apache.commons.lang.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.ozone.container.common.impl.ChunkManagerImpl; @@ -32,6 +34,7 @@ import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel; +import org.apache.hadoop.util.Time; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Scope; @@ -41,6 +44,7 @@ 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; @@ -75,6 +79,14 @@ public class BenchMarkDatanodeDispatcher { private AtomicInteger keyCount; private AtomicInteger chunkCount; + final int initContainers = 100; + final int initKeys = 50; + final int initChunks = 100; + + List<Long> containers; + List<Long> keys; + List<String> chunks; + @Setup(Level.Trial) public void initialize() throws IOException { datanodeUuid = UUID.randomUUID().toString(); @@ -110,20 +122,39 @@ public class BenchMarkDatanodeDispatcher { keyCount = new AtomicInteger(); chunkCount = new AtomicInteger(); + containers = new ArrayList<>(); + keys = new ArrayList<>(); + chunks = new ArrayList<>(); + // Create containers - for (int x = 0; x < 100; x++) { - String containerName = "container-" + containerCount.getAndIncrement(); - dispatcher.dispatch(getCreateContainerCommand(containerName)); + for (int x = 0; x < initContainers; x++) { + long containerID = Time.getUtcTime() + x; + ContainerCommandRequestProto req = getCreateContainerCommand(containerID); + dispatcher.dispatch(req); + containers.add(containerID); + containerCount.getAndIncrement(); } + + for (int x = 0; x < initKeys; x++) { + keys.add(Time.getUtcTime()+x); + } + + for (int x = 0; x < initChunks; x++) { + chunks.add("chunk-" + x); + } + // Add chunk and keys to the containers - for (int x = 0; x < 50; x++) { - String chunkName = "chunk-" + chunkCount.getAndIncrement(); - String keyName = "key-" + keyCount.getAndIncrement(); - for (int y = 0; y < 100; y++) { - String containerName = "container-" + y; - dispatcher.dispatch(getWriteChunkCommand(containerName, chunkName)); + for (int x = 0; x < initKeys; x++) { + String chunkName = chunks.get(x); + chunkCount.getAndIncrement(); + long key = keys.get(x); + keyCount.getAndIncrement(); + for (int y = 0; y < initContainers; y++) { + long containerID = containers.get(y); + BlockID blockID = new BlockID(containerID, key); dispatcher - .dispatch(getPutKeyCommand(containerName, chunkName, keyName)); + .dispatch(getPutKeyCommand(blockID, chunkName)); + dispatcher.dispatch(getWriteChunkCommand(blockID, chunkName)); } } } @@ -134,147 +165,166 @@ public class BenchMarkDatanodeDispatcher { FileUtils.deleteDirectory(new File(baseDir)); } - private ContainerCommandRequestProto getCreateContainerCommand( - String containerName) { + private ContainerCommandRequestProto getCreateContainerCommand(long containerID) { CreateContainerRequestProto.Builder createRequest = CreateContainerRequestProto.newBuilder(); createRequest.setPipeline( - new Pipeline(containerName, pipelineChannel).getProtobufMessage()); + new Pipeline(pipelineChannel).getProtobufMessage()); createRequest.setContainerData( - ContainerData.newBuilder().setName(containerName).build()); + ContainerData.newBuilder().setContainerID( + containerID).build()); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.CreateContainer); request.setCreateContainer(createRequest); request.setDatanodeUuid(datanodeUuid); - request.setTraceID(containerName + "-trace"); + request.setTraceID(containerID + "-trace"); return request.build(); } private ContainerCommandRequestProto getWriteChunkCommand( - String containerName, String key) { - + BlockID blockID, String chunkName) { WriteChunkRequestProto.Builder writeChunkRequest = WriteChunkRequestProto .newBuilder() - .setPipeline( - new Pipeline(containerName, pipelineChannel).getProtobufMessage()) - .setKeyName(key) - .setChunkData(getChunkInfo(containerName, key)) + .setBlockID(blockID.getProtobuf()) + .setChunkData(getChunkInfo(blockID, chunkName)) .setData(data); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto .newBuilder(); request.setCmdType(ContainerProtos.Type.WriteChunk) - .setTraceID(containerName + "-" + key +"-trace") + .setTraceID(getBlockTraceID(blockID)) .setDatanodeUuid(datanodeUuid) .setWriteChunk(writeChunkRequest); return request.build(); } private ContainerCommandRequestProto getReadChunkCommand( - String containerName, String key) { + BlockID blockID, String chunkName) { ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto .newBuilder() - .setPipeline( - new Pipeline(containerName, pipelineChannel).getProtobufMessage()) - .setKeyName(key) - .setChunkData(getChunkInfo(containerName, key)); + .setBlockID(blockID.getProtobuf()) + .setChunkData(getChunkInfo(blockID, chunkName)); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto .newBuilder(); request.setCmdType(ContainerProtos.Type.ReadChunk) - .setTraceID(containerName + "-" + key +"-trace") + .setTraceID(getBlockTraceID(blockID)) .setDatanodeUuid(datanodeUuid) .setReadChunk(readChunkRequest); return request.build(); } private ContainerProtos.ChunkInfo getChunkInfo( - String containerName, String key) { + BlockID blockID, String chunkName) { ContainerProtos.ChunkInfo.Builder builder = ContainerProtos.ChunkInfo.newBuilder() .setChunkName( - DigestUtils.md5Hex(key) + "_stream_" + containerName + "_chunk_" - + key) + DigestUtils.md5Hex(chunkName) + + "_stream_" + blockID.getContainerID() + "_block_" + + blockID.getLocalID()) .setOffset(0).setLen(data.size()); return builder.build(); } private ContainerCommandRequestProto getPutKeyCommand( - String containerName, String chunkKey, String key) { + BlockID blockID, String chunkKey) { PutKeyRequestProto.Builder putKeyRequest = PutKeyRequestProto .newBuilder() - .setPipeline( - new Pipeline(containerName, pipelineChannel).getProtobufMessage()) - .setKeyData(getKeyData(containerName, chunkKey, key)); + .setKeyData(getKeyData(blockID, chunkKey)); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto .newBuilder(); request.setCmdType(ContainerProtos.Type.PutKey) - .setTraceID(containerName + "-" + key +"-trace") + .setTraceID(getBlockTraceID(blockID)) .setDatanodeUuid(datanodeUuid) .setPutKey(putKeyRequest); return request.build(); } private ContainerCommandRequestProto getGetKeyCommand( - String containerName, String chunkKey, String key) { + BlockID blockID, String chunkKey) { GetKeyRequestProto.Builder readKeyRequest = GetKeyRequestProto.newBuilder() - .setPipeline( - new Pipeline(containerName, pipelineChannel).getProtobufMessage()) - .setKeyData(getKeyData(containerName, chunkKey, key)); + .setKeyData(getKeyData(blockID, chunkKey)); ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto .newBuilder() .setCmdType(ContainerProtos.Type.GetKey) - .setTraceID(containerName + "-" + key +"-trace") + .setTraceID(getBlockTraceID(blockID)) .setDatanodeUuid(datanodeUuid) .setGetKey(readKeyRequest); return request.build(); } private ContainerProtos.KeyData getKeyData( - String containerName, String chunkKey, String key) { + BlockID blockID, String chunkKey) { ContainerProtos.KeyData.Builder builder = ContainerProtos.KeyData .newBuilder() - .setContainerName(containerName) - .setName(key) - .addChunks(getChunkInfo(containerName, chunkKey)); + .setBlockID(blockID.getProtobuf()) + .addChunks(getChunkInfo(blockID, chunkKey)); return builder.build(); } @Benchmark public void createContainer(BenchMarkDatanodeDispatcher bmdd) { - bmdd.dispatcher.dispatch(getCreateContainerCommand( - "container-" + containerCount.getAndIncrement())); + long containerID = RandomUtils.nextLong(); + ContainerCommandRequestProto req = getCreateContainerCommand(containerID); + bmdd.dispatcher.dispatch(req); + bmdd.containers.add(containerID); + bmdd.containerCount.getAndIncrement(); } @Benchmark public void writeChunk(BenchMarkDatanodeDispatcher bmdd) { - String containerName = "container-" + random.nextInt(containerCount.get()); bmdd.dispatcher.dispatch(getWriteChunkCommand( - containerName, "chunk-" + chunkCount.getAndIncrement())); + getRandomBlockID(), getNewChunkToWrite())); } @Benchmark public void readChunk(BenchMarkDatanodeDispatcher bmdd) { - String containerName = "container-" + random.nextInt(containerCount.get()); - String chunkKey = "chunk-" + random.nextInt(chunkCount.get()); - bmdd.dispatcher.dispatch(getReadChunkCommand(containerName, chunkKey)); + BlockID blockID = getRandomBlockID(); + String chunkKey = getRandomChunkToRead(); + bmdd.dispatcher.dispatch(getReadChunkCommand(blockID, chunkKey)); } @Benchmark public void putKey(BenchMarkDatanodeDispatcher bmdd) { - String containerName = "container-" + random.nextInt(containerCount.get()); - String chunkKey = "chunk-" + random.nextInt(chunkCount.get()); - bmdd.dispatcher.dispatch(getPutKeyCommand( - containerName, chunkKey, "key-" + keyCount.getAndIncrement())); + BlockID blockID = getRandomBlockID(); + String chunkKey = getNewChunkToWrite(); + bmdd.dispatcher.dispatch(getPutKeyCommand(blockID, chunkKey)); } @Benchmark public void getKey(BenchMarkDatanodeDispatcher bmdd) { - String containerName = "container-" + random.nextInt(containerCount.get()); - String chunkKey = "chunk-" + random.nextInt(chunkCount.get()); - String key = "key-" + random.nextInt(keyCount.get()); - bmdd.dispatcher.dispatch(getGetKeyCommand(containerName, chunkKey, key)); + BlockID blockID = getRandomBlockID(); + String chunkKey = getNewChunkToWrite(); + bmdd.dispatcher.dispatch(getGetKeyCommand(blockID, chunkKey)); + } + + // Chunks writes from benchmark only reaches certain containers + // Use initChunks instead of updated counters to guarantee + // key/chunks are readable. + + private BlockID getRandomBlockID() { + return new BlockID(getRandomContainerID(), getRandomKeyID()); + } + + private long getRandomContainerID() { + return containers.get(random.nextInt(initContainers)); + } + + private long getRandomKeyID() { + return keys.get(random.nextInt(initKeys)); + } + + private String getRandomChunkToRead() { + return chunks.get(random.nextInt(initChunks)); + } + + private String getNewChunkToWrite() { + return "chunk-" + chunkCount.getAndIncrement(); + } + + private String getBlockTraceID(BlockID blockID) { + return blockID.getContainerID() + "-" + blockID.getLocalID() +"-trace"; } } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkRocksDbStore.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkRocksDbStore.java index 0890e4bf72..c4c6f9eed6 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkRocksDbStore.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkRocksDbStore.java @@ -72,21 +72,21 @@ public class BenchMarkRocksDbStore { .toFile(); opts.setCreateIfMissing(true); opts.setWriteBufferSize( - (long) StorageUnit.MB.toBytes(Long.valueOf(writeBufferSize))); - opts.setMaxWriteBufferNumber(Integer.valueOf(maxWriteBufferNumber)); - opts.setMaxBackgroundFlushes(Integer.valueOf(maxBackgroundFlushes)); + (long) StorageUnit.MB.toBytes(Long.parseLong(writeBufferSize))); + opts.setMaxWriteBufferNumber(Integer.parseInt(maxWriteBufferNumber)); + opts.setMaxBackgroundFlushes(Integer.parseInt(maxBackgroundFlushes)); BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); tableConfig.setBlockSize( - (long) StorageUnit.KB.toBytes(Long.valueOf(blockSize))); - opts.setMaxOpenFiles(Integer.valueOf(maxOpenFiles)); + (long) StorageUnit.KB.toBytes(Long.parseLong(blockSize))); + opts.setMaxOpenFiles(Integer.parseInt(maxOpenFiles)); opts.setMaxBytesForLevelBase( - (long) StorageUnit.MB.toBytes(Long.valueOf(maxBytesForLevelBase))); + (long) StorageUnit.MB.toBytes(Long.parseLong(maxBytesForLevelBase))); opts.setCompactionStyle(CompactionStyle.UNIVERSAL); opts.setLevel0FileNumCompactionTrigger(10); opts.setLevel0SlowdownWritesTrigger(20); opts.setLevel0StopWritesTrigger(40); opts.setTargetFileSizeBase( - (long) StorageUnit.MB.toBytes(Long.valueOf(maxBytesForLevelBase)) / 10); + (long) StorageUnit.MB.toBytes(Long.parseLong(maxBytesForLevelBase)) / 10); opts.setMaxBackgroundCompactions(8); opts.setUseFsync(false); opts.setBytesPerSync(8388608); |