hemantk-12 commented on code in PR #8016: URL: https://github.com/apache/ozone/pull/8016#discussion_r2053422703
########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionDag.java: ########## @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ozone.rocksdiff; + +import com.google.common.graph.GraphBuilder; +import com.google.common.graph.MutableGraph; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.ozone.compaction.log.CompactionFileInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Helper methods for creating compaction DAGs. Review Comment: nit: please update this comment. It is not a helper anymore. ########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/CompactionDag.java: ########## @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ozone.rocksdiff; + +import com.google.common.graph.GraphBuilder; +import com.google.common.graph.MutableGraph; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.ozone.compaction.log.CompactionFileInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Helper methods for creating compaction DAGs. + */ +public class CompactionDag { + private static final Logger LOG = LoggerFactory.getLogger(CompactionDag.class); + + private final ConcurrentMap<String, CompactionNode> compactionNodeMap = new ConcurrentHashMap<>(); + private final MutableGraph<CompactionNode> forwardCompactionDAG = GraphBuilder.directed().build(); + private final MutableGraph<CompactionNode> backwardCompactionDAG = GraphBuilder.directed().build(); + + private CompactionNode addNodeToDAG(String file, long seqNum, String startKey, String endKey, String columnFamily) { + CompactionNode fileNode = new CompactionNode(file, seqNum, startKey, endKey, columnFamily); + backwardCompactionDAG.addNode(fileNode); + forwardCompactionDAG.addNode(fileNode); + return fileNode; + } + + /** + * Populate the compaction DAG with input and output SST files lists. + * + * @param inputFiles List of compaction input files. + * @param outputFiles List of compaction output files. + * @param seqNum DB transaction sequence number. + */ + public void populateCompactionDAG(List<CompactionFileInfo> inputFiles, + List<CompactionFileInfo> outputFiles, + long seqNum) { + + if (LOG.isDebugEnabled()) { + LOG.debug("Input files: {} -> Output files: {}", inputFiles, outputFiles); + } + + for (CompactionFileInfo outfile : outputFiles) { + final CompactionNode outfileNode = compactionNodeMap.computeIfAbsent(outfile.getFileName(), + file -> addNodeToDAG(file, seqNum, outfile.getStartKey(), outfile.getEndKey(), outfile.getColumnFamily())); + + for (CompactionFileInfo infile : inputFiles) { + final CompactionNode infileNode = compactionNodeMap.computeIfAbsent(infile.getFileName(), + file -> addNodeToDAG(file, seqNum, infile.getStartKey(), infile.getEndKey(), infile.getColumnFamily())); + + // Draw the edges + if (!Objects.equals(outfileNode.getFileName(), infileNode.getFileName())) { + forwardCompactionDAG.putEdge(outfileNode, infileNode); + backwardCompactionDAG.putEdge(infileNode, outfileNode); + } + } + } + } + + public Set<String> pruneNodesFromDag(Set<CompactionNode> nodesToRemove) { + pruneBackwardDag(backwardCompactionDAG, nodesToRemove); + Set<String> sstFilesPruned = pruneForwardDag(forwardCompactionDAG, nodesToRemove); + // Remove SST file nodes from compactionNodeMap too, + // since those nodes won't be needed after clean up. + nodesToRemove.forEach(compactionNodeMap::remove); + return sstFilesPruned; + } + + /** + * Prunes backward DAG's upstream from the level, that needs to be removed. + */ + Set<String> pruneBackwardDag(MutableGraph<CompactionNode> backwardDag, + Set<CompactionNode> startNodes) { + Set<String> removedFiles = new HashSet<>(); + Set<CompactionNode> currentLevel = startNodes; + + synchronized (this) { Review Comment: `CompactionDag` is not completely thread safe. I think `synchronized` should be on `RocksDBCheckpointDiffer` and should be responsible to update `CompactionDag` in a thread-safe manner. ########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java: ########## @@ -1037,7 +964,7 @@ public Comparator<CompactionNode> reversed() { @VisibleForTesting void dumpCompactionNodeTable() { - List<CompactionNode> nodeList = compactionNodeMap.values().stream() + List<CompactionNode> nodeList = compactionDag.getCompactionMap().values().stream() Review Comment: Similar to the previous one. ########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java: ########## @@ -1386,14 +1205,9 @@ public boolean shouldRun() { return !suspended.get(); } - @VisibleForTesting - public boolean debugEnabled(Integer level) { - return DEBUG_LEVEL.contains(level); - } - @VisibleForTesting public ConcurrentHashMap<String, CompactionNode> getCompactionNodeMap() { - return compactionNodeMap; + return (ConcurrentHashMap<String, CompactionNode>) compactionDag.getCompactionMap(); Review Comment: It is Ok to return like this in this patch, but we should do a clean-up and use `CompactionDag` as encapsulation. ########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java: ########## @@ -938,7 +866,7 @@ synchronized void internalGetSSTDiffList( continue; } - CompactionNode infileNode = compactionNodeMap.get(fileName); + CompactionNode infileNode = compactionDag.getCompactionMap().get(fileName); Review Comment: nit: maybe add a function `getCompactionNode` in `CompactionDag`. ########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java: ########## @@ -758,22 +681,27 @@ public void loadAllCompactionLogs() { synchronized (this) { preconditionChecksForLoadAllCompactionLogs(); addEntriesFromLogFilesToDagAndCompactionLogTable(); Review Comment: Can you please comment here that it is only for backward compatibility? -- 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: issues-unsubscr...@ozone.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For additional commands, e-mail: issues-h...@ozone.apache.org