HDDS-887. Add DispatcherContext info to Dispatcher from containerStateMachine. Contributed by Shashikant Banerjee.

This commit is contained in:
Shashikant Banerjee 2018-12-02 08:00:35 +05:30
parent d15dc43659
commit 5a3c7714c4
20 changed files with 321 additions and 188 deletions

View File

@ -373,17 +373,10 @@ enum ChecksumType {
MD5 = 5;
}
enum Stage {
WRITE_DATA = 1;
COMMIT_DATA = 2;
COMBINED = 3;
}
message WriteChunkRequestProto {
required DatanodeBlockID blockID = 1;
required ChunkInfo chunkData = 2;
optional bytes data = 3;
optional Stage stage = 4 [default = COMBINED];
}
message WriteChunkResponseProto {
@ -392,7 +385,6 @@ message WriteChunkResponseProto {
message ReadChunkRequestProto {
required DatanodeBlockID blockID = 1;
required ChunkInfo chunkData = 2;
optional bool readFromTmpFile = 3 [default = false];
}
message ReadChunkResponseProto {

View File

@ -47,6 +47,8 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis
.DispatcherContext;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@ -133,7 +135,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
@Override
public ContainerCommandResponseProto dispatch(
ContainerCommandRequestProto msg) {
ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
Preconditions.checkNotNull(msg);
LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
msg.getTraceID());
@ -194,7 +196,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
audit(action, eventType, params, AuditEventStatus.FAILURE, ex);
return ContainerUtils.logAndReturnError(LOG, ex, msg);
}
responseProto = handler.handle(msg, container);
responseProto = handler.handle(msg, container, dispatcherContext);
if (responseProto != null) {
metrics.incContainerOpsLatencies(cmdType, System.nanoTime() - startTime);
@ -269,7 +271,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
// TODO: Assuming the container type to be KeyValueContainer for now.
// We need to get container type from the containerRequest.
Handler handler = getHandler(containerType);
handler.handle(requestBuilder.build(), null);
handler.handle(requestBuilder.build(), null, null);
}
/**

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
/**
* Dispatcher acts as the bridge between the transport layer and
@ -37,9 +38,11 @@ public interface ContainerDispatcher {
/**
* Dispatches commands to container layer.
* @param msg - Command Request
* @param context - Context info related to ContainerStateMachine
* @return Command Response
*/
ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg);
ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg,
DispatcherContext context);
/**
* Validates whether the container command should be executed on the pipeline

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerExcep
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
@ -103,7 +104,8 @@ public abstract class Handler {
}
public abstract ContainerCommandResponseProto handle(
ContainerCommandRequestProto msg, Container container);
ContainerCommandRequestProto msg, Container container,
DispatcherContext dispatcherContext);
/**
* Import container data from a raw input stream.

View File

@ -53,7 +53,8 @@ public class GrpcXceiverService extends
@Override
public void onNext(ContainerCommandRequestProto request) {
try {
ContainerCommandResponseProto resp = dispatcher.dispatch(request);
ContainerCommandResponseProto resp =
dispatcher.dispatch(request, null);
responseObserver.onNext(resp);
} catch (Throwable e) {
LOG.error("{} got exception when processing"

View File

@ -130,7 +130,7 @@ public final class XceiverServerGrpc implements XceiverServerSpi {
public void submitRequest(ContainerCommandRequestProto request,
HddsProtos.PipelineID pipelineID) throws IOException {
ContainerProtos.ContainerCommandResponseProto response =
storageContainer.dispatch(request);
storageContainer.dispatch(request, null);
if (response.getResult() != ContainerProtos.Result.SUCCESS) {
throw new StorageContainerException(response.getMessage(),
response.getResult());

View File

@ -22,8 +22,6 @@ import com.google.common.base.Preconditions;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
import org.apache.ratis.protocol.RaftGroup;
import org.apache.ratis.protocol.RaftGroupId;
@ -35,7 +33,6 @@ import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
import org.apache.ratis.thirdparty.com.google.protobuf
.InvalidProtocolBufferException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Stage;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@ -237,7 +234,6 @@ public class ContainerStateMachine extends BaseStateMachine {
final WriteChunkRequestProto dataWriteChunkProto =
WriteChunkRequestProto
.newBuilder(write)
.setStage(Stage.WRITE_DATA)
.build();
ContainerCommandRequestProto dataContainerCommandProto =
ContainerCommandRequestProto
@ -252,7 +248,6 @@ public class ContainerStateMachine extends BaseStateMachine {
.setChunkData(write.getChunkData())
// skipping the data field as it is
// already set in statemachine data proto
.setStage(Stage.COMMIT_DATA)
.build();
ContainerCommandRequestProto commitContainerCommandProto =
ContainerCommandRequestProto
@ -292,15 +287,18 @@ public class ContainerStateMachine extends BaseStateMachine {
}
private ContainerCommandResponseProto dispatchCommand(
ContainerCommandRequestProto requestProto) {
ContainerCommandRequestProto requestProto,
DispatcherContext context) {
LOG.trace("dispatch {}", requestProto);
ContainerCommandResponseProto response = dispatcher.dispatch(requestProto);
ContainerCommandResponseProto response =
dispatcher.dispatch(requestProto, context);
LOG.trace("response {}", response);
return response;
}
private Message runCommand(ContainerCommandRequestProto requestProto) {
return dispatchCommand(requestProto)::toByteString;
private Message runCommand(ContainerCommandRequestProto requestProto,
DispatcherContext context) {
return dispatchCommand(requestProto, context)::toByteString;
}
private ExecutorService getCommandExecutor(
@ -310,7 +308,7 @@ public class ContainerStateMachine extends BaseStateMachine {
}
private CompletableFuture<Message> handleWriteChunk(
ContainerCommandRequestProto requestProto, long entryIndex) {
ContainerCommandRequestProto requestProto, long entryIndex, long term) {
final WriteChunkRequestProto write = requestProto.getWriteChunk();
RaftServer server = ratisServer.getServer();
Preconditions.checkState(server instanceof RaftServerProxy);
@ -321,8 +319,14 @@ public class ContainerStateMachine extends BaseStateMachine {
} catch (IOException ioe) {
return completeExceptionally(ioe);
}
DispatcherContext context =
new DispatcherContext.Builder()
.setTerm(term)
.setLogIndex(entryIndex)
.setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
.build();
CompletableFuture<Message> writeChunkFuture = CompletableFuture
.supplyAsync(() -> runCommand(requestProto), chunkExecutor);
.supplyAsync(() -> runCommand(requestProto, context), chunkExecutor);
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
LOG.debug("writeChunk writeStateMachineData : blockId " + write.getBlockID()
+ " logIndex " + entryIndex + " chunkName " + write.getChunkData()
@ -355,7 +359,8 @@ public class ContainerStateMachine extends BaseStateMachine {
// CreateContainer will happen as a part of writeChunk only.
switch (cmdType) {
case WriteChunk:
return handleWriteChunk(requestProto, entry.getIndex());
return handleWriteChunk(requestProto, entry.getIndex(),
entry.getTerm());
default:
throw new IllegalStateException("Cmd Type:" + cmdType
+ " should not have state machine data");
@ -372,39 +377,36 @@ public class ContainerStateMachine extends BaseStateMachine {
metrics.incNumReadStateMachineOps();
final ContainerCommandRequestProto requestProto =
getRequestProto(request.getContent());
return CompletableFuture.completedFuture(runCommand(requestProto));
return CompletableFuture.completedFuture(runCommand(requestProto, null));
} catch (IOException e) {
metrics.incNumReadStateMachineFails();
return completeExceptionally(e);
}
}
private ByteString readStateMachineData(ContainerCommandRequestProto
requestProto) {
private ByteString readStateMachineData(
ContainerCommandRequestProto requestProto, long term, long index) {
WriteChunkRequestProto writeChunkRequestProto =
requestProto.getWriteChunk();
// Assert that store log entry is for COMMIT_DATA, the WRITE_DATA is
// written through writeStateMachineData.
Preconditions
.checkArgument(writeChunkRequestProto.getStage() == Stage.COMMIT_DATA);
// prepare the chunk to be read
ReadChunkRequestProto.Builder readChunkRequestProto =
ReadChunkRequestProto.newBuilder()
.setBlockID(writeChunkRequestProto.getBlockID())
.setChunkData(writeChunkRequestProto.getChunkData())
// set readFromTempFile to true in case, the chunkFile does
// not exist as applyTransaction is not executed for this entry yet.
.setReadFromTmpFile(true);
.setChunkData(writeChunkRequestProto.getChunkData());
ContainerCommandRequestProto dataContainerCommandProto =
ContainerCommandRequestProto.newBuilder(requestProto)
.setCmdType(Type.ReadChunk)
.setReadChunk(readChunkRequestProto)
.build();
DispatcherContext context =
new DispatcherContext.Builder()
.setTerm(term)
.setLogIndex(index)
.setReadFromTmpFile(true)
.build();
// read the chunk
ContainerCommandResponseProto response =
dispatchCommand(dataContainerCommandProto);
dispatchCommand(dataContainerCommandProto, context);
ReadChunkResponseProto responseProto = response.getReadChunk();
ByteString data = responseProto.getData();
@ -416,14 +418,14 @@ public class ContainerStateMachine extends BaseStateMachine {
/**
* Reads the Entry from the Cache or loads it back by reading from disk.
*/
private ByteString getCachedStateMachineData(Long logIndex,
private ByteString getCachedStateMachineData(Long logIndex, long term,
ContainerCommandRequestProto requestProto) throws ExecutionException {
try {
return reconstructWriteChunkRequest(
stateMachineDataCache.get(logIndex, new Callable<ByteString>() {
@Override
public ByteString call() throws Exception {
return readStateMachineData(requestProto);
return readStateMachineData(requestProto, term, logIndex);
}
}), requestProto);
} catch (ExecutionException e) {
@ -439,7 +441,7 @@ public class ContainerStateMachine extends BaseStateMachine {
final WriteChunkRequestProto.Builder dataWriteChunkProto =
WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
// adding the state machine data
.setData(data).setStage(Stage.WRITE_DATA);
.setData(data);
ContainerCommandRequestProto.Builder newStateMachineProto =
ContainerCommandRequestProto.newBuilder(requestProto)
@ -486,7 +488,8 @@ public class ContainerStateMachine extends BaseStateMachine {
CompletableFuture<ByteString> future = new CompletableFuture<>();
return future.supplyAsync(() -> {
try {
return getCachedStateMachineData(entry.getIndex(), requestProto);
return getCachedStateMachineData(entry.getIndex(), entry.getTerm(),
requestProto);
} catch (ExecutionException e) {
future.completeExceptionally(e);
return null;
@ -524,6 +527,10 @@ public class ContainerStateMachine extends BaseStateMachine {
@Override
public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
long index = trx.getLogEntry().getIndex();
DispatcherContext.Builder builder =
new DispatcherContext.Builder()
.setTerm(trx.getLogEntry().getTerm())
.setLogIndex(index);
// ApplyTransaction call can come with an entryIndex much greater than
// lastIndex updated because in between entries in the raft log can be
@ -539,51 +546,16 @@ public class ContainerStateMachine extends BaseStateMachine {
getRequestProto(trx.getStateMachineLogEntry().getLogData());
Type cmdType = requestProto.getCmdType();
CompletableFuture<Message> future;
if (cmdType == Type.PutBlock || cmdType == Type.PutSmallFile) {
BlockData blockData;
ContainerProtos.BlockData blockDataProto = cmdType == Type.PutBlock ?
requestProto.getPutBlock().getBlockData() :
requestProto.getPutSmallFile().getBlock().getBlockData();
// set the blockCommitSequenceId
try {
blockData = BlockData.getFromProtoBuf(blockDataProto);
} catch (IOException ioe) {
LOG.error("unable to retrieve blockData info for Block {}",
blockDataProto.getBlockID());
return completeExceptionally(ioe);
}
blockData.setBlockCommitSequenceId(index);
final ContainerProtos.PutBlockRequestProto putBlockRequestProto =
ContainerProtos.PutBlockRequestProto
.newBuilder(requestProto.getPutBlock())
.setBlockData(blockData.getProtoBufMessage()).build();
ContainerCommandRequestProto containerCommandRequestProto;
if (cmdType == Type.PutSmallFile) {
ContainerProtos.PutSmallFileRequestProto smallFileRequestProto =
ContainerProtos.PutSmallFileRequestProto
.newBuilder(requestProto.getPutSmallFile())
.setBlock(putBlockRequestProto).build();
containerCommandRequestProto =
ContainerCommandRequestProto.newBuilder(requestProto)
.setPutSmallFile(smallFileRequestProto).build();
} else {
containerCommandRequestProto =
ContainerCommandRequestProto.newBuilder(requestProto)
.setPutBlock(putBlockRequestProto).build();
}
future = CompletableFuture
.supplyAsync(() -> runCommand(containerCommandRequestProto),
getCommandExecutor(requestProto));
} else {
// Make sure that in write chunk, the user data is not set
if (cmdType == Type.WriteChunk) {
Preconditions.checkArgument(requestProto
.getWriteChunk().getData().isEmpty());
}
future = CompletableFuture.supplyAsync(() -> runCommand(requestProto),
getCommandExecutor(requestProto));
// Make sure that in write chunk, the user data is not set
if (cmdType == Type.WriteChunk) {
Preconditions
.checkArgument(requestProto.getWriteChunk().getData().isEmpty());
builder
.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA);
}
future = CompletableFuture
.supplyAsync(() -> runCommand(requestProto, builder.build()),
getCommandExecutor(requestProto));
lastIndex = index;
future.thenAccept(m -> {
final Long previous =

View File

@ -0,0 +1,133 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.ozone.container.common.transport.server.ratis;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* DispatcherContext class holds transport protocol specfic context info
* required for execution of container commands over the container dispatcher.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DispatcherContext {
/**
* Determines which stage of writeChunk a write chunk request is for.
*/
public enum WriteChunkStage {
WRITE_DATA, COMMIT_DATA, COMBINED
}
// whether the chunk data needs to be written or committed or both
private final WriteChunkStage stage;
// indicates whether the read from tmp chunk files is allowed
private final boolean readFromTmpFile;
// which term the request is being served in Ratis
private final long term;
// the log index in Ratis log to which the request belongs to
private final long logIndex;
private DispatcherContext(long term, long index, WriteChunkStage stage,
boolean readFromTmpFile) {
this.term = term;
this.logIndex = index;
this.stage = stage;
this.readFromTmpFile = readFromTmpFile;
}
public long getLogIndex() {
return logIndex;
}
public boolean isReadFromTmpFile() {
return readFromTmpFile;
}
public long getTerm() {
return term;
}
public WriteChunkStage getStage() {
return stage;
}
/**
* Builder class for building DispatcherContext.
*/
public static final class Builder {
private WriteChunkStage stage = WriteChunkStage.COMBINED;
private boolean readFromTmpFile = false;
private long term;
private long logIndex;
/**
* Sets the WriteChunkStage.
*
* @param stage WriteChunk Stage
* @return DispatcherContext.Builder
*/
public Builder setStage(WriteChunkStage stage) {
this.stage = stage;
return this;
}
/**
* Sets the flag for reading from tmp chunk files.
*
* @param readFromTmpFile whether to read from tmp chunk file or not
* @return DispatcherContext.Builder
*/
public Builder setReadFromTmpFile(boolean readFromTmpFile) {
this.readFromTmpFile = readFromTmpFile;
return this;
}
/**
* Sets the current term for the container request from Ratis.
*
* @param term current term
* @return DispatcherContext.Builder
*/
public Builder setTerm(long term) {
this.term = term;
return this;
}
/**
* Sets the logIndex for the container request from Ratis.
*
* @param logIndex log index
* @return DispatcherContext.Builder
*/
public Builder setLogIndex(long logIndex) {
this.logIndex = logIndex;
return this;
}
/**
* Builds and returns DatanodeDetails instance.
*
* @return DispatcherContext
*/
public DispatcherContext build() {
return new DispatcherContext(term, logIndex, stage, readFromTmpFile);
}
}
}

View File

@ -58,6 +58,10 @@ import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis
.DispatcherContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis
.DispatcherContext.WriteChunkStage;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.common.volume
.RoundRobinVolumeChoosingPolicy;
@ -81,8 +85,6 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import static org.apache.hadoop.hdds.HddsConfigKeys
.HDDS_DATANODE_VOLUME_CHOOSING_POLICY;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.*;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.Stage;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
import static org.apache.hadoop.ozone.OzoneConfigKeys
@ -146,7 +148,8 @@ public class KeyValueHandler extends Handler {
@Override
public ContainerCommandResponseProto handle(
ContainerCommandRequestProto request, Container container) {
ContainerCommandRequestProto request, Container container,
DispatcherContext dispatcherContext) {
Type cmdType = request.getCmdType();
KeyValueContainer kvContainer = (KeyValueContainer) container;
@ -164,7 +167,7 @@ public class KeyValueHandler extends Handler {
case CloseContainer:
return handleCloseContainer(request, kvContainer);
case PutBlock:
return handlePutBlock(request, kvContainer);
return handlePutBlock(request, kvContainer, dispatcherContext);
case GetBlock:
return handleGetBlock(request, kvContainer);
case DeleteBlock:
@ -172,17 +175,17 @@ public class KeyValueHandler extends Handler {
case ListBlock:
return handleUnsupportedOp(request);
case ReadChunk:
return handleReadChunk(request, kvContainer);
return handleReadChunk(request, kvContainer, dispatcherContext);
case DeleteChunk:
return handleDeleteChunk(request, kvContainer);
case WriteChunk:
return handleWriteChunk(request, kvContainer);
return handleWriteChunk(request, kvContainer, dispatcherContext);
case ListChunk:
return handleUnsupportedOp(request);
case CompactChunk:
return handleUnsupportedOp(request);
case PutSmallFile:
return handlePutSmallFile(request, kvContainer);
return handlePutSmallFile(request, kvContainer, dispatcherContext);
case GetSmallFile:
return handleGetSmallFile(request, kvContainer);
case GetCommittedBlockLength:
@ -392,7 +395,8 @@ public class KeyValueHandler extends Handler {
* Handle Put Block operation. Calls BlockManager to process the request.
*/
ContainerCommandResponseProto handlePutBlock(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
long blockLength;
if (!request.hasPutBlock()) {
@ -401,14 +405,18 @@ public class KeyValueHandler extends Handler {
return ContainerUtils.malformedRequest(request);
}
BlockData blockData;
try {
checkContainerOpen(kvContainer);
BlockData blockData = BlockData.getFromProtoBuf(
blockData = BlockData.getFromProtoBuf(
request.getPutBlock().getBlockData());
Preconditions.checkNotNull(blockData);
long bcsId =
dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
blockData.setBlockCommitSequenceId(bcsId);
long numBytes = blockData.getProtoBufMessage().toByteArray().length;
blockLength = blockManager.putBlock(kvContainer, blockData);
blockManager.putBlock(kvContainer, blockData);
metrics.incContainerBytesStats(Type.PutBlock, numBytes);
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
@ -418,7 +426,7 @@ public class KeyValueHandler extends Handler {
request);
}
return BlockUtils.putBlockResponseSuccess(request, blockLength);
return BlockUtils.putBlockResponseSuccess(request, blockData);
}
/**
@ -514,7 +522,8 @@ public class KeyValueHandler extends Handler {
* Handle Read Chunk operation. Calls ChunkManager to process the request.
*/
ContainerCommandResponseProto handleReadChunk(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasReadChunk()) {
LOG.debug("Malformed Read Chunk request. trace ID: {}",
@ -531,8 +540,10 @@ public class KeyValueHandler extends Handler {
.getChunkData());
Preconditions.checkNotNull(chunkInfo);
data = chunkManager.readChunk(kvContainer, blockID, chunkInfo,
request.getReadChunk().getReadFromTmpFile());
boolean isReadFromTmpFile = dispatcherContext == null ? false :
dispatcherContext.isReadFromTmpFile();
data = chunkManager
.readChunk(kvContainer, blockID, chunkInfo, isReadFromTmpFile);
metrics.incContainerBytesStats(Type.ReadChunk, data.length);
} catch (StorageContainerException ex) {
return ContainerUtils.logAndReturnError(LOG, ex, request);
@ -583,7 +594,8 @@ public class KeyValueHandler extends Handler {
* Handle Write Chunk operation. Calls ChunkManager to process the request.
*/
ContainerCommandResponseProto handleWriteChunk(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasWriteChunk()) {
LOG.debug("Malformed Write Chunk request. trace ID: {}",
@ -602,17 +614,19 @@ public class KeyValueHandler extends Handler {
Preconditions.checkNotNull(chunkInfo);
ByteBuffer data = null;
if (request.getWriteChunk().getStage() == Stage.WRITE_DATA ||
request.getWriteChunk().getStage() == Stage.COMBINED) {
WriteChunkStage stage =
dispatcherContext == null ? WriteChunkStage.COMBINED :
dispatcherContext.getStage();
if (stage == WriteChunkStage.WRITE_DATA ||
stage == WriteChunkStage.COMBINED) {
data = request.getWriteChunk().getData().asReadOnlyByteBuffer();
}
chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data,
request.getWriteChunk().getStage());
chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data, stage);
// We should increment stats after writeChunk
if (request.getWriteChunk().getStage() == Stage.WRITE_DATA ||
request.getWriteChunk().getStage() == Stage.COMBINED) {
if (stage == WriteChunkStage.WRITE_DATA||
stage == WriteChunkStage.COMBINED) {
metrics.incContainerBytesStats(Type.WriteChunk, request.getWriteChunk()
.getChunkData().getLen());
}
@ -633,7 +647,8 @@ public class KeyValueHandler extends Handler {
* request.
*/
ContainerCommandResponseProto handlePutSmallFile(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasPutSmallFile()) {
LOG.debug("Malformed Put Small File request. trace ID: {}",
@ -642,13 +657,14 @@ public class KeyValueHandler extends Handler {
}
PutSmallFileRequestProto putSmallFileReq =
request.getPutSmallFile();
BlockData blockData;
try {
checkContainerOpen(kvContainer);
BlockID blockID = BlockID.getFromProtobuf(putSmallFileReq.getBlock()
.getBlockData().getBlockID());
BlockData blockData = BlockData.getFromProtoBuf(
blockData = BlockData.getFromProtoBuf(
putSmallFileReq.getBlock().getBlockData());
Preconditions.checkNotNull(blockData);
@ -656,15 +672,20 @@ public class KeyValueHandler extends Handler {
putSmallFileReq.getChunkInfo());
Preconditions.checkNotNull(chunkInfo);
ByteBuffer data = putSmallFileReq.getData().asReadOnlyByteBuffer();
WriteChunkStage stage =
dispatcherContext == null ? WriteChunkStage.COMBINED :
dispatcherContext.getStage();
// chunks will be committed as a part of handling putSmallFile
// here. There is no need to maintain this info in openContainerBlockMap.
chunkManager.writeChunk(
kvContainer, blockID, chunkInfo, data, Stage.COMBINED);
chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data, stage);
List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
chunks.add(chunkInfo.getProtoBufMessage());
blockData.setChunks(chunks);
// TODO: add bcsId as a part of putSmallFile transaction
long bcsId =
dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
blockData.setBlockCommitSequenceId(bcsId);
blockManager.putBlock(kvContainer, blockData);
metrics.incContainerBytesStats(Type.PutSmallFile, data.capacity());
@ -676,7 +697,7 @@ public class KeyValueHandler extends Handler {
PUT_SMALL_FILE_ERROR), request);
}
return SmallFileUtils.getPutFileResponseSuccess(request);
return SmallFileUtils.getPutFileResponseSuccess(request, blockData);
}
/**

View File

@ -133,12 +133,12 @@ public final class BlockUtils {
* @return Response.
*/
public static ContainerCommandResponseProto putBlockResponseSuccess(
ContainerCommandRequestProto msg, long blockLength) {
ContainerProtos.BlockData blockData = msg.getPutBlock().getBlockData();
ContainerCommandRequestProto msg, BlockData blockData) {
ContainerProtos.BlockData blockDataProto = blockData.getProtoBufMessage();
GetCommittedBlockLengthResponseProto.Builder
committedBlockLengthResponseBuilder =
getCommittedBlockLengthResponseBuilder(blockLength,
blockData.getBlockID());
getCommittedBlockLengthResponseBuilder(blockData.getSize(),
blockDataProto.getBlockID());
PutBlockResponseProto.Builder putKeyResponse =
PutBlockResponseProto.newBuilder();
putKeyResponse

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.keyvalue.helpers;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
@ -44,15 +45,14 @@ public final class SmallFileUtils {
* @return - ContainerCommandResponseProto
*/
public static ContainerCommandResponseProto getPutFileResponseSuccess(
ContainerCommandRequestProto msg) {
ContainerCommandRequestProto msg, BlockData blockData) {
ContainerProtos.PutSmallFileResponseProto.Builder getResponse =
ContainerProtos.PutSmallFileResponseProto.newBuilder();
ContainerProtos.BlockData blockData =
msg.getPutSmallFile().getBlock().getBlockData();
ContainerProtos.BlockData blockDataProto = blockData.getProtoBufMessage();
ContainerProtos.GetCommittedBlockLengthResponseProto.Builder
committedBlockLengthResponseBuilder = BlockUtils
.getCommittedBlockLengthResponseBuilder(blockData.getSize(),
blockData.getBlockID());
.getCommittedBlockLengthResponseBuilder(blockDataProto.getSize(),
blockDataProto.getBlockID());
getResponse.setCommittedBlockLength(committedBlockLengthResponseBuilder);
ContainerCommandResponseProto.Builder builder =
ContainerUtils.getSuccessResponseBuilder(msg);

View File

@ -21,10 +21,10 @@ package org.apache.hadoop.ozone.container.keyvalue.impl;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
@ -66,7 +66,7 @@ public class ChunkManagerImpl implements ChunkManager {
* @throws StorageContainerException
*/
public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
ByteBuffer data, ContainerProtos.Stage stage)
ByteBuffer data, DispatcherContext.WriteChunkStage stage)
throws StorageContainerException {
try {

View File

@ -19,10 +19,11 @@ package org.apache.hadoop.ozone.container.keyvalue.interfaces;
*/
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import java.nio.ByteBuffer;
/**
@ -42,7 +43,7 @@ public interface ChunkManager {
* @throws StorageContainerException
*/
void writeChunk(Container container, BlockID blockID, ChunkInfo info,
ByteBuffer data, ContainerProtos.Stage stage)
ByteBuffer data, DispatcherContext.WriteChunkStage stage)
throws StorageContainerException;
/**

View File

@ -99,16 +99,16 @@ public class TestHddsDispatcher {
HddsDispatcher hddsDispatcher = new HddsDispatcher(
conf, containerSet, volumeSet, handlers, context, metrics);
hddsDispatcher.setScmId(scmId.toString());
ContainerCommandResponseProto responseOne = hddsDispatcher.dispatch(
getWriteChunkRequest(dd.getUuidString(), 1L, 1L));
ContainerCommandResponseProto responseOne = hddsDispatcher
.dispatch(getWriteChunkRequest(dd.getUuidString(), 1L, 1L), null);
Assert.assertEquals(ContainerProtos.Result.SUCCESS,
responseOne.getResult());
verify(context, times(0))
.addContainerActionIfAbsent(Mockito.any(ContainerAction.class));
containerData.setBytesUsed(Double.valueOf(
StorageUnit.MB.toBytes(950)).longValue());
ContainerCommandResponseProto responseTwo = hddsDispatcher.dispatch(
getWriteChunkRequest(dd.getUuidString(), 1L, 2L));
ContainerCommandResponseProto responseTwo = hddsDispatcher
.dispatch(getWriteChunkRequest(dd.getUuidString(), 1L, 2L), null);
Assert.assertEquals(ContainerProtos.Result.SUCCESS,
responseTwo.getResult());
verify(context, times(1))
@ -150,16 +150,16 @@ public class TestHddsDispatcher {
getWriteChunkRequest(dd.getUuidString(), 1L, 1L);
// send read chunk request and make sure container does not exist
ContainerCommandResponseProto response =
hddsDispatcher.dispatch(getReadChunkRequest(writeChunkRequest));
hddsDispatcher.dispatch(getReadChunkRequest(writeChunkRequest), null);
Assert.assertEquals(response.getResult(),
ContainerProtos.Result.CONTAINER_NOT_FOUND);
// send write chunk request without sending create container
response = hddsDispatcher.dispatch(writeChunkRequest);
response = hddsDispatcher.dispatch(writeChunkRequest, null);
// container should be created as part of write chunk request
Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
// send read chunk request to read the chunk written above
response =
hddsDispatcher.dispatch(getReadChunkRequest(writeChunkRequest));
hddsDispatcher.dispatch(getReadChunkRequest(writeChunkRequest), null);
Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
Assert.assertEquals(response.getReadChunk().getData(),
writeChunkRequest.getWriteChunk().getData());

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage;
import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
@ -114,7 +115,7 @@ public class TestChunkManagerImpl {
// As no chunks are written to the volume writeBytes should be 0
checkWriteIOStats(0, 0);
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.WRITE_DATA);
ByteBuffer.wrap(data), WriteChunkStage.WRITE_DATA);
// Now a chunk file is being written with Stage WRITE_DATA, so it should
// create a temporary chunk file.
assertTrue(chunksPath.listFiles().length == 1);
@ -131,7 +132,7 @@ public class TestChunkManagerImpl {
checkWriteIOStats(data.length, 1);
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.COMMIT_DATA);
ByteBuffer.wrap(data), WriteChunkStage.COMMIT_DATA);
checkWriteIOStats(data.length, 1);
@ -151,7 +152,7 @@ public class TestChunkManagerImpl {
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
.getLocalID(), 0), 0, randomLength);
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.WRITE_DATA);
ByteBuffer.wrap(data), WriteChunkStage.WRITE_DATA);
fail("testWriteChunkIncorrectLength failed");
} catch (StorageContainerException ex) {
// As we got an exception, writeBytes should be 0.
@ -172,7 +173,7 @@ public class TestChunkManagerImpl {
assertTrue(chunksPath.listFiles().length == 0);
checkWriteIOStats(0, 0);
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.COMBINED);
ByteBuffer.wrap(data), WriteChunkStage.COMBINED);
// Now a chunk file is being written with Stage COMBINED_DATA, so it should
// create a chunk file.
assertTrue(chunksPath.listFiles().length == 1);
@ -185,7 +186,7 @@ public class TestChunkManagerImpl {
public void testReadChunk() throws Exception {
checkWriteIOStats(0, 0);
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.COMBINED);
ByteBuffer.wrap(data), WriteChunkStage.COMBINED);
checkWriteIOStats(data.length, 1);
checkReadIOStats(0, 0);
byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
@ -199,7 +200,7 @@ public class TestChunkManagerImpl {
public void testDeleteChunk() throws Exception {
File chunksPath = new File(keyValueContainerData.getChunksPath());
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.COMBINED);
ByteBuffer.wrap(data), WriteChunkStage.COMBINED);
assertTrue(chunksPath.listFiles().length == 1);
chunkManager.deleteChunk(keyValueContainer, blockID, chunkInfo);
assertTrue(chunksPath.listFiles().length == 0);
@ -209,7 +210,7 @@ public class TestChunkManagerImpl {
public void testDeleteChunkUnsupportedRequest() throws Exception {
try {
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.COMBINED);
ByteBuffer.wrap(data), WriteChunkStage.COMBINED);
long randomLength = 200L;
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
.getLocalID(), 0), 0, randomLength);
@ -241,7 +242,7 @@ public class TestChunkManagerImpl {
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
.getLocalID(), i), 0, data.length);
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo,
ByteBuffer.wrap(data), ContainerProtos.Stage.COMBINED);
ByteBuffer.wrap(data), WriteChunkStage.COMBINED);
}
checkWriteIOStats(data.length*100, 100);
assertTrue(hddsVolume.getVolumeIOStats().getWriteTime() > 0);

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
@ -84,10 +85,10 @@ public class TestKeyValueHandler {
handler = Mockito.mock(KeyValueHandler.class);
dispatcher = Mockito.mock(HddsDispatcher.class);
Mockito.when(dispatcher.getHandler(any())).thenReturn(handler);
Mockito.when(dispatcher.dispatch(any())).thenCallRealMethod();
Mockito.when(dispatcher.dispatch(any(), any())).thenCallRealMethod();
Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
Mockito.mock(KeyValueContainer.class));
Mockito.when(handler.handle(any(), any())).thenCallRealMethod();
Mockito.when(handler.handle(any(), any(), any())).thenCallRealMethod();
doCallRealMethod().when(dispatcher).setMetricsForTesting(any());
dispatcher.setMetricsForTesting(Mockito.mock(ContainerMetrics.class));
Mockito.when(dispatcher.buildAuditMessageForFailure(any(), any(), any()))
@ -111,112 +112,113 @@ public class TestKeyValueHandler {
.setCreateContainer(ContainerProtos.CreateContainerRequestProto
.getDefaultInstance())
.build();
dispatcher.dispatch(createContainerRequest);
DispatcherContext context = new DispatcherContext.Builder().build();
dispatcher.dispatch(createContainerRequest, context);
Mockito.verify(handler, times(1)).handleCreateContainer(
any(ContainerCommandRequestProto.class), any());
// Test Read Container Request handling
ContainerCommandRequestProto readContainerRequest =
getDummyCommandRequestProto(ContainerProtos.Type.ReadContainer);
dispatcher.dispatch(readContainerRequest);
dispatcher.dispatch(readContainerRequest, context);
Mockito.verify(handler, times(1)).handleReadContainer(
any(ContainerCommandRequestProto.class), any());
// Test Update Container Request handling
ContainerCommandRequestProto updateContainerRequest =
getDummyCommandRequestProto(ContainerProtos.Type.UpdateContainer);
dispatcher.dispatch(updateContainerRequest);
dispatcher.dispatch(updateContainerRequest, context);
Mockito.verify(handler, times(1)).handleUpdateContainer(
any(ContainerCommandRequestProto.class), any());
// Test Delete Container Request handling
ContainerCommandRequestProto deleteContainerRequest =
getDummyCommandRequestProto(ContainerProtos.Type.DeleteContainer);
dispatcher.dispatch(deleteContainerRequest);
dispatcher.dispatch(deleteContainerRequest, null);
Mockito.verify(handler, times(1)).handleDeleteContainer(
any(ContainerCommandRequestProto.class), any());
// Test List Container Request handling
ContainerCommandRequestProto listContainerRequest =
getDummyCommandRequestProto(ContainerProtos.Type.ListContainer);
dispatcher.dispatch(listContainerRequest);
dispatcher.dispatch(listContainerRequest, context);
Mockito.verify(handler, times(1)).handleUnsupportedOp(
any(ContainerCommandRequestProto.class));
// Test Close Container Request handling
ContainerCommandRequestProto closeContainerRequest =
getDummyCommandRequestProto(ContainerProtos.Type.CloseContainer);
dispatcher.dispatch(closeContainerRequest);
dispatcher.dispatch(closeContainerRequest, context);
Mockito.verify(handler, times(1)).handleCloseContainer(
any(ContainerCommandRequestProto.class), any());
// Test Put Block Request handling
ContainerCommandRequestProto putBlockRequest =
getDummyCommandRequestProto(ContainerProtos.Type.PutBlock);
dispatcher.dispatch(putBlockRequest);
dispatcher.dispatch(putBlockRequest, context);
Mockito.verify(handler, times(1)).handlePutBlock(
any(ContainerCommandRequestProto.class), any());
any(ContainerCommandRequestProto.class), any(), any());
// Test Get Block Request handling
ContainerCommandRequestProto getBlockRequest =
getDummyCommandRequestProto(ContainerProtos.Type.GetBlock);
dispatcher.dispatch(getBlockRequest);
dispatcher.dispatch(getBlockRequest, context);
Mockito.verify(handler, times(1)).handleGetBlock(
any(ContainerCommandRequestProto.class), any());
// Test Delete Block Request handling
ContainerCommandRequestProto deleteBlockRequest =
getDummyCommandRequestProto(ContainerProtos.Type.DeleteBlock);
dispatcher.dispatch(deleteBlockRequest);
dispatcher.dispatch(deleteBlockRequest, context);
Mockito.verify(handler, times(1)).handleDeleteBlock(
any(ContainerCommandRequestProto.class), any());
// Test List Block Request handling
ContainerCommandRequestProto listBlockRequest =
getDummyCommandRequestProto(ContainerProtos.Type.ListBlock);
dispatcher.dispatch(listBlockRequest);
dispatcher.dispatch(listBlockRequest, context);
Mockito.verify(handler, times(2)).handleUnsupportedOp(
any(ContainerCommandRequestProto.class));
// Test Read Chunk Request handling
ContainerCommandRequestProto readChunkRequest =
getDummyCommandRequestProto(ContainerProtos.Type.ReadChunk);
dispatcher.dispatch(readChunkRequest);
dispatcher.dispatch(readChunkRequest, context);
Mockito.verify(handler, times(1)).handleReadChunk(
any(ContainerCommandRequestProto.class), any());
any(ContainerCommandRequestProto.class), any(), any());
// Test Delete Chunk Request handling
ContainerCommandRequestProto deleteChunkRequest =
getDummyCommandRequestProto(ContainerProtos.Type.DeleteChunk);
dispatcher.dispatch(deleteChunkRequest);
dispatcher.dispatch(deleteChunkRequest, context);
Mockito.verify(handler, times(1)).handleDeleteChunk(
any(ContainerCommandRequestProto.class), any());
// Test Write Chunk Request handling
ContainerCommandRequestProto writeChunkRequest =
getDummyCommandRequestProto(ContainerProtos.Type.WriteChunk);
dispatcher.dispatch(writeChunkRequest);
dispatcher.dispatch(writeChunkRequest, context);
Mockito.verify(handler, times(1)).handleWriteChunk(
any(ContainerCommandRequestProto.class), any());
any(ContainerCommandRequestProto.class), any(), any());
// Test List Chunk Request handling
ContainerCommandRequestProto listChunkRequest =
getDummyCommandRequestProto(ContainerProtos.Type.ListChunk);
dispatcher.dispatch(listChunkRequest);
dispatcher.dispatch(listChunkRequest, context);
Mockito.verify(handler, times(3)).handleUnsupportedOp(
any(ContainerCommandRequestProto.class));
// Test Put Small File Request handling
ContainerCommandRequestProto putSmallFileRequest =
getDummyCommandRequestProto(ContainerProtos.Type.PutSmallFile);
dispatcher.dispatch(putSmallFileRequest);
dispatcher.dispatch(putSmallFileRequest, context);
Mockito.verify(handler, times(1)).handlePutSmallFile(
any(ContainerCommandRequestProto.class), any());
any(ContainerCommandRequestProto.class), any(), any());
// Test Get Small File Request handling
ContainerCommandRequestProto getSmallFileRequest =
getDummyCommandRequestProto(ContainerProtos.Type.GetSmallFile);
dispatcher.dispatch(getSmallFileRequest);
dispatcher.dispatch(getSmallFileRequest, context);
Mockito.verify(handler, times(1)).handleGetSmallFile(
any(ContainerCommandRequestProto.class), any());
}
@ -294,7 +296,7 @@ public class TestKeyValueHandler {
.setCloseContainer(ContainerProtos.CloseContainerRequestProto
.getDefaultInstance())
.build();
dispatcher.dispatch(closeContainerRequest);
dispatcher.dispatch(closeContainerRequest, null);
Mockito.when(handler.handleCloseContainer(any(), any()))
.thenCallRealMethod();

View File

@ -78,7 +78,7 @@ import java.util.UUID;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BCSID_MISMATCH;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID;
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Stage.COMBINED;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
import static org.apache.hadoop.ozone.container.ContainerTestHelper.setDataChecksum;
@ -334,7 +334,7 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
return info;
}
@ -375,7 +375,7 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
fileHashMap.put(fileName, info);
}
@ -433,7 +433,7 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
byte[] readData = chunkManager.readChunk(container, blockID, info, false);
assertTrue(Arrays.equals(data, readData));
@ -466,13 +466,13 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
// With the overwrite flag it should work now.
info.addMetadata(OzoneConsts.CHUNK_OVERWRITE, "true");
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
long bytesUsed = container.getContainerData().getBytesUsed();
Assert.assertEquals(datalen, bytesUsed);
@ -507,7 +507,7 @@ public class TestContainerPersistence {
oldSha.update(data);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
}
// Request to read the whole data in a single go.
@ -540,7 +540,7 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
chunkManager.deleteChunk(container, blockID, info);
exception.expect(StorageContainerException.class);
exception.expectMessage("Unable to find the chunk file.");
@ -655,7 +655,7 @@ public class TestContainerPersistence {
byte[] data = getData(datalen);
setDataChecksum(info, data);
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
COMBINED);
WriteChunkStage.COMBINED);
totalSize += datalen;
chunkList.add(info);
}

View File

@ -169,7 +169,8 @@ public class TestCSMMetrics {
*/
@Override
public ContainerCommandResponseProto dispatch(
ContainerCommandRequestProto msg) {
ContainerCommandRequestProto msg,
DispatcherContext context) {
return ContainerTestHelper.getCreateContainerResponse(msg);
}

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.ozone.container.replication.GrpcReplicationService;
@ -236,8 +237,9 @@ public class TestContainerServer {
* @return Command Response
*/
@Override
public ContainerCommandResponseProto
dispatch(ContainerCommandRequestProto msg) {
public ContainerCommandResponseProto dispatch(
ContainerCommandRequestProto msg,
DispatcherContext context) {
return ContainerTestHelper.getCreateContainerResponse(msg);
}

View File

@ -130,7 +130,7 @@ public class BenchMarkDatanodeDispatcher {
for (int x = 0; x < INIT_CONTAINERS; x++) {
long containerID = HddsUtils.getUtcTime() + x;
ContainerCommandRequestProto req = getCreateContainerCommand(containerID);
dispatcher.dispatch(req);
dispatcher.dispatch(req, null);
containers.add(containerID);
containerCount.getAndIncrement();
}
@ -153,8 +153,8 @@ public class BenchMarkDatanodeDispatcher {
long containerID = containers.get(y);
BlockID blockID = new BlockID(containerID, key);
dispatcher
.dispatch(getPutBlockCommand(blockID, chunkName));
dispatcher.dispatch(getWriteChunkCommand(blockID, chunkName));
.dispatch(getPutBlockCommand(blockID, chunkName), null);
dispatcher.dispatch(getWriteChunkCommand(blockID, chunkName), null);
}
}
}
@ -268,7 +268,7 @@ public class BenchMarkDatanodeDispatcher {
public void createContainer(BenchMarkDatanodeDispatcher bmdd) {
long containerID = RandomUtils.nextLong();
ContainerCommandRequestProto req = getCreateContainerCommand(containerID);
bmdd.dispatcher.dispatch(req);
bmdd.dispatcher.dispatch(req, null);
bmdd.containers.add(containerID);
bmdd.containerCount.getAndIncrement();
}
@ -277,27 +277,27 @@ public class BenchMarkDatanodeDispatcher {
@Benchmark
public void writeChunk(BenchMarkDatanodeDispatcher bmdd) {
bmdd.dispatcher.dispatch(getWriteChunkCommand(
getRandomBlockID(), getNewChunkToWrite()));
getRandomBlockID(), getNewChunkToWrite()), null);
}
@Benchmark
public void readChunk(BenchMarkDatanodeDispatcher bmdd) {
BlockID blockID = getRandomBlockID();
String chunkKey = getRandomChunkToRead();
bmdd.dispatcher.dispatch(getReadChunkCommand(blockID, chunkKey));
bmdd.dispatcher.dispatch(getReadChunkCommand(blockID, chunkKey), null);
}
@Benchmark
public void putBlock(BenchMarkDatanodeDispatcher bmdd) {
BlockID blockID = getRandomBlockID();
String chunkKey = getNewChunkToWrite();
bmdd.dispatcher.dispatch(getPutBlockCommand(blockID, chunkKey));
bmdd.dispatcher.dispatch(getPutBlockCommand(blockID, chunkKey), null);
}
@Benchmark
public void getBlock(BenchMarkDatanodeDispatcher bmdd) {
BlockID blockID = getRandomBlockID();
bmdd.dispatcher.dispatch(getGetBlockCommand(blockID));
bmdd.dispatcher.dispatch(getGetBlockCommand(blockID), null);
}
// Chunks writes from benchmark only reaches certain containers