hemantk-12 commented on code in PR #8016: URL: https://github.com/apache/ozone/pull/8016#discussion_r2025755150
########## hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/compaction/log/CompactionDagHelper.java: ########## @@ -0,0 +1,254 @@ +/* + * 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.compaction.log; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.ozone.compaction.log.RocksDBConsts.COMPACTION_LOG_COMMENT_LINE_PREFIX; +import static org.apache.ozone.compaction.log.RocksDBConsts.COMPACTION_LOG_ENTRY_FILE_DELIMITER; +import static org.apache.ozone.compaction.log.RocksDBConsts.COMPACTION_LOG_ENTRY_INPUT_OUTPUT_FILES_DELIMITER; +import static org.apache.ozone.compaction.log.RocksDBConsts.COMPACTION_LOG_ENTRY_LINE_PREFIX; +import static org.apache.ozone.compaction.log.RocksDBConsts.COMPACTION_LOG_FILE_NAME_SUFFIX; +import static org.apache.ozone.compaction.log.RocksDBConsts.COMPACTION_LOG_SEQ_NUM_LINE_PREFIX; +import static org.apache.ozone.compaction.log.RocksDBConsts.LONG_MAX_STR_LEN; +import static org.apache.ozone.compaction.log.RocksDBConsts.SPACE_DELIMITER; +import static org.apache.ozone.compaction.log.RocksDBConsts.SST_FILE_EXTENSION; + +import com.google.common.base.Preconditions; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.hdds.utils.db.managed.ManagedOptions; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; +import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReader; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; +import org.rocksdb.TableProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility methods to populate compactionLogTable from a compaction-log file. + */ +public final class CompactionDagHelper { Review Comment: But now, we have duplicated code. ########## hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java: ########## @@ -273,6 +273,7 @@ public static boolean isReadOnly( case TransferLeadership: case SetSafeMode: case PrintCompactionLogDag: + // deprecated by HDDS-12053, keeping it here for compatibility Review Comment: Can you please move it above the `PrintCompactionLogDag`? Or on the side of `PrintCompactionLogDag`. Currently, it feels like `GetSnapshotInfo` API is deprecated. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/om/CompactionLogDagPrinter.java: ########## @@ -0,0 +1,191 @@ +/* + * 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.hadoop.ozone.debug.om; + +import static org.apache.hadoop.ozone.OzoneConsts.COMPACTION_LOG_TABLE; + +import com.google.common.graph.GraphBuilder; +import com.google.common.graph.MutableGraph; +import com.google.protobuf.InvalidProtocolBufferException; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.hadoop.hdds.cli.AbstractSubcommand; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; +import org.apache.hadoop.ozone.debug.RocksDBUtils; +import org.apache.ozone.compaction.log.CompactionFileInfo; +import org.apache.ozone.compaction.log.CompactionLogEntry; +import org.apache.ozone.graph.PrintableGraph; +import org.apache.ozone.rocksdiff.CompactionNode; +import org.apache.ozone.rocksdiff.RocksDiffUtils; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; +import picocli.CommandLine; + +/** + * Handler to generate image for current compaction DAG. + * ozone debug om print-compaction-dag. + */ +@CommandLine.Command( + name = "print-compaction-dag", + aliases = "pcd", + description = "Create an image of the current compaction log DAG. " + + "This command is an offline command. i.e., it can run on any instance of om.db " + + "and does not require OM to be up.") +public class CompactionLogDagPrinter extends AbstractSubcommand implements Callable<Void> { + + @CommandLine.Option(names = {"-o", "--output-file"}, + required = true, + description = "Path to location at which image will be downloaded. " + + "Should include the image file name with \".png\" extension.") + private String imageLocation; + + @CommandLine.Option(names = {"--db"}, + required = true, + scope = CommandLine.ScopeType.INHERIT, + description = "Path to OM RocksDB") + private String dbPath; + + // TODO: Change graphType to enum. + @CommandLine.Option(names = {"-t", "--graph-type"}, + description = "Type of node name to use in the graph image. (optional)\n Accepted values are: \n" + + " FILE_NAME (default) : to use file name as node name in DAG,\n" + + " KEY_SIZE: to show the no. of keys in the file along with file name in the DAG node name,\n" + + " CUMULATIVE_SIZE: to show the cumulative size along with file name in the DAG node name.", + defaultValue = "FILE_NAME") + private String graphType; + + @Override + public Void call() throws Exception { + try { + CreateCompactionDag createCompactionDag = new CreateCompactionDag(dbPath); + createCompactionDag.pngPrintMutableGraph(imageLocation, PrintableGraph.GraphType.valueOf(graphType)); + out().println("Graph was generated at '" + imageLocation + "'."); + } catch (RocksDBException ex) { + err().println("Failed to open RocksDB: " + ex); + throw new IOException(ex); + } + return null; + } + + class CreateCompactionDag { + // Hash table to track CompactionNode for a given SST File. + private final ConcurrentHashMap<String, CompactionNode> compactionNodeMap = + new ConcurrentHashMap<>(); + private final MutableGraph<CompactionNode> backwardCompactionDAG = + GraphBuilder.directed().build(); + + private ColumnFamilyHandle compactionLogTableCFHandle; + private ManagedRocksDB activeRocksDB; + + CreateCompactionDag(String dbPath) throws RocksDBException { + final List<ColumnFamilyHandle> cfHandleList = new ArrayList<>(); + List<ColumnFamilyDescriptor> cfDescList = RocksDBUtils.getColumnFamilyDescriptors(dbPath); + activeRocksDB = ManagedRocksDB.openReadOnly(dbPath, cfDescList, cfHandleList); + compactionLogTableCFHandle = RocksDBUtils.getColumnFamilyHandle(COMPACTION_LOG_TABLE, cfHandleList); + } + + public void pngPrintMutableGraph(String filePath, PrintableGraph.GraphType gType) + throws IOException, RocksDBException { + Objects.requireNonNull(filePath, "Image file path is required."); + Objects.requireNonNull(gType, "Graph type is required."); + + loadAllCompactionLogs(); + + PrintableGraph graph; + graph = new PrintableGraph(backwardCompactionDAG, gType); + graph.generateImage(filePath); + } + + public void loadAllCompactionLogs() { + try (ManagedRocksIterator managedRocksIterator = new ManagedRocksIterator( + activeRocksDB.get().newIterator(compactionLogTableCFHandle))) { + managedRocksIterator.get().seekToFirst(); + while (managedRocksIterator.get().isValid()) { + byte[] value = managedRocksIterator.get().value(); + CompactionLogEntry compactionLogEntry = + CompactionLogEntry.getFromProtobuf( + HddsProtos.CompactionLogEntryProto.parseFrom(value)); + populateCompactionDAG(compactionLogEntry.getInputFileInfoList(), + compactionLogEntry.getOutputFileInfoList(), + compactionLogEntry.getDbSequenceNumber()); + managedRocksIterator.get().next(); + } + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } + + /** + * 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. + */ + private void populateCompactionDAG(List<CompactionFileInfo> inputFiles, + List<CompactionFileInfo> outputFiles, + long seqNum) { + + 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 (!outfileNode.getFileName().equals(infileNode.getFileName())) { + backwardCompactionDAG.putEdge(infileNode, outfileNode); + } + } + } + } + + private CompactionNode addNodeToDAG(String file, long seqNum, String startKey, + String endKey, String columnFamily) { + long numKeys = 0L; + try { + numKeys = RocksDiffUtils.getSSTFileSummary(dbPath + "/" + file); + } catch (RocksDBException e) { + err().println("Warning: Can't get num of keys in SST '" + file + "'. Reason: " + e.getMessage()); + } catch (FileNotFoundException e) { + out().println("Warning: Can't find SST : " + file); Review Comment: This message will come most of the time because the file may not exist in dbPath because it is either in a snapshotted DB or a compaction-back-up dir. You have to check it in all the snapshot dirs and back-up dir. I'll suggest not having `numKeys` for this tool. ########## hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/om/CompactionLogDagPrinter.java: ########## @@ -0,0 +1,202 @@ +/* + * 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.hadoop.ozone.debug.om; + +import static org.apache.hadoop.ozone.OzoneConsts.COMPACTION_LOG_TABLE; + +import com.google.common.graph.GraphBuilder; +import com.google.common.graph.MutableGraph; +import com.google.protobuf.InvalidProtocolBufferException; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.hadoop.hdds.cli.AbstractSubcommand; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; +import org.apache.hadoop.ozone.debug.RocksDBUtils; +import org.apache.ozone.compaction.log.CompactionDagHelper; +import org.apache.ozone.compaction.log.CompactionFileInfo; +import org.apache.ozone.compaction.log.CompactionLogEntry; +import org.apache.ozone.graph.PrintableGraph; +import org.apache.ozone.rocksdiff.CompactionNode; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; +import picocli.CommandLine; + +/** + * Handler to generate image for current compaction DAG in the OM leader node. + * ozone debug om print-compaction-dag. + */ +@CommandLine.Command( + name = "print-compaction-dag", + aliases = "pcd", + description = "Create an image of the current compaction log DAG. " + + "This command is an offline command. i.e., it can run on any instance of om.db " + + "and does not require OM to be up.") +public class CompactionLogDagPrinter extends AbstractSubcommand implements Callable<Void> { + + @CommandLine.Option(names = {"-o", "--output-file"}, + required = true, + description = "Path to location at which image will be downloaded. " + + "Should include the image file name with \".png\" extension.") + private String imageLocation; + + @CommandLine.Option(names = {"--db"}, + required = true, + scope = CommandLine.ScopeType.INHERIT, + description = "Path to OM RocksDB") + private String dbPath; + + @CommandLine.Option(names = {"--compaction-log"}, + scope = CommandLine.ScopeType.INHERIT, + description = "Path to compaction-log directory.") + private String compactionLogDir; + + // TODO: Change graphType to enum. + @CommandLine.Option(names = {"-t", "--graph-type"}, + description = "Type of node name to use in the graph image. (optional)\n Accepted values are: \n" + + " FILE_NAME (default) : to use file name as node name in DAG,\n" + + " KEY_SIZE: to show the no. of keys in the file along with file name in the DAG node name,\n" + + " CUMULATIVE_SIZE: to show the cumulative size along with file name in the DAG node name.", + defaultValue = "FILE_NAME") + private String graphType; + + @Override + public Void call() throws Exception { + try { + CreateCompactionDag createCompactionDag = new CreateCompactionDag(dbPath, compactionLogDir); + createCompactionDag.pngPrintMutableGraph(imageLocation, PrintableGraph.GraphType.valueOf(graphType)); + out().println("Graph was generated at '" + imageLocation + "'."); + } catch (RocksDBException ex) { + err().println("Failed to open RocksDB: " + ex); + throw new IOException(ex); + } + return null; + } + + class CreateCompactionDag { + // Hash table to track CompactionNode for a given SST File. + private final ConcurrentHashMap<String, CompactionNode> compactionNodeMap = + new ConcurrentHashMap<>(); + private final MutableGraph<CompactionNode> backwardCompactionDAG = + GraphBuilder.directed().build(); + + private ColumnFamilyHandle compactionLogTableCFHandle; + private ManagedRocksDB activeRocksDB; + private CompactionDagHelper compactionDagHelper; + + CreateCompactionDag(String dbPath, String compactDir) throws RocksDBException { + final List<ColumnFamilyHandle> cfHandleList = new ArrayList<>(); + List<ColumnFamilyDescriptor> cfDescList = RocksDBUtils.getColumnFamilyDescriptors(dbPath); + activeRocksDB = ManagedRocksDB.openReadOnly(dbPath, cfDescList, cfHandleList); + compactionLogTableCFHandle = RocksDBUtils.getColumnFamilyHandle(COMPACTION_LOG_TABLE, cfHandleList); + compactionDagHelper = new CompactionDagHelper(compactDir, activeRocksDB, compactionLogTableCFHandle); + } + + public void pngPrintMutableGraph(String filePath, PrintableGraph.GraphType gType) + throws IOException, RocksDBException { + Objects.requireNonNull(filePath, "Image file path is required."); + Objects.requireNonNull(gType, "Graph type is required."); + + loadAllCompactionLogs(); + + PrintableGraph graph; Review Comment: Since it is an ops tool, IOM, it is OK if you just throw an exception and remove the older implementation in [OzoneManager](https://github.com/apache/ozone/blob/42133076373dd3c1b72f0f8028771166a0594b50/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java#L5050) and [OzoneManagerProtocolClientSideTranslatorPB](https://github.com/apache/ozone/blob/2b48e8c6ec1739d541d5c02183ad1a91d9f7a308/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java#L1333). Change the exception message in [OzoneManagerProtocol](https://github.com/apache/ozone/blob/2b48e8c6ec1739d541d5c02183ad1a91d9f7a308/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java#L782). -- 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