|
@@ -45,7 +45,8 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK;
|
|
|
-
|
|
|
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID;
|
|
|
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BCSID_MISMATCH;
|
|
|
/**
|
|
|
* This class is for performing block related operations on the KeyValue
|
|
|
* Container.
|
|
@@ -90,24 +91,23 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
// Should never fail.
|
|
|
Preconditions.checkNotNull(db, "DB cannot be null here");
|
|
|
|
|
|
- long blockCommitSequenceId = data.getBlockCommitSequenceId();
|
|
|
- byte[] blockCommitSequenceIdValue = db.get(blockCommitSequenceIdKey);
|
|
|
+ long bcsId = data.getBlockCommitSequenceId();
|
|
|
+ long containerBCSId = ((KeyValueContainerData) container.getContainerData())
|
|
|
+ .getBlockCommitSequenceId();
|
|
|
|
|
|
// default blockCommitSequenceId for any block is 0. It the putBlock
|
|
|
// request is not coming via Ratis(for test scenarios), it will be 0.
|
|
|
// In such cases, we should overwrite the block as well
|
|
|
- if (blockCommitSequenceIdValue != null && blockCommitSequenceId != 0) {
|
|
|
- if (blockCommitSequenceId <= Longs
|
|
|
- .fromByteArray(blockCommitSequenceIdValue)) {
|
|
|
+ if (bcsId != 0) {
|
|
|
+ if (bcsId <= containerBCSId) {
|
|
|
// Since the blockCommitSequenceId stored in the db is greater than
|
|
|
// equal to blockCommitSequenceId to be updated, it means the putBlock
|
|
|
// transaction is reapplied in the ContainerStateMachine on restart.
|
|
|
// It also implies that the given block must already exist in the db.
|
|
|
// just log and return
|
|
|
- LOG.warn("blockCommitSequenceId " + Longs
|
|
|
- .fromByteArray(blockCommitSequenceIdValue)
|
|
|
+ LOG.warn("blockCommitSequenceId " + containerBCSId
|
|
|
+ " in the Container Db is greater than" + " the supplied value "
|
|
|
- + blockCommitSequenceId + " .Ignoring it");
|
|
|
+ + bcsId + " .Ignoring it");
|
|
|
return data.getSize();
|
|
|
}
|
|
|
}
|
|
@@ -116,9 +116,9 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
batch.put(Longs.toByteArray(data.getLocalID()),
|
|
|
data.getProtoBufMessage().toByteArray());
|
|
|
batch.put(blockCommitSequenceIdKey,
|
|
|
- Longs.toByteArray(blockCommitSequenceId));
|
|
|
+ Longs.toByteArray(bcsId));
|
|
|
db.writeBatch(batch);
|
|
|
- container.updateBlockCommitSequenceId(blockCommitSequenceId);
|
|
|
+ container.updateBlockCommitSequenceId(bcsId);
|
|
|
// Increment keycount here
|
|
|
container.getContainerData().incrKeyCount();
|
|
|
return data.getSize();
|
|
@@ -129,10 +129,12 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
*
|
|
|
* @param container - Container from which block need to be fetched.
|
|
|
* @param blockID - BlockID of the block.
|
|
|
+ * @param bcsId latest commit Id of the block
|
|
|
* @return Key Data.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public BlockData getBlock(Container container, BlockID blockID)
|
|
|
+ @Override
|
|
|
+ public BlockData getBlock(Container container, BlockID blockID, long bcsId)
|
|
|
throws IOException {
|
|
|
Preconditions.checkNotNull(blockID,
|
|
|
"BlockID cannot be null in GetBlock request");
|
|
@@ -145,6 +147,14 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
// This is a post condition that acts as a hint to the user.
|
|
|
// Should never fail.
|
|
|
Preconditions.checkNotNull(db, "DB cannot be null here");
|
|
|
+
|
|
|
+ long containerBCSId = containerData.getBlockCommitSequenceId();
|
|
|
+ if (containerBCSId < bcsId) {
|
|
|
+ throw new StorageContainerException(
|
|
|
+ "Unable to find the block with bcsID " + bcsId + " .Container "
|
|
|
+ + container.getContainerData().getContainerID() + " bcsId is "
|
|
|
+ + containerBCSId + ".", UNKNOWN_BCSID);
|
|
|
+ }
|
|
|
byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID()));
|
|
|
if (kData == null) {
|
|
|
throw new StorageContainerException("Unable to find the block.",
|
|
@@ -152,6 +162,12 @@ public class BlockManagerImpl implements BlockManager {
|
|
|
}
|
|
|
ContainerProtos.BlockData blockData =
|
|
|
ContainerProtos.BlockData.parseFrom(kData);
|
|
|
+ long id = blockData.getBlockCommitSequenceId();
|
|
|
+ if (id < bcsId) {
|
|
|
+ throw new StorageContainerException(
|
|
|
+ "bcsId " + bcsId + " mismatches with existing block Id "
|
|
|
+ + id + " for block " + blockID + ".", BCSID_MISMATCH);
|
|
|
+ }
|
|
|
return BlockData.getFromProtoBuf(blockData);
|
|
|
}
|
|
|
|