sumitagrawl commented on code in PR #9272:
URL: https://github.com/apache/ozone/pull/9272#discussion_r2522735764
##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -514,6 +514,14 @@ message DeletedBlocksTransactionInfo {
optional int32 count = 4;
}
+message DeletedBlocksTransactionSummary {
+ optional int64 firstTxID = 1; // starting ID of transaction to be counted
into summary
Review Comment:
Is this useful after the upgrade and old transaction is removed? If not then
may not be requried, count itself can tell last n transaction used for
summation.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogStateManagerImpl.java:
##########
@@ -139,8 +144,26 @@ public void removeFromDB() {
}
@Override
- public void addTransactionsToDB(ArrayList<DeletedBlocksTransaction> txs)
- throws IOException {
+ public void addTransactionsToDB(ArrayList<DeletedBlocksTransaction> txs,
+ DeletedBlocksTransactionSummary summary) throws IOException {
+ Map<ContainerID, Long> containerIdToTxnIdMap = new HashMap<>();
+ transactionBuffer.pauseAutoFlush();
Review Comment:
We do not need pause the flush, as this call is comming from
applyTransaction and till its finished, it will not allow takeSnapshot() even
triggered via auto-flush. So this is not a problem and can remove this logic.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -147,13 +156,23 @@ public void incrementCount(List<Long> txIDs)
}
private DeletedBlocksTransaction constructNewTransaction(
- long txID, long containerID, List<Long> blocks) {
- return DeletedBlocksTransaction.newBuilder()
+ long txID, long containerID, List<DeletedBlock> blocks) {
+ List<Long> localIdList = blocks.stream().map(b ->
b.getBlockID().getLocalID()).collect(Collectors.toList());
+ DeletedBlocksTransaction.Builder builder =
DeletedBlocksTransaction.newBuilder()
.setTxID(txID)
.setContainerID(containerID)
- .addAllLocalID(blocks)
- .setCount(0)
- .build();
+ .addAllLocalID(localIdList)
+ .setCount(0);
+
+ if
(VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.STORAGE_SPACE_DISTRIBUTION))
{
Review Comment:
What if actual value received from OM is updated in Transaction? there is no
impact as value is set with actual coming and not get updated in summary if not
finalized. May be this if condition and default value with "-1" is not required.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -366,6 +387,12 @@ public DatanodeDeletedBlockTransactions getTransactions(
keyValue = iter.next();
DeletedBlocksTransaction txn = keyValue.getValue();
final ContainerID id = ContainerID.valueOf(txn.getContainerID());
+ if
(VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.STORAGE_SPACE_DISTRIBUTION)
&&
Review Comment:
We do not need control here, it can be handled at time of summary flush if
required.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMDeletedBlockTransactionStatusManager.java:
##########
@@ -415,6 +450,76 @@ private boolean alreadyExecuted(DatanodeID dnId, long
txId) {
.contains(dnId);
}
+ @VisibleForTesting
+ public void addTransactions(ArrayList<DeletedBlocksTransaction> txList)
throws IOException {
+ if (txList.isEmpty()) {
+ return;
+ }
+ if
(VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.STORAGE_SPACE_DISTRIBUTION)
&&
+ !disableDataDistributionForTest) {
+ for (DeletedBlocksTransaction tx: txList) {
+ if (tx.hasTotalBlockSize()) {
+ if (!isFirstTxIdForDataDistributionSet) {
+ // set the first transaction ID for data distribution
+ isFirstTxIdForDataDistributionSet = true;
+ firstTxIdForDataDistribution = tx.getTxID();
+ }
+ incrDeletedBlocksSummary(tx);
+ }
+ }
+ deletedBlockLogStateManager.addTransactionsToDB(txList, getSummary());
+ return;
+ }
+ deletedBlockLogStateManager.addTransactionsToDB(txList);
+ }
+
+ private void incrDeletedBlocksSummary(DeletedBlocksTransaction tx) {
+ totalTxCount.addAndGet(1);
+ totalBlockCount.addAndGet(tx.getLocalIDCount());
+ totalBlocksSize.addAndGet(tx.getTotalBlockSize());
+ totalReplicatedBlocksSize.addAndGet(tx.getTotalBlockReplicatedSize());
+ }
+
+ @VisibleForTesting
+ public void removeTransactions(ArrayList<Long> txIDs) throws IOException {
+ if (txIDs.isEmpty()) {
+ return;
+ }
+ if
(VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.STORAGE_SPACE_DISTRIBUTION)
&&
+ !disableDataDistributionForTest) {
+ ArrayList<Long> txToQueryList = new ArrayList<>();
+ for (Long txID: txIDs) {
+ if (txID >= firstTxIdForDataDistribution) {
Review Comment:
do we need check this? if map do not have txId, it will be like a no-ops.
So this might be not required
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMDeletedBlockTransactionStatusManager.java:
##########
@@ -415,6 +450,76 @@ private boolean alreadyExecuted(DatanodeID dnId, long
txId) {
.contains(dnId);
}
+ @VisibleForTesting
+ public void addTransactions(ArrayList<DeletedBlocksTransaction> txList)
throws IOException {
+ if (txList.isEmpty()) {
+ return;
+ }
+ if
(VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.STORAGE_SPACE_DISTRIBUTION)
&&
+ !disableDataDistributionForTest) {
+ for (DeletedBlocksTransaction tx: txList) {
+ if (tx.hasTotalBlockSize()) {
+ if (!isFirstTxIdForDataDistributionSet) {
+ // set the first transaction ID for data distribution
+ isFirstTxIdForDataDistributionSet = true;
+ firstTxIdForDataDistribution = tx.getTxID();
+ }
+ incrDeletedBlocksSummary(tx);
+ }
+ }
+ deletedBlockLogStateManager.addTransactionsToDB(txList, getSummary());
+ return;
+ }
+ deletedBlockLogStateManager.addTransactionsToDB(txList);
+ }
+
+ private void incrDeletedBlocksSummary(DeletedBlocksTransaction tx) {
+ totalTxCount.addAndGet(1);
+ totalBlockCount.addAndGet(tx.getLocalIDCount());
+ totalBlocksSize.addAndGet(tx.getTotalBlockSize());
+ totalReplicatedBlocksSize.addAndGet(tx.getTotalBlockReplicatedSize());
+ }
+
+ @VisibleForTesting
+ public void removeTransactions(ArrayList<Long> txIDs) throws IOException {
Review Comment:
It can sync summary from leader itself instead of calculating at every SCM.
This can ensure all SCM have same data. For transactions not present at the
current SCM and notified by DN, I think we can ignore this.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]