HDDS-1. Remove SCM Block DB. Contributed by Xiaoyu Yao.

This commit is contained in:
Anu Engineer 2018-05-07 14:42:18 -07:00
parent a3a1552c33
commit 3a43ac2851
127 changed files with 2059 additions and 2402 deletions

View File

@ -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();
}
}

View File

@ -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 {

View File

@ -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!");

View File

@ -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);

View File

@ -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);

View File

@ -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());
}
}

View File

@ -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;
/**

View File

@ -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() {

View File

@ -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);
}
}

View File

@ -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;
}
/**

View File

@ -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,20 +143,10 @@ 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.
*
@ -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());

View File

@ -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,

View File

@ -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
@ -34,17 +33,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.

View File

@ -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;

View File

@ -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;
@ -34,14 +34,8 @@ 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
.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;
/**
@ -81,41 +74,6 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
this.rpcProxy = rpcProxy;
}
/**
* 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.
@ -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();

View File

@ -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();

View File

@ -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

View File

@ -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;
}

View File

@ -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;
}
}

View File

@ -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;
}
/**

View File

@ -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)

View File

@ -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) {

View File

@ -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");

View File

@ -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;
}
}
}

View File

@ -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();

View File

@ -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;
}

View File

@ -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;
@ -138,14 +116,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.
*/

View File

@ -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;

View File

@ -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;
}

View File

@ -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

View File

@ -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) {
@ -156,15 +154,6 @@ public class ContainerData {
return builder.build();
}
/**
* Returns the name of the container.
*
* @return - name
*/
public String getContainerName() {
return containerName;
}
/**
* Adds metadata.
*/
@ -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())));
}

View File

@ -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());
@ -102,15 +101,6 @@ public class ContainerReport {
return report;
}
/**
* Gets the container name.
*
* @return - Name
*/
public String getContainerName() {
return containerName;
}
/**
* Returns the final signature for this container.
*
@ -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())

View File

@ -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();
}

View File

@ -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();

View File

@ -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();

View File

@ -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.

View File

@ -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());

View File

@ -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(); }
/**

View File

@ -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;
}

View File

@ -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;

View File

@ -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;

View File

@ -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"

View File

@ -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.

View File

@ -41,6 +41,6 @@ public interface ContainerDeletionChoosingPolicy {
* @throws StorageContainerException
*/
List<ContainerData> chooseContainerForBlockDeletion(int count,
Map<String, ContainerStatus> candidateContainers)
Map<Long, ContainerStatus> candidateContainers)
throws StorageContainerException;
}

View File

@ -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.

View File

@ -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;
/**

View File

@ -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);

View File

@ -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;

View File

@ -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,

View File

@ -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;

View File

@ -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);
}

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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;
}
/**

View File

@ -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);
}
}

View File

@ -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;
@ -42,14 +42,6 @@ public interface BlockManager extends Closeable {
AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType type,
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
@ -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.

View File

@ -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;
@ -53,11 +48,8 @@ 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,70 +302,29 @@ 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()));
}
// 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();
boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED);
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();
}
/**
* 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);
}
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();
}
}
/**
* Deletes a list of blocks in an atomic operation. Internally, SCM writes
* these blocks into a
@ -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();
}

View File

@ -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;
}

View File

@ -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;
/**

View File

@ -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);

View File

@ -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()) {

View File

@ -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);
}

View File

@ -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;
/**

View File

@ -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())) {

View File

@ -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());
}

View File

@ -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) {

View File

@ -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);
}
}

View File

@ -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);
}
/**

View File

@ -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);

View File

@ -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;
@ -138,20 +136,6 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol {
getBlockRpcServer().join();
}
@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
@ -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(),

View File

@ -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.
*

View File

@ -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);
}
/**

View File

@ -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;
@ -106,33 +104,13 @@ public class TestBlockManager {
Assert.assertNotNull(block);
}
@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 {

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -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)

View File

@ -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);

View File

@ -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());
}

View File

@ -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 {

View File

@ -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 =

View File

@ -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);
}

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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());

View File

@ -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);
}

View File

@ -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();
}

View File

@ -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());

View File

@ -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();
}

View File

@ -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);
}
}
/**

View File

@ -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 {

View File

@ -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);

View File

@ -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());
}
}

View File

@ -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)){

View File

@ -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());
}

View File

@ -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);
}

View File

@ -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;
}
}

View File

@ -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();
}
}

Some files were not shown because too many files have changed in this diff Show More