|
@@ -32,6 +32,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
|
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto
|
|
|
+ .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
|
|
|
+ .DeleteBlockTransactionResult;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.utils.MetadataKeyFilters;
|
|
|
import org.apache.hadoop.utils.MetadataStore;
|
|
@@ -45,6 +48,7 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
@@ -56,7 +60,8 @@ import java.util.stream.Collectors;
|
|
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
|
|
.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
|
|
|
-import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Matchers.anyLong;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
/**
|
|
|
* Tests for DeletedBlockLog.
|
|
@@ -66,6 +71,8 @@ public class TestDeletedBlockLog {
|
|
|
private static DeletedBlockLogImpl deletedBlockLog;
|
|
|
private OzoneConfiguration conf;
|
|
|
private File testDir;
|
|
|
+ private Mapping containerManager;
|
|
|
+ private List<DatanodeDetails> dnList;
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws Exception {
|
|
@@ -74,7 +81,36 @@ public class TestDeletedBlockLog {
|
|
|
conf = new OzoneConfiguration();
|
|
|
conf.setInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 20);
|
|
|
conf.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
|
|
|
- deletedBlockLog = new DeletedBlockLogImpl(conf);
|
|
|
+ containerManager = Mockito.mock(ContainerMapping.class);
|
|
|
+ deletedBlockLog = new DeletedBlockLogImpl(conf, containerManager);
|
|
|
+ dnList = new ArrayList<>(3);
|
|
|
+ setupContainerManager();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setupContainerManager() throws IOException {
|
|
|
+ dnList.add(
|
|
|
+ DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
|
|
|
+ .build());
|
|
|
+ dnList.add(
|
|
|
+ DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
|
|
|
+ .build());
|
|
|
+ dnList.add(
|
|
|
+ DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
|
|
|
+ .build());
|
|
|
+
|
|
|
+ ContainerInfo containerInfo =
|
|
|
+ new ContainerInfo.Builder().setContainerID(1).build();
|
|
|
+ Pipeline pipeline =
|
|
|
+ new Pipeline(null, LifeCycleState.CLOSED, ReplicationType.RATIS,
|
|
|
+ ReplicationFactor.THREE, null);
|
|
|
+ pipeline.addMember(dnList.get(0));
|
|
|
+ pipeline.addMember(dnList.get(1));
|
|
|
+ pipeline.addMember(dnList.get(2));
|
|
|
+ ContainerWithPipeline containerWithPipeline =
|
|
|
+ new ContainerWithPipeline(containerInfo, pipeline);
|
|
|
+ when(containerManager.getContainerWithPipeline(anyLong()))
|
|
|
+ .thenReturn(containerWithPipeline);
|
|
|
+ when(containerManager.getContainer(anyLong())).thenReturn(containerInfo);
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -101,45 +137,50 @@ public class TestDeletedBlockLog {
|
|
|
return blockMap;
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testGetTransactions() throws Exception {
|
|
|
- List<DeletedBlocksTransaction> blocks =
|
|
|
- deletedBlockLog.getTransactions(30);
|
|
|
- Assert.assertEquals(0, blocks.size());
|
|
|
-
|
|
|
- // Creates 40 TX in the log.
|
|
|
- for (Map.Entry<Long, List<Long>> entry : generateData(40).entrySet()){
|
|
|
- deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
|
|
|
+ private void commitTransactions(
|
|
|
+ List<DeleteBlockTransactionResult> transactionResults,
|
|
|
+ DatanodeDetails... dns) {
|
|
|
+ for (DatanodeDetails dnDetails : dns) {
|
|
|
+ deletedBlockLog
|
|
|
+ .commitTransactions(transactionResults, dnDetails.getUuid());
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- // Get first 30 TXs.
|
|
|
- blocks = deletedBlockLog.getTransactions(30);
|
|
|
- Assert.assertEquals(30, blocks.size());
|
|
|
- for (int i = 0; i < 30; i++) {
|
|
|
- Assert.assertEquals(i + 1, blocks.get(i).getTxID());
|
|
|
- }
|
|
|
+ private void commitTransactions(
|
|
|
+ List<DeleteBlockTransactionResult> transactionResults) {
|
|
|
+ commitTransactions(transactionResults,
|
|
|
+ dnList.toArray(new DatanodeDetails[3]));
|
|
|
+ }
|
|
|
|
|
|
- // Get another 30 TXs.
|
|
|
- // The log only 10 left, so this time it will only return 10 TXs.
|
|
|
- blocks = deletedBlockLog.getTransactions(30);
|
|
|
- Assert.assertEquals(10, blocks.size());
|
|
|
- for (int i = 30; i < 40; i++) {
|
|
|
- Assert.assertEquals(i + 1, blocks.get(i - 30).getTxID());
|
|
|
- }
|
|
|
+ private void commitTransactions(
|
|
|
+ Collection<DeletedBlocksTransaction> deletedBlocksTransactions,
|
|
|
+ DatanodeDetails... dns) {
|
|
|
+ commitTransactions(deletedBlocksTransactions.stream()
|
|
|
+ .map(this::createDeleteBlockTransactionResult)
|
|
|
+ .collect(Collectors.toList()), dns);
|
|
|
+ }
|
|
|
|
|
|
- // Get another 50 TXs.
|
|
|
- // By now the position should have moved to the beginning,
|
|
|
- // this call will return all 40 TXs.
|
|
|
- blocks = deletedBlockLog.getTransactions(50);
|
|
|
- Assert.assertEquals(40, blocks.size());
|
|
|
- for (int i = 0; i < 40; i++) {
|
|
|
- Assert.assertEquals(i + 1, blocks.get(i).getTxID());
|
|
|
- }
|
|
|
- List<Long> txIDs = new ArrayList<>();
|
|
|
- for (DeletedBlocksTransaction block : blocks) {
|
|
|
- txIDs.add(block.getTxID());
|
|
|
- }
|
|
|
- deletedBlockLog.commitTransactions(txIDs);
|
|
|
+ private void commitTransactions(
|
|
|
+ Collection<DeletedBlocksTransaction> deletedBlocksTransactions) {
|
|
|
+ commitTransactions(deletedBlocksTransactions.stream()
|
|
|
+ .map(this::createDeleteBlockTransactionResult)
|
|
|
+ .collect(Collectors.toList()));
|
|
|
+ }
|
|
|
+
|
|
|
+ private DeleteBlockTransactionResult createDeleteBlockTransactionResult(
|
|
|
+ DeletedBlocksTransaction transaction) {
|
|
|
+ return DeleteBlockTransactionResult.newBuilder()
|
|
|
+ .setContainerID(transaction.getContainerID()).setSuccess(true)
|
|
|
+ .setTxID(transaction.getTxID()).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ private List<DeletedBlocksTransaction> getTransactions(
|
|
|
+ int maximumAllowedTXNum) throws IOException {
|
|
|
+ DatanodeDeletedBlockTransactions transactions =
|
|
|
+ new DatanodeDeletedBlockTransactions(containerManager,
|
|
|
+ maximumAllowedTXNum, 3);
|
|
|
+ deletedBlockLog.getTransactions(transactions);
|
|
|
+ return transactions.getDatanodeTransactions(dnList.get(0).getUuid());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -153,7 +194,7 @@ public class TestDeletedBlockLog {
|
|
|
|
|
|
// This will return all TXs, total num 30.
|
|
|
List<DeletedBlocksTransaction> blocks =
|
|
|
- deletedBlockLog.getTransactions(40);
|
|
|
+ getTransactions(40);
|
|
|
List<Long> txIDs = blocks.stream().map(DeletedBlocksTransaction::getTxID)
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
@@ -164,13 +205,13 @@ public class TestDeletedBlockLog {
|
|
|
// Increment another time so it exceed the maxRetry.
|
|
|
// On this call, count will be set to -1 which means TX eventually fails.
|
|
|
deletedBlockLog.incrementCount(txIDs);
|
|
|
- blocks = deletedBlockLog.getTransactions(40);
|
|
|
+ blocks = getTransactions(40);
|
|
|
for (DeletedBlocksTransaction block : blocks) {
|
|
|
Assert.assertEquals(-1, block.getCount());
|
|
|
}
|
|
|
|
|
|
// If all TXs are failed, getTransactions call will always return nothing.
|
|
|
- blocks = deletedBlockLog.getTransactions(40);
|
|
|
+ blocks = getTransactions(40);
|
|
|
Assert.assertEquals(blocks.size(), 0);
|
|
|
}
|
|
|
|
|
@@ -180,16 +221,26 @@ public class TestDeletedBlockLog {
|
|
|
deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
|
|
|
}
|
|
|
List<DeletedBlocksTransaction> blocks =
|
|
|
- deletedBlockLog.getTransactions(20);
|
|
|
- List<Long> txIDs = new ArrayList<>();
|
|
|
- for (DeletedBlocksTransaction block : blocks) {
|
|
|
- txIDs.add(block.getTxID());
|
|
|
- }
|
|
|
- // Add an invalid txID.
|
|
|
- txIDs.add(70L);
|
|
|
- deletedBlockLog.commitTransactions(txIDs);
|
|
|
- blocks = deletedBlockLog.getTransactions(50);
|
|
|
+ getTransactions(20);
|
|
|
+ // Add an invalid txn.
|
|
|
+ blocks.add(
|
|
|
+ DeletedBlocksTransaction.newBuilder().setContainerID(1).setTxID(70)
|
|
|
+ .setCount(0).addLocalID(0).build());
|
|
|
+ commitTransactions(blocks);
|
|
|
+ blocks.remove(blocks.size() - 1);
|
|
|
+
|
|
|
+ blocks = getTransactions(50);
|
|
|
+ Assert.assertEquals(30, blocks.size());
|
|
|
+ commitTransactions(blocks, dnList.get(1), dnList.get(2),
|
|
|
+ DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
|
|
|
+ .build());
|
|
|
+
|
|
|
+ blocks = getTransactions(50);
|
|
|
Assert.assertEquals(30, blocks.size());
|
|
|
+ commitTransactions(blocks, dnList.get(0));
|
|
|
+
|
|
|
+ blocks = getTransactions(50);
|
|
|
+ Assert.assertEquals(0, blocks.size());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -213,20 +264,16 @@ public class TestDeletedBlockLog {
|
|
|
}
|
|
|
added += 10;
|
|
|
} else if (state == 1) {
|
|
|
- blocks = deletedBlockLog.getTransactions(20);
|
|
|
+ blocks = getTransactions(20);
|
|
|
txIDs = new ArrayList<>();
|
|
|
for (DeletedBlocksTransaction block : blocks) {
|
|
|
txIDs.add(block.getTxID());
|
|
|
}
|
|
|
deletedBlockLog.incrementCount(txIDs);
|
|
|
} else if (state == 2) {
|
|
|
- txIDs = new ArrayList<>();
|
|
|
- for (DeletedBlocksTransaction block : blocks) {
|
|
|
- txIDs.add(block.getTxID());
|
|
|
- }
|
|
|
+ commitTransactions(blocks);
|
|
|
+ committed += blocks.size();
|
|
|
blocks = new ArrayList<>();
|
|
|
- committed += txIDs.size();
|
|
|
- deletedBlockLog.commitTransactions(txIDs);
|
|
|
} else {
|
|
|
// verify the number of added and committed.
|
|
|
List<Map.Entry<byte[], byte[]>> result =
|
|
@@ -234,6 +281,8 @@ public class TestDeletedBlockLog {
|
|
|
Assert.assertEquals(added, result.size() + committed);
|
|
|
}
|
|
|
}
|
|
|
+ blocks = getTransactions(1000);
|
|
|
+ commitTransactions(blocks);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -244,16 +293,13 @@ public class TestDeletedBlockLog {
|
|
|
// close db and reopen it again to make sure
|
|
|
// transactions are stored persistently.
|
|
|
deletedBlockLog.close();
|
|
|
- deletedBlockLog = new DeletedBlockLogImpl(conf);
|
|
|
+ deletedBlockLog = new DeletedBlockLogImpl(conf, containerManager);
|
|
|
List<DeletedBlocksTransaction> blocks =
|
|
|
- deletedBlockLog.getTransactions(10);
|
|
|
- List<Long> txIDs = new ArrayList<>();
|
|
|
- for (DeletedBlocksTransaction block : blocks) {
|
|
|
- txIDs.add(block.getTxID());
|
|
|
- }
|
|
|
- deletedBlockLog.commitTransactions(txIDs);
|
|
|
- blocks = deletedBlockLog.getTransactions(10);
|
|
|
- Assert.assertEquals(10, blocks.size());
|
|
|
+ getTransactions(10);
|
|
|
+ commitTransactions(blocks);
|
|
|
+ blocks = getTransactions(100);
|
|
|
+ Assert.assertEquals(40, blocks.size());
|
|
|
+ commitTransactions(blocks);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -262,32 +308,11 @@ public class TestDeletedBlockLog {
|
|
|
int maximumAllowedTXNum = 5;
|
|
|
List<DeletedBlocksTransaction> blocks = null;
|
|
|
List<Long> containerIDs = new LinkedList<>();
|
|
|
+ DatanodeDetails dnId1 = dnList.get(0), dnId2 = dnList.get(1);
|
|
|
|
|
|
int count = 0;
|
|
|
long containerID = 0L;
|
|
|
- DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
|
|
|
- DatanodeDetails.Port.Name.STANDALONE, 0);
|
|
|
- DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
|
|
|
- DatanodeDetails.Port.Name.RATIS, 0);
|
|
|
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
|
|
|
- DatanodeDetails.Port.Name.REST, 0);
|
|
|
- DatanodeDetails dnId1 = DatanodeDetails.newBuilder()
|
|
|
- .setUuid(UUID.randomUUID().toString())
|
|
|
- .setIpAddress("127.0.0.1")
|
|
|
- .setHostName("localhost")
|
|
|
- .addPort(containerPort)
|
|
|
- .addPort(ratisPort)
|
|
|
- .addPort(restPort)
|
|
|
- .build();
|
|
|
- DatanodeDetails dnId2 = DatanodeDetails.newBuilder()
|
|
|
- .setUuid(UUID.randomUUID().toString())
|
|
|
- .setIpAddress("127.0.0.1")
|
|
|
- .setHostName("localhost")
|
|
|
- .addPort(containerPort)
|
|
|
- .addPort(ratisPort)
|
|
|
- .addPort(restPort)
|
|
|
- .build();
|
|
|
- Mapping mappingService = mock(ContainerMapping.class);
|
|
|
+
|
|
|
// Creates {TXNum} TX in the log.
|
|
|
for (Map.Entry<Long, List<Long>> entry : generateData(txNum)
|
|
|
.entrySet()) {
|
|
@@ -298,29 +323,25 @@ public class TestDeletedBlockLog {
|
|
|
|
|
|
// make TX[1-6] for datanode1; TX[7-10] for datanode2
|
|
|
if (count <= (maximumAllowedTXNum + 1)) {
|
|
|
- mockContainerInfo(mappingService, containerID, dnId1);
|
|
|
+ mockContainerInfo(containerID, dnId1);
|
|
|
} else {
|
|
|
- mockContainerInfo(mappingService, containerID, dnId2);
|
|
|
+ mockContainerInfo(containerID, dnId2);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
DatanodeDeletedBlockTransactions transactions =
|
|
|
- new DatanodeDeletedBlockTransactions(mappingService,
|
|
|
+ new DatanodeDeletedBlockTransactions(containerManager,
|
|
|
maximumAllowedTXNum, 2);
|
|
|
deletedBlockLog.getTransactions(transactions);
|
|
|
|
|
|
- List<Long> txIDs = new LinkedList<>();
|
|
|
for (UUID id : transactions.getDatanodeIDs()) {
|
|
|
List<DeletedBlocksTransaction> txs = transactions
|
|
|
.getDatanodeTransactions(id);
|
|
|
- for (DeletedBlocksTransaction tx : txs) {
|
|
|
- txIDs.add(tx.getTxID());
|
|
|
- }
|
|
|
+ // delete TX ID
|
|
|
+ commitTransactions(txs);
|
|
|
}
|
|
|
|
|
|
- // delete TX ID
|
|
|
- deletedBlockLog.commitTransactions(txIDs);
|
|
|
- blocks = deletedBlockLog.getTransactions(txNum);
|
|
|
+ blocks = getTransactions(txNum);
|
|
|
// There should be one block remained since dnID1 reaches
|
|
|
// the maximum value (5).
|
|
|
Assert.assertEquals(1, blocks.size());
|
|
@@ -337,7 +358,8 @@ public class TestDeletedBlockLog {
|
|
|
builder.setTxID(11);
|
|
|
builder.setContainerID(containerID);
|
|
|
builder.setCount(0);
|
|
|
- transactions.addTransaction(builder.build());
|
|
|
+ transactions.addTransaction(builder.build(),
|
|
|
+ null);
|
|
|
|
|
|
// The number of TX in dnID2 should not be changed.
|
|
|
Assert.assertEquals(size,
|
|
@@ -349,14 +371,14 @@ public class TestDeletedBlockLog {
|
|
|
builder.setTxID(12);
|
|
|
builder.setContainerID(containerID);
|
|
|
builder.setCount(0);
|
|
|
- mockContainerInfo(mappingService, containerID, dnId2);
|
|
|
- transactions.addTransaction(builder.build());
|
|
|
+ mockContainerInfo(containerID, dnId2);
|
|
|
+ transactions.addTransaction(builder.build(),
|
|
|
+ null);
|
|
|
// Since all node are full, then transactions is full.
|
|
|
Assert.assertTrue(transactions.isFull());
|
|
|
}
|
|
|
|
|
|
- private void mockContainerInfo(Mapping mappingService, long containerID,
|
|
|
- DatanodeDetails dd) throws IOException {
|
|
|
+ private void mockContainerInfo(long containerID, DatanodeDetails dd) throws IOException {
|
|
|
Pipeline pipeline =
|
|
|
new Pipeline("fake", LifeCycleState.OPEN,
|
|
|
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "fake");
|
|
@@ -370,9 +392,9 @@ public class TestDeletedBlockLog {
|
|
|
ContainerInfo containerInfo = builder.build();
|
|
|
ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
|
|
|
containerInfo, pipeline);
|
|
|
- Mockito.doReturn(containerInfo).when(mappingService)
|
|
|
+ Mockito.doReturn(containerInfo).when(containerManager)
|
|
|
.getContainer(containerID);
|
|
|
- Mockito.doReturn(containerWithPipeline).when(mappingService)
|
|
|
+ Mockito.doReturn(containerWithPipeline).when(containerManager)
|
|
|
.getContainerWithPipeline(containerID);
|
|
|
}
|
|
|
}
|