HDDS-450. Generate BlockCommitSequenceId in ContainerStateMachine for every commit operation in Ratis. Contributed by Shashikant Banerjee.
This commit is contained in:
parent
5b1cfcaeff
commit
7367ff333b
@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.storage;
|
package org.apache.hadoop.hdds.scm.storage;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
|
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
|
||||||
import org.apache.commons.codec.digest.DigestUtils;
|
import org.apache.commons.codec.digest.DigestUtils;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
@ -65,6 +66,7 @@ public class ChunkOutputStream extends OutputStream {
|
|||||||
private final String streamId;
|
private final String streamId;
|
||||||
private int chunkIndex;
|
private int chunkIndex;
|
||||||
private int chunkSize;
|
private int chunkSize;
|
||||||
|
private long blockCommitSequenceId;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new ChunkOutputStream.
|
* Creates a new ChunkOutputStream.
|
||||||
@ -93,12 +95,17 @@ public ChunkOutputStream(BlockID blockID, String key,
|
|||||||
this.buffer = ByteBuffer.allocate(chunkSize);
|
this.buffer = ByteBuffer.allocate(chunkSize);
|
||||||
this.streamId = UUID.randomUUID().toString();
|
this.streamId = UUID.randomUUID().toString();
|
||||||
this.chunkIndex = 0;
|
this.chunkIndex = 0;
|
||||||
|
blockCommitSequenceId = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getBuffer() {
|
public ByteBuffer getBuffer() {
|
||||||
return buffer;
|
return buffer;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getBlockCommitSequenceId() {
|
||||||
|
return blockCommitSequenceId;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void write(int b) throws IOException {
|
public void write(int b) throws IOException {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
@ -155,7 +162,10 @@ public void close() throws IOException {
|
|||||||
writeChunkToContainer();
|
writeChunkToContainer();
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
putBlock(xceiverClient, containerBlockData.build(), traceID);
|
ContainerProtos.PutBlockResponseProto responseProto =
|
||||||
|
putBlock(xceiverClient, containerBlockData.build(), traceID);
|
||||||
|
blockCommitSequenceId =
|
||||||
|
responseProto.getCommittedBlockLength().getBlockCommitSequenceId();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new IOException(
|
throw new IOException(
|
||||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||||
|
@ -141,24 +141,23 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
|
|||||||
* @param xceiverClient client to perform call
|
* @param xceiverClient client to perform call
|
||||||
* @param containerBlockData block data to identify container
|
* @param containerBlockData block data to identify container
|
||||||
* @param traceID container protocol call args
|
* @param traceID container protocol call args
|
||||||
|
* @return putBlockResponse
|
||||||
* @throws IOException if there is an I/O error while performing the call
|
* @throws IOException if there is an I/O error while performing the call
|
||||||
*/
|
*/
|
||||||
public static void putBlock(XceiverClientSpi xceiverClient,
|
public static ContainerProtos.PutBlockResponseProto putBlock(
|
||||||
BlockData containerBlockData, String traceID) throws IOException {
|
XceiverClientSpi xceiverClient, BlockData containerBlockData,
|
||||||
PutBlockRequestProto.Builder createBlockRequest = PutBlockRequestProto
|
String traceID) throws IOException {
|
||||||
.newBuilder()
|
PutBlockRequestProto.Builder createBlockRequest =
|
||||||
.setBlockData(containerBlockData);
|
PutBlockRequestProto.newBuilder().setBlockData(containerBlockData);
|
||||||
String id = xceiverClient.getPipeline().getLeader().getUuidString();
|
String id = xceiverClient.getPipeline().getLeader().getUuidString();
|
||||||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
ContainerCommandRequestProto request =
|
||||||
.newBuilder()
|
ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutBlock)
|
||||||
.setCmdType(Type.PutBlock)
|
.setContainerID(containerBlockData.getBlockID().getContainerID())
|
||||||
.setContainerID(containerBlockData.getBlockID().getContainerID())
|
.setTraceID(traceID).setDatanodeUuid(id)
|
||||||
.setTraceID(traceID)
|
.setPutBlock(createBlockRequest).build();
|
||||||
.setDatanodeUuid(id)
|
|
||||||
.setPutBlock(createBlockRequest)
|
|
||||||
.build();
|
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||||
validateContainerResponse(response);
|
validateContainerResponse(response);
|
||||||
|
return response.getPutBlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -34,6 +34,7 @@
|
|||||||
public class BlockData {
|
public class BlockData {
|
||||||
private final BlockID blockID;
|
private final BlockID blockID;
|
||||||
private final Map<String, String> metadata;
|
private final Map<String, String> metadata;
|
||||||
|
private long blockCommitSequenceId;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represent a list of chunks.
|
* Represent a list of chunks.
|
||||||
@ -64,6 +65,15 @@ public BlockData(BlockID blockID) {
|
|||||||
this.blockID = blockID;
|
this.blockID = blockID;
|
||||||
this.metadata = new TreeMap<>();
|
this.metadata = new TreeMap<>();
|
||||||
this.size = 0;
|
this.size = 0;
|
||||||
|
blockCommitSequenceId = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getBlockCommitSequenceId() {
|
||||||
|
return blockCommitSequenceId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBlockCommitSequenceId(long blockCommitSequenceId) {
|
||||||
|
this.blockCommitSequenceId = blockCommitSequenceId;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -85,6 +95,7 @@ public static BlockData getFromProtoBuf(ContainerProtos.BlockData data) throws
|
|||||||
if (data.hasSize()) {
|
if (data.hasSize()) {
|
||||||
Preconditions.checkArgument(data.getSize() == blockData.getSize());
|
Preconditions.checkArgument(data.getSize() == blockData.getSize());
|
||||||
}
|
}
|
||||||
|
blockData.setBlockCommitSequenceId(data.getBlockCommitSequenceId());
|
||||||
return blockData;
|
return blockData;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -104,6 +115,7 @@ public ContainerProtos.BlockData getProtoBufMessage() {
|
|||||||
}
|
}
|
||||||
builder.addAllChunks(getChunks());
|
builder.addAllChunks(getChunks());
|
||||||
builder.setSize(size);
|
builder.setSize(size);
|
||||||
|
builder.setBlockCommitSequenceId(blockCommitSequenceId);
|
||||||
return builder.build();
|
return builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -300,6 +300,7 @@ message BlockData {
|
|||||||
repeated KeyValue metadata = 3;
|
repeated KeyValue metadata = 3;
|
||||||
repeated ChunkInfo chunks = 4;
|
repeated ChunkInfo chunks = 4;
|
||||||
optional int64 size = 5;
|
optional int64 size = 5;
|
||||||
|
optional uint64 blockCommitSequenceId = 6;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Block Messages.
|
// Block Messages.
|
||||||
@ -331,6 +332,7 @@ message GetCommittedBlockLengthRequestProto {
|
|||||||
message GetCommittedBlockLengthResponseProto {
|
message GetCommittedBlockLengthResponseProto {
|
||||||
required DatanodeBlockID blockID = 1;
|
required DatanodeBlockID blockID = 1;
|
||||||
required int64 blockLength = 2;
|
required int64 blockLength = 2;
|
||||||
|
optional uint64 blockCommitSequenceId = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
message DeleteBlockResponseProto {
|
message DeleteBlockResponseProto {
|
||||||
|
@ -21,6 +21,8 @@
|
|||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
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.protocol.RaftGroup;
|
import org.apache.ratis.protocol.RaftGroup;
|
||||||
import org.apache.ratis.protocol.RaftGroupId;
|
import org.apache.ratis.protocol.RaftGroupId;
|
||||||
import org.apache.ratis.server.RaftServer;
|
import org.apache.ratis.server.RaftServer;
|
||||||
@ -506,17 +508,36 @@ CompletableFuture<Message> handleStateMachineData(
|
|||||||
// on a container
|
// on a container
|
||||||
|
|
||||||
private CompletableFuture<Message> handlePutBlock(
|
private CompletableFuture<Message> handlePutBlock(
|
||||||
ContainerCommandRequestProto requestProto) {
|
ContainerCommandRequestProto requestProto, long index) {
|
||||||
List<CompletableFuture<Message>> futureList = new ArrayList<>();
|
List<CompletableFuture<Message>> futureList = new ArrayList<>();
|
||||||
long localId =
|
BlockData blockData = null;
|
||||||
requestProto.getPutBlock().getBlockData().getBlockID().getLocalID();
|
ContainerProtos.BlockData blockDataProto =
|
||||||
|
requestProto.getPutBlock().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 =
|
||||||
|
ContainerCommandRequestProto.newBuilder(requestProto)
|
||||||
|
.setPutBlock(putBlockRequestProto).build();
|
||||||
|
long localId = blockDataProto.getBlockID().getLocalID();
|
||||||
// Need not wait for create container future here as it has already
|
// Need not wait for create container future here as it has already
|
||||||
// finished.
|
// finished.
|
||||||
if (block2ChunkMap.get(localId) != null) {
|
if (block2ChunkMap.get(localId) != null) {
|
||||||
futureList.addAll(block2ChunkMap.get(localId).getAll());
|
futureList.addAll(block2ChunkMap.get(localId).getAll());
|
||||||
}
|
}
|
||||||
CompletableFuture<Message> effectiveFuture =
|
CompletableFuture<Message> effectiveFuture =
|
||||||
runCommandAfterFutures(futureList, requestProto);
|
runCommandAfterFutures(futureList, containerCommandRequestProto);
|
||||||
|
|
||||||
CompletableFuture<Message> putBlockFuture =
|
CompletableFuture<Message> putBlockFuture =
|
||||||
effectiveFuture.thenApply(message -> {
|
effectiveFuture.thenApply(message -> {
|
||||||
@ -616,7 +637,7 @@ CompletableFuture<Message> executeContainerCommand(
|
|||||||
case CloseContainer:
|
case CloseContainer:
|
||||||
return handleCloseContainer(requestProto);
|
return handleCloseContainer(requestProto);
|
||||||
case PutBlock:
|
case PutBlock:
|
||||||
return handlePutBlock(requestProto);
|
return handlePutBlock(requestProto, index);
|
||||||
case CreateContainer:
|
case CreateContainer:
|
||||||
return handleCreateContainer(requestProto);
|
return handleCreateContainer(requestProto);
|
||||||
default:
|
default:
|
||||||
|
@ -133,10 +133,13 @@ public static BlockData getBlockData(byte[] bytes) throws IOException {
|
|||||||
*/
|
*/
|
||||||
public static ContainerCommandResponseProto putBlockResponseSuccess(
|
public static ContainerCommandResponseProto putBlockResponseSuccess(
|
||||||
ContainerCommandRequestProto msg, long blockLength) {
|
ContainerCommandRequestProto msg, long blockLength) {
|
||||||
|
ContainerProtos.BlockData blockData = msg.getPutBlock().getBlockData();
|
||||||
GetCommittedBlockLengthResponseProto.Builder
|
GetCommittedBlockLengthResponseProto.Builder
|
||||||
committedBlockLengthResponseBuilder =
|
committedBlockLengthResponseBuilder =
|
||||||
getCommittedBlockLengthResponseBuilder(blockLength,
|
getCommittedBlockLengthResponseBuilder(blockLength,
|
||||||
msg.getPutBlock().getBlockData().getBlockID());
|
blockData.getBlockID());
|
||||||
|
committedBlockLengthResponseBuilder
|
||||||
|
.setBlockCommitSequenceId(blockData.getBlockCommitSequenceId());
|
||||||
PutBlockResponseProto.Builder putKeyResponse =
|
PutBlockResponseProto.Builder putKeyResponse =
|
||||||
PutBlockResponseProto.newBuilder();
|
PutBlockResponseProto.newBuilder();
|
||||||
putKeyResponse
|
putKeyResponse
|
||||||
|
@ -85,7 +85,6 @@ public long putBlock(Container container, BlockData data) throws IOException {
|
|||||||
Preconditions.checkNotNull(db, "DB cannot be null here");
|
Preconditions.checkNotNull(db, "DB cannot be null here");
|
||||||
db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
|
db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
|
||||||
.toByteArray());
|
.toByteArray());
|
||||||
|
|
||||||
// Increment keycount here
|
// Increment keycount here
|
||||||
container.getContainerData().incrKeyCount();
|
container.getContainerData().incrKeyCount();
|
||||||
return data.getSize();
|
return data.getSize();
|
||||||
|
@ -117,13 +117,15 @@ public List<ChunkOutputStreamEntry> getStreamEntries() {
|
|||||||
return streamEntries;
|
return streamEntries;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<OmKeyLocationInfo> getLocationInfoList() {
|
public List<OmKeyLocationInfo> getLocationInfoList() throws IOException {
|
||||||
List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
|
List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
|
||||||
for (ChunkOutputStreamEntry streamEntry : streamEntries) {
|
for (ChunkOutputStreamEntry streamEntry : streamEntries) {
|
||||||
OmKeyLocationInfo info =
|
OmKeyLocationInfo info =
|
||||||
new OmKeyLocationInfo.Builder().setBlockID(streamEntry.blockID)
|
new OmKeyLocationInfo.Builder().setBlockID(streamEntry.blockID)
|
||||||
.setShouldCreateContainer(false)
|
.setShouldCreateContainer(false)
|
||||||
.setLength(streamEntry.currentPosition).setOffset(0).build();
|
.setLength(streamEntry.currentPosition).setOffset(0)
|
||||||
|
.setBlockCommitSequenceId(streamEntry.getBlockCommitSequenceId())
|
||||||
|
.build();
|
||||||
locationInfoList.add(info);
|
locationInfoList.add(info);
|
||||||
}
|
}
|
||||||
return locationInfoList;
|
return locationInfoList;
|
||||||
@ -153,8 +155,6 @@ public ChunkGroupOutputStream(
|
|||||||
this.chunkSize = chunkSize;
|
this.chunkSize = chunkSize;
|
||||||
this.requestID = requestId;
|
this.requestID = requestId;
|
||||||
this.retryPolicy = retryPolicy;
|
this.retryPolicy = retryPolicy;
|
||||||
LOG.debug("Expecting open key with one block, but got" +
|
|
||||||
info.getKeyLocationVersions().size());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -708,6 +708,14 @@ ByteBuffer getBuffer() throws IOException {
|
|||||||
throw new IOException("Invalid Output Stream for Key: " + key);
|
throw new IOException("Invalid Output Stream for Key: " + key);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
long getBlockCommitSequenceId() throws IOException {
|
||||||
|
if (this.outputStream instanceof ChunkOutputStream) {
|
||||||
|
ChunkOutputStream out = (ChunkOutputStream) this.outputStream;
|
||||||
|
return out.getBlockCommitSequenceId();
|
||||||
|
}
|
||||||
|
throw new IOException("Invalid Output Stream for Key: " + key);
|
||||||
|
}
|
||||||
|
|
||||||
public void cleanup() {
|
public void cleanup() {
|
||||||
checkStream();
|
checkStream();
|
||||||
if (this.outputStream instanceof ChunkOutputStream) {
|
if (this.outputStream instanceof ChunkOutputStream) {
|
||||||
|
@ -31,13 +31,15 @@ public final class OmKeyLocationInfo {
|
|||||||
private final long offset;
|
private final long offset;
|
||||||
// the version number indicating when this block was added
|
// the version number indicating when this block was added
|
||||||
private long createVersion;
|
private long createVersion;
|
||||||
|
private final long blockCommitSequenceId;
|
||||||
|
|
||||||
private OmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer,
|
private OmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer,
|
||||||
long length, long offset) {
|
long length, long offset, long blockCommitSequenceId) {
|
||||||
this.blockID = blockID;
|
this.blockID = blockID;
|
||||||
this.shouldCreateContainer = shouldCreateContainer;
|
this.shouldCreateContainer = shouldCreateContainer;
|
||||||
this.length = length;
|
this.length = length;
|
||||||
this.offset = offset;
|
this.offset = offset;
|
||||||
|
this.blockCommitSequenceId = blockCommitSequenceId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setCreateVersion(long version) {
|
public void setCreateVersion(long version) {
|
||||||
@ -84,6 +86,7 @@ public static class Builder {
|
|||||||
private boolean shouldCreateContainer;
|
private boolean shouldCreateContainer;
|
||||||
private long length;
|
private long length;
|
||||||
private long offset;
|
private long offset;
|
||||||
|
private long blockCommitSequenceId;
|
||||||
|
|
||||||
public Builder setBlockID(BlockID blockId) {
|
public Builder setBlockID(BlockID blockId) {
|
||||||
this.blockID = blockId;
|
this.blockID = blockId;
|
||||||
@ -105,9 +108,14 @@ public Builder setOffset(long off) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder setBlockCommitSequenceId(long sequenceId) {
|
||||||
|
this.blockCommitSequenceId = sequenceId;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public OmKeyLocationInfo build() {
|
public OmKeyLocationInfo build() {
|
||||||
return new OmKeyLocationInfo(blockID,
|
return new OmKeyLocationInfo(blockID,
|
||||||
shouldCreateContainer, length, offset);
|
shouldCreateContainer, length, offset, blockCommitSequenceId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -118,6 +126,7 @@ public KeyLocation getProtobuf() {
|
|||||||
.setLength(length)
|
.setLength(length)
|
||||||
.setOffset(offset)
|
.setOffset(offset)
|
||||||
.setCreateVersion(createVersion)
|
.setCreateVersion(createVersion)
|
||||||
|
.setBlockCommitSequenceId(blockCommitSequenceId)
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -126,7 +135,8 @@ public static OmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
|
|||||||
BlockID.getFromProtobuf(keyLocation.getBlockID()),
|
BlockID.getFromProtobuf(keyLocation.getBlockID()),
|
||||||
keyLocation.getShouldCreateContainer(),
|
keyLocation.getShouldCreateContainer(),
|
||||||
keyLocation.getLength(),
|
keyLocation.getLength(),
|
||||||
keyLocation.getOffset());
|
keyLocation.getOffset(),
|
||||||
|
keyLocation.getBlockCommitSequenceId());
|
||||||
info.setCreateVersion(keyLocation.getCreateVersion());
|
info.setCreateVersion(keyLocation.getCreateVersion());
|
||||||
return info;
|
return info;
|
||||||
}
|
}
|
||||||
@ -138,6 +148,7 @@ public String toString() {
|
|||||||
", shouldCreateContainer=" + shouldCreateContainer +
|
", shouldCreateContainer=" + shouldCreateContainer +
|
||||||
", length=" + length +
|
", length=" + length +
|
||||||
", offset=" + offset +
|
", offset=" + offset +
|
||||||
|
", blockCommitSequenceId=" + blockCommitSequenceId +
|
||||||
", createVersion=" + createVersion + '}';
|
", createVersion=" + createVersion + '}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -244,6 +244,7 @@ message KeyLocation {
|
|||||||
required uint64 length = 4;
|
required uint64 length = 4;
|
||||||
// indicated at which version this block gets created.
|
// indicated at which version this block gets created.
|
||||||
optional uint64 createVersion = 5;
|
optional uint64 createVersion = 5;
|
||||||
|
optional uint64 blockCommitSequenceId = 6;
|
||||||
}
|
}
|
||||||
|
|
||||||
message KeyLocationList {
|
message KeyLocationList {
|
||||||
|
@ -219,7 +219,7 @@ public void tesPutKeyResposne() throws Exception {
|
|||||||
ContainerProtos.PutBlockResponseProto response;
|
ContainerProtos.PutBlockResponseProto response;
|
||||||
String traceID = UUID.randomUUID().toString();
|
String traceID = UUID.randomUUID().toString();
|
||||||
ContainerWithPipeline container = storageContainerLocationClient
|
ContainerWithPipeline container = storageContainerLocationClient
|
||||||
.allocateContainer(xceiverClientManager.getType(),
|
.allocateContainer(HddsProtos.ReplicationType.RATIS,
|
||||||
HddsProtos.ReplicationFactor.ONE, containerOwner);
|
HddsProtos.ReplicationFactor.ONE, containerOwner);
|
||||||
long containerID = container.getContainerInfo().getContainerID();
|
long containerID = container.getContainerInfo().getContainerID();
|
||||||
Pipeline pipeline = container.getPipeline();
|
Pipeline pipeline = container.getPipeline();
|
||||||
@ -249,6 +249,8 @@ public void tesPutKeyResposne() throws Exception {
|
|||||||
blockID);
|
blockID);
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
response.getCommittedBlockLength().getBlockLength(), data.length);
|
response.getCommittedBlockLength().getBlockLength(), data.length);
|
||||||
|
Assert.assertTrue(
|
||||||
|
response.getCommittedBlockLength().getBlockCommitSequenceId() > 0);
|
||||||
xceiverClientManager.releaseClient(client);
|
xceiverClientManager.releaseClient(client);
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -172,6 +172,7 @@ public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID)
|
|||||||
.setBlockID(allocatedBlock.getBlockID())
|
.setBlockID(allocatedBlock.getBlockID())
|
||||||
.setShouldCreateContainer(allocatedBlock.getCreateContainer())
|
.setShouldCreateContainer(allocatedBlock.getCreateContainer())
|
||||||
.setLength(scmBlockSize)
|
.setLength(scmBlockSize)
|
||||||
|
.setBlockCommitSequenceId(0)
|
||||||
.setOffset(0)
|
.setOffset(0)
|
||||||
.build();
|
.build();
|
||||||
// current version not committed, so new blocks coming now are added to
|
// current version not committed, so new blocks coming now are added to
|
||||||
@ -236,6 +237,7 @@ public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
|||||||
.setBlockID(allocatedBlock.getBlockID())
|
.setBlockID(allocatedBlock.getBlockID())
|
||||||
.setShouldCreateContainer(allocatedBlock.getCreateContainer())
|
.setShouldCreateContainer(allocatedBlock.getCreateContainer())
|
||||||
.setLength(allocateSize)
|
.setLength(allocateSize)
|
||||||
|
.setBlockCommitSequenceId(0)
|
||||||
.setOffset(0)
|
.setOffset(0)
|
||||||
.build();
|
.build();
|
||||||
locations.add(subKeyInfo);
|
locations.add(subKeyInfo);
|
||||||
|
Loading…
Reference in New Issue
Block a user