HDDS-708. Validate BCSID while reading blocks from containers in datanodes. Contributed by Shashikant Banerjee.
This commit is contained in:
parent
0b62983c5a
commit
b61846392e
@ -81,14 +81,17 @@ private ContainerProtocolCalls() {
|
||||
* @param xceiverClient client to perform call
|
||||
* @param datanodeBlockID blockID to identify container
|
||||
* @param traceID container protocol call args
|
||||
* @param blockCommitSequenceId latest commit Id of the block
|
||||
* @return container protocol get block response
|
||||
* @throws IOException if there is an I/O error while performing the call
|
||||
*/
|
||||
public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
|
||||
DatanodeBlockID datanodeBlockID, String traceID) throws IOException {
|
||||
DatanodeBlockID datanodeBlockID, String traceID,
|
||||
long blockCommitSequenceId) throws IOException {
|
||||
GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto
|
||||
.newBuilder()
|
||||
.setBlockID(datanodeBlockID);
|
||||
.setBlockID(datanodeBlockID)
|
||||
.setBlockCommitSequenceId(blockCommitSequenceId);
|
||||
String id = xceiverClient.getPipeline().getLeader().getUuidString();
|
||||
|
||||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
||||
|
@ -140,6 +140,8 @@ enum Result {
|
||||
UNKNOWN_CONTAINER_TYPE = 34;
|
||||
BLOCK_NOT_COMMITTED = 35;
|
||||
CONTAINER_UNHEALTHY = 36;
|
||||
UNKNOWN_BCSID = 37;
|
||||
BCSID_MISMATCH = 38;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -315,6 +317,7 @@ message PutBlockResponseProto {
|
||||
|
||||
message GetBlockRequestProto {
|
||||
required DatanodeBlockID blockID = 1;
|
||||
optional uint64 blockCommitSequenceId = 2 [default = 0];
|
||||
}
|
||||
|
||||
message GetBlockResponseProto {
|
||||
@ -333,7 +336,7 @@ message GetCommittedBlockLengthRequestProto {
|
||||
message GetCommittedBlockLengthResponseProto {
|
||||
required DatanodeBlockID blockID = 1;
|
||||
required int64 blockLength = 2;
|
||||
optional uint64 blockCommitSequenceId = 3;
|
||||
optional uint64 blockCommitSequenceId = 3 [default = 0];
|
||||
}
|
||||
|
||||
message DeleteBlockResponseProto {
|
||||
|
@ -483,7 +483,8 @@ ContainerCommandResponseProto handleGetBlock(
|
||||
try {
|
||||
BlockID blockID = BlockID.getFromProtobuf(
|
||||
request.getGetBlock().getBlockID());
|
||||
responseData = blockManager.getBlock(kvContainer, blockID);
|
||||
responseData = blockManager.getBlock(kvContainer, blockID,
|
||||
request.getGetBlock().getBlockCommitSequenceId());
|
||||
long numBytes = responseData.getProtoBufMessage().toByteArray().length;
|
||||
metrics.incContainerBytesStats(Type.GetBlock, numBytes);
|
||||
|
||||
@ -722,6 +723,7 @@ ContainerCommandResponseProto handlePutSmallFile(
|
||||
List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
|
||||
chunks.add(chunkInfo.getProtoBufMessage());
|
||||
blockData.setChunks(chunks);
|
||||
// TODO: add bcsId as a part of putSmallFile transaction
|
||||
blockManager.putBlock(kvContainer, blockData);
|
||||
metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
|
||||
|
||||
@ -755,7 +757,9 @@ ContainerCommandResponseProto handleGetSmallFile(
|
||||
try {
|
||||
BlockID blockID = BlockID.getFromProtobuf(getSmallFileReq.getBlock()
|
||||
.getBlockID());
|
||||
BlockData responseData = blockManager.getBlock(kvContainer, blockID);
|
||||
// TODO: add bcsId as a part of getSmallFile transaction
|
||||
// by default its 0
|
||||
BlockData responseData = blockManager.getBlock(kvContainer, blockID, 0);
|
||||
|
||||
ContainerProtos.ChunkInfo chunkInfo = null;
|
||||
ByteString dataBuf = ByteString.EMPTY;
|
||||
|
@ -45,7 +45,8 @@
|
||||
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 long putBlock(Container container, BlockData data) throws IOException {
|
||||
// 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 long putBlock(Container container, BlockData data) throws IOException {
|
||||
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 long putBlock(Container container, BlockData data) throws IOException {
|
||||
*
|
||||
* @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 BlockData getBlock(Container container, BlockID blockID)
|
||||
// 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 BlockData getBlock(Container container, BlockID blockID)
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -45,10 +45,12 @@ public interface BlockManager {
|
||||
*
|
||||
* @param container - Container from which block need to be get.
|
||||
* @param blockID - BlockID of the Block.
|
||||
* @param bcsId latest commit id of the block
|
||||
* @return Block Data.
|
||||
* @throws IOException
|
||||
*/
|
||||
BlockData getBlock(Container container, BlockID blockID) throws IOException;
|
||||
BlockData getBlock(Container container, BlockID blockID, long bcsId)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Deletes an existing block.
|
||||
|
@ -113,7 +113,7 @@ public void testPutAndGetBlock() throws Exception {
|
||||
assertEquals(1, keyValueContainer.getContainerData().getKeyCount());
|
||||
//Get Block
|
||||
BlockData fromGetBlockData = blockManager.getBlock(keyValueContainer,
|
||||
blockData.getBlockID());
|
||||
blockData.getBlockID(), 0);
|
||||
|
||||
assertEquals(blockData.getContainerID(), fromGetBlockData.getContainerID());
|
||||
assertEquals(blockData.getLocalID(), fromGetBlockData.getLocalID());
|
||||
@ -139,7 +139,7 @@ public void testDeleteBlock() throws Exception {
|
||||
assertEquals(0,
|
||||
keyValueContainer.getContainerData().getKeyCount());
|
||||
try {
|
||||
blockManager.getBlock(keyValueContainer, blockID);
|
||||
blockManager.getBlock(keyValueContainer, blockID, 0);
|
||||
fail("testDeleteBlock");
|
||||
} catch (StorageContainerException ex) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
@ -197,7 +197,7 @@ public void testGetNoSuchBlock() throws Exception {
|
||||
keyValueContainer.getContainerData().getKeyCount());
|
||||
try {
|
||||
//Since the block has been deleted, we should not be able to find it
|
||||
blockManager.getBlock(keyValueContainer, blockID);
|
||||
blockManager.getBlock(keyValueContainer, blockID, 0);
|
||||
fail("testGetNoSuchBlock failed");
|
||||
} catch (StorageContainerException ex) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
|
@ -294,7 +294,8 @@ public static LengthInputStream getFromOmKeyInfo(
|
||||
ContainerProtos.DatanodeBlockID datanodeBlockID = blockID
|
||||
.getDatanodeBlockIDProtobuf();
|
||||
ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls
|
||||
.getBlock(xceiverClient, datanodeBlockID, requestId);
|
||||
.getBlock(xceiverClient, datanodeBlockID, requestId,
|
||||
omKeyLocationInfo.getBlockCommitSequenceId());
|
||||
List<ContainerProtos.ChunkInfo> chunks =
|
||||
response.getBlockData().getChunksList();
|
||||
for (ContainerProtos.ChunkInfo chunk : chunks) {
|
||||
|
@ -151,7 +151,7 @@ public void testContainerReplication() throws Exception {
|
||||
.getHandler(ContainerType.KeyValueContainer);
|
||||
|
||||
BlockData key = handler.getBlockManager()
|
||||
.getBlock(container, BlockID.getFromProtobuf(blockID));
|
||||
.getBlock(container, BlockID.getFromProtobuf(blockID), 0);
|
||||
|
||||
Assert.assertNotNull(key);
|
||||
Assert.assertEquals(1, key.getChunks().size());
|
||||
|
@ -256,6 +256,6 @@ public void testCloseContainer() throws Exception {
|
||||
openContainerBlockMap.getBlockDataMap(testContainerID));
|
||||
// Make sure the key got committed
|
||||
Assert.assertNotNull(handler.getBlockManager()
|
||||
.getBlock(container, blockID));
|
||||
.getBlock(container, blockID, 0));
|
||||
}
|
||||
}
|
@ -73,6 +73,8 @@
|
||||
import java.util.Map;
|
||||
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 static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
|
||||
import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
|
||||
@ -556,7 +558,63 @@ public void testPutBlock() throws IOException, NoSuchAlgorithmException {
|
||||
blockData.setChunks(chunkList);
|
||||
blockManager.putBlock(container, blockData);
|
||||
BlockData readBlockData = blockManager.
|
||||
getBlock(container, blockData.getBlockID());
|
||||
getBlock(container, blockData.getBlockID(), 0);
|
||||
ChunkInfo readChunk =
|
||||
ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(0));
|
||||
Assert.assertEquals(info.getChecksum(), readChunk.getChecksum());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests a put block and read block with invalid bcsId.
|
||||
*
|
||||
* @throws IOException
|
||||
* @throws NoSuchAlgorithmException
|
||||
*/
|
||||
@Test
|
||||
public void testPutBlockWithInvalidBCSId()
|
||||
throws IOException, NoSuchAlgorithmException {
|
||||
long testContainerID = getTestContainerID();
|
||||
Container container = addContainer(containerSet, testContainerID);
|
||||
|
||||
BlockID blockID1 = ContainerTestHelper.getTestBlockID(testContainerID);
|
||||
ChunkInfo info = writeChunkHelper(blockID1);
|
||||
BlockData blockData = new BlockData(blockID1);
|
||||
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
||||
chunkList.add(info.getProtoBufMessage());
|
||||
blockData.setChunks(chunkList);
|
||||
blockData.setBlockCommitSequenceId(3);
|
||||
blockManager.putBlock(container, blockData);
|
||||
chunkList.clear();
|
||||
|
||||
// write a 2nd block
|
||||
BlockID blockID2 = ContainerTestHelper.getTestBlockID(testContainerID);
|
||||
info = writeChunkHelper(blockID2);
|
||||
blockData = new BlockData(blockID2);
|
||||
chunkList.add(info.getProtoBufMessage());
|
||||
blockData.setChunks(chunkList);
|
||||
blockData.setBlockCommitSequenceId(4);
|
||||
blockManager.putBlock(container, blockData);
|
||||
BlockData readBlockData;
|
||||
try {
|
||||
// read with bcsId higher than container bcsId
|
||||
blockManager.
|
||||
getBlock(container, blockID1, 5);
|
||||
Assert.fail("Expected exception not thrown");
|
||||
} catch (StorageContainerException sce) {
|
||||
Assert.assertTrue(sce.getResult() == UNKNOWN_BCSID);
|
||||
}
|
||||
|
||||
try {
|
||||
// read with bcsId lower than container bcsId but greater than committed
|
||||
// bcsId.
|
||||
blockManager.
|
||||
getBlock(container, blockID1, 4);
|
||||
Assert.fail("Expected exception not thrown");
|
||||
} catch (StorageContainerException sce) {
|
||||
Assert.assertTrue(sce.getResult() == BCSID_MISMATCH);
|
||||
}
|
||||
readBlockData = blockManager.
|
||||
getBlock(container, blockData.getBlockID(), 4);
|
||||
ChunkInfo readChunk =
|
||||
ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(0));
|
||||
Assert.assertEquals(info.getChecksum(), readChunk.getChecksum());
|
||||
@ -608,7 +666,7 @@ public void testPutBlockWithLotsOfChunks() throws IOException,
|
||||
blockData.setChunks(chunkProtoList);
|
||||
blockManager.putBlock(container, blockData);
|
||||
BlockData readBlockData = blockManager.
|
||||
getBlock(container, blockData.getBlockID());
|
||||
getBlock(container, blockData.getBlockID(), 0);
|
||||
ChunkInfo lastChunk = chunkList.get(chunkList.size() - 1);
|
||||
ChunkInfo readChunk =
|
||||
ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(readBlockData
|
||||
@ -636,7 +694,7 @@ public void testDeleteBlock() throws IOException, NoSuchAlgorithmException {
|
||||
blockManager.deleteBlock(container, blockID);
|
||||
exception.expect(StorageContainerException.class);
|
||||
exception.expectMessage("Unable to find the block.");
|
||||
blockManager.getBlock(container, blockData.getBlockID());
|
||||
blockManager.getBlock(container, blockData.getBlockID(), 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -699,8 +699,8 @@ public void testDeleteKey() throws Exception {
|
||||
.KeyValueContainer);
|
||||
KeyValueContainer container = (KeyValueContainer) cm.getContainerSet()
|
||||
.getContainer(location.getBlockID().getContainerID());
|
||||
BlockData blockInfo = keyValueHandler
|
||||
.getBlockManager().getBlock(container, location.getBlockID());
|
||||
BlockData blockInfo = keyValueHandler.getBlockManager()
|
||||
.getBlock(container, location.getBlockID(), 0);
|
||||
KeyValueContainerData containerData =
|
||||
(KeyValueContainerData) container.getContainerData();
|
||||
File dataDir = new File(containerData.getChunksPath());
|
||||
|
Loading…
Reference in New Issue
Block a user