HDDS-1. Remove SCM Block DB. Contributed by Xiaoyu Yao.
This commit is contained in:
parent
a3a1552c33
commit
3a43ac2851
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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!");
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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())));
|
||||
}
|
||||
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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(); }
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -41,6 +41,6 @@ public interface ContainerDeletionChoosingPolicy {
|
|||
* @throws StorageContainerException
|
||||
*/
|
||||
List<ContainerData> chooseContainerForBlockDeletion(int count,
|
||||
Map<String, ContainerStatus> candidateContainers)
|
||||
Map<Long, ContainerStatus> candidateContainers)
|
||||
throws StorageContainerException;
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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)){
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue