You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "mladjan-gadzic (via GitHub)" <gi...@apache.org> on 2023/12/04 13:58:31 UTC

[PR] HDDS-9824. Provide CLI that scans containers for keys [ozone]

mladjan-gadzic opened a new pull request, #5724:
URL: https://github.com/apache/ozone/pull/5724

   ## What changes were proposed in this pull request?
   HDDS-9824. Provide CLI that scans containers for keys
   
   These changes are mostly quality of life improvements. There were use cases when it was necessary to check whether unhealthy container was empty. These changes help in that matter - this CLI outputs keys for input container ids (separated by comma). 
   
   How is this done? Everything was done on the client side, so there is no performance impact on the server side. RocksDB tables (keyTable, fileTable and dirTable) are read. KeyTable is pretty straightforward, it is read and container id is taken from OmKeyInfo. The situation is a little different for fileTable. In order to provide key name, fileTable and dirTable have to be read, and key name needs to be constructed, because FSO buckets does not follow the same ideology as non-FSO buckets in regards to key names.
   
   Usage example: ozone debug ldb --db=/data/metadata/om.db ckscanner -ids=1,2,3
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-9824
   
   ## How was this patch tested?
   Integration tests
   


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1448919297


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java:
##########
@@ -34,6 +28,8 @@
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447601284


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/OMAdmin.java:
##########
@@ -43,6 +40,9 @@
 
 import java.util.Collection;
 
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY;
+

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447584728


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(

Review Comment:
   I tried that, but couldn't find a way to squeeze functionality from [FSODirectoryPathResolver#getAbsolutePathForObjectIDs](https://github.com/apache/ozone/blob/15c77c6e9bb22a0bbf5d0080f81c9310051229a9/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/FSODirectoryPathResolver.java#L72) that is needed here. There is a significant difference in accesing the tables between methods (iterators used etc.).



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1422458315


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Parse a list of container IDs"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2.(Separated by ',')")

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "errose28 (via GitHub)" <gi...@apache.org>.
errose28 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1430723431


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;

Review Comment:
   I'm not sure this command belongs under the `ldb` subcommand. Right now that command has generic utils that work on any Ozone RocksDB instance, and this is specific to OM. Maybe it should be its own top level debug subcommand?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447593472


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(

Review Comment:
   Removed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1422836571


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =

Review Comment:
   Don't we need to add `default`?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),

Review Comment:
   Do we need to get `DBDefinition` and `DBColumnFamilyDefinition`? I think you can directly pass `DIRECTORY_TABLE.getBytes(UTF_8)` rather than getting it all from `DBDefinition` and then `DBColumnFamilyDefinition`.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(

Review Comment:
   Similar to comment, I don't think it is dbDefinition is needed. You can directly access CFH.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, FILE_TABLE);
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, KEY_TABLE);
+    }
+    return new ContainerKeyInfoWrapper(keysProcessed, containerKeyInfos);
+  }
+
+  private long processTable(DBDefinition dbDefinition,
+                            List<ColumnFamilyHandle> columnFamilyHandles,
+                            ManagedRocksDB db,
+                            List<ContainerKeyInfo> containerKeyInfos,
+                            String tableName)
+      throws IOException {
+    long keysProcessed = 0;
+    DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+        dbDefinition.getColumnFamily(tableName);
+    if (columnFamilyDefinition == null) {
+      throw new IllegalStateException(
+          "Table with name" + tableName + " not found");
+    }
+
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+        columnFamilyDefinition.getName().getBytes(UTF_8),
+        columnFamilyHandles);
+    if (columnFamilyHandle == null) {
+      throw new IllegalStateException("columnFamilyHandle is null");
+    }
+
+    try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+        db.get().newIterator(columnFamilyHandle))) {
+      iterator.get().seekToFirst();
+      while (iterator.get().isValid()) {
+        OmKeyInfo value = ((OmKeyInfo) columnFamilyDefinition.getValueCodec()
+            .fromPersistedFormat(iterator.get().value()));
+        List<OmKeyLocationInfoGroup> keyLocationVersions =
+            value.getKeyLocationVersions();
+        if (Objects.isNull(keyLocationVersions)) {
+          iterator.get().next();
+          keysProcessed++;
+          continue;
+        }
+
+        long volumeId = 0;
+        long bucketId = 0;
+        // volumeId and bucketId are only applicable to file table
+        if (tableName.equals(FILE_TABLE)) {
+          String key = new String(iterator.get().key(), UTF_8);
+          String[] keyParts = key.split(OM_KEY_PREFIX);
+          volumeId = Long.parseLong(keyParts[1]);
+          bucketId = Long.parseLong(keyParts[2]);
+        }
+
+        for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+          for (List<OmKeyLocationInfo> locationInfos :
+              locationInfoGroup.getLocationVersionMap().values()) {
+            for (OmKeyLocationInfo locationInfo : locationInfos) {
+              if (containerIds.contains(locationInfo.getContainerID())) {
+                // Generate asbolute key path for FSO keys
+                StringBuilder keyName = new StringBuilder();
+                if (tableName.equals(FILE_TABLE)) {
+                  // Load directory table only after the first fso key is found
+                  // to reduce necessary load if there are not fso keys
+                  if (!isDirTableLoaded) {
+                    long start = System.currentTimeMillis();
+                    directoryTable = getDirectoryTableData(parent.getDbPath());
+                    long end = System.currentTimeMillis();
+                    out().println(
+                        "directoryTable loaded in " + (end - start) + " ms.");
+                    isDirTableLoaded = true;
+                  }
+                  keyName.append(getFsoKeyPrefix(volumeId, bucketId, value));
+                }
+                keyName.append(value.getKeyName());
+                containerKeyInfos.add(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        value.getVolumeName(), volumeId, value.getBucketName(),
+                        bucketId, keyName.toString(),
+                        value.getParentObjectID()));
+              }
+            }
+          }
+        }

Review Comment:
   nit: Please move this to helper function. Too much nesting.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),

Review Comment:
   nit: Use [StringUtils](https://github.com/apache/ozone/blob/15c77c6e9bb22a0bbf5d0080f81c9310051229a9/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/StringUtils.java#L40) to/from byte conversion.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(

Review Comment:
   getAbsolutePathForObjectIDs is almost same as  [FSODirectoryPathResolver#getAbsolutePathForObjectIDs](https://github.com/apache/ozone/blob/15c77c6e9bb22a0bbf5d0080f81c9310051229a9/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/FSODirectoryPathResolver.java#L72). Can we just reuse existing function rather than creating new one?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()

Review Comment:
   Similar to previous comment, it can be `OmDirectoryInfo.getCodec().fromPersistedFormat(iterator.get().value())`.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "errose28 (via GitHub)" <gi...@apache.org>.
errose28 commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1936566190

   I'm +1 for adding a CLI even if Recon has a similar feature. We've also seen instances where there are issues with Recon so it would be good to have a way to check OM RocksDB directly.


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1862495605

   @hemantk-12  @errose28 , thank you both for the review and comments. Unfortunately, I'm currently dealing with a health crisis and won't be able to address these until the start of next week. I'll do my best to handle everything as soon as I'm able to.


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "errose28 (via GitHub)" <gi...@apache.org>.
errose28 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1430679517


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +
+      "  \"keysProcessed\": 3,\n" +
+      "  \"containerKeys\": {\n" +
+      "    \"1\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 1,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": -123,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": -456,\n" +
+      "        \"keyName\": \"dir1/key1\",\n" +
+      "        \"parentId\": -789\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"2\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 2,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key2\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"3\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 3,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key3\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ]\n" +
+      "  }\n" +
+      "}\n";
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new RDBParser())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForConatainerIds() throws IOException {

Review Comment:
   nit.
   ```suggestion
     void testWhenThereAreKeysForContainerIds() throws IOException {
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +

Review Comment:
   Can we store the expected output in a gson or jackson object instead? IIRC gson is what's being used for client side json in other places. Then the string written to stdout can be serialized into an object of the same type and they can be compared that way. This makes sure both are valid json, is more readable, and does not require exact white space matching.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {

Review Comment:
   Let's add more tests for all 3 bucket layout types, buckets with multiple keys at different levels of the directory tree, keys with multiple blocks and containers, and a mix of input where some containers have keys and some do not.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoWrapper.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import java.util.List;
+
+/**
+ * Class for aggregation of collected data.
+ */
+public class ContainerKeyInfoWrapper {

Review Comment:
   Can we make this an inner class of `ContainerKeyScanner` since it is only used in one place to wrap a method return? Also can we combine this class with `ContaienrKeyInfoResponse` by just doing the json processing on this object?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, FILE_TABLE);
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, KEY_TABLE);
+    }
+    return new ContainerKeyInfoWrapper(keysProcessed, containerKeyInfos);
+  }
+
+  private long processTable(DBDefinition dbDefinition,
+                            List<ColumnFamilyHandle> columnFamilyHandles,
+                            ManagedRocksDB db,
+                            List<ContainerKeyInfo> containerKeyInfos,
+                            String tableName)
+      throws IOException {
+    long keysProcessed = 0;
+    DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+        dbDefinition.getColumnFamily(tableName);
+    if (columnFamilyDefinition == null) {
+      throw new IllegalStateException(
+          "Table with name" + tableName + " not found");
+    }
+
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+        columnFamilyDefinition.getName().getBytes(UTF_8),
+        columnFamilyHandles);
+    if (columnFamilyHandle == null) {
+      throw new IllegalStateException("columnFamilyHandle is null");
+    }
+
+    try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+        db.get().newIterator(columnFamilyHandle))) {
+      iterator.get().seekToFirst();
+      while (iterator.get().isValid()) {
+        OmKeyInfo value = ((OmKeyInfo) columnFamilyDefinition.getValueCodec()
+            .fromPersistedFormat(iterator.get().value()));
+        List<OmKeyLocationInfoGroup> keyLocationVersions =
+            value.getKeyLocationVersions();
+        if (Objects.isNull(keyLocationVersions)) {
+          iterator.get().next();
+          keysProcessed++;
+          continue;
+        }
+
+        long volumeId = 0;
+        long bucketId = 0;
+        // volumeId and bucketId are only applicable to file table
+        if (tableName.equals(FILE_TABLE)) {
+          String key = new String(iterator.get().key(), UTF_8);
+          String[] keyParts = key.split(OM_KEY_PREFIX);
+          volumeId = Long.parseLong(keyParts[1]);
+          bucketId = Long.parseLong(keyParts[2]);
+        }
+
+        for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+          for (List<OmKeyLocationInfo> locationInfos :
+              locationInfoGroup.getLocationVersionMap().values()) {
+            for (OmKeyLocationInfo locationInfo : locationInfos) {
+              if (containerIds.contains(locationInfo.getContainerID())) {
+                // Generate asbolute key path for FSO keys
+                StringBuilder keyName = new StringBuilder();
+                if (tableName.equals(FILE_TABLE)) {
+                  // Load directory table only after the first fso key is found
+                  // to reduce necessary load if there are not fso keys
+                  if (!isDirTableLoaded) {
+                    long start = System.currentTimeMillis();
+                    directoryTable = getDirectoryTableData(parent.getDbPath());
+                    long end = System.currentTimeMillis();
+                    out().println(
+                        "directoryTable loaded in " + (end - start) + " ms.");
+                    isDirTableLoaded = true;
+                  }
+                  keyName.append(getFsoKeyPrefix(volumeId, bucketId, value));
+                }
+                keyName.append(value.getKeyName());
+                containerKeyInfos.add(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        value.getVolumeName(), volumeId, value.getBucketName(),
+                        bucketId, keyName.toString(),
+                        value.getParentObjectID()));
+              }
+            }
+          }
+        }
+        iterator.get().next();
+        keysProcessed++;
+      }
+      return keysProcessed;
+    } catch (RocksDBException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static String removeBeginningSlash(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      return path.substring(1);
+    }
+
+    return path;
+  }
+
+  private String getFsoKeyPrefix(long volumeId, long bucketId,
+                                 OmKeyInfo value) {
+    String prefix =
+        OM_KEY_PREFIX + volumeId + OM_KEY_PREFIX + bucketId +
+            OM_KEY_PREFIX;
+    Set<Long> dirObjIds = new HashSet<>();
+    dirObjIds.add(value.getParentObjectID());
+    Map<Long, Path> absolutePaths =
+        getAbsolutePathForObjectIDs(bucketId, prefix, Optional.of(dirObjIds));
+    Path path = absolutePaths.get(value.getParentObjectID());
+    String keyPath;
+    if (path.toString().equals(OM_KEY_PREFIX)) {
+      keyPath = path.toString();
+    } else {
+      keyPath = path + OM_KEY_PREFIX;
+    }
+
+    return removeBeginningSlash(keyPath);
+  }
+
+
+  private ColumnFamilyHandle getColumnFamilyHandle(
+      byte[] name, List<ColumnFamilyHandle> columnFamilyHandles) {
+    return columnFamilyHandles
+        .stream()
+        .filter(
+            handle -> {
+              try {
+                return Arrays.equals(handle.getName(), name);
+              } catch (Exception ex) {
+                throw new RuntimeException(ex);
+              }
+            })
+        .findAny()
+        .orElse(null);
+  }
+
+  private String removeTrailingSlashIfNeeded(String dbPath) {
+    if (dbPath.endsWith(OzoneConsts.OZONE_URI_DELIMITER)) {
+      dbPath = dbPath.substring(0, dbPath.length() - 1);
+    }
+    return dbPath;
+  }
+
+  private void printOutput(ContainerKeyInfoWrapper containerKeyInfoWrapper) {
+    List<ContainerKeyInfo> containerKeyInfos =
+        containerKeyInfoWrapper.getContainerKeyInfos();
+    if (containerKeyInfos.isEmpty()) {
+      out().println("No keys were found for container IDs: " + containerIds);
+      out().println(

Review Comment:
   For json tools we should only print json to stdout and print everything else to stderr otherwise it will break when piping to tools like `jq`.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/OMAdmin.java:
##########
@@ -43,6 +40,9 @@
 
 import java.util.Collection;
 
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY;
+

Review Comment:
   nit. Let's avoid import re-ordering for otherwise unaffected files.



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java:
##########
@@ -34,6 +28,8 @@
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;

Review Comment:
   Please turn off the auto-import reordering in your IDE. It makes it difficult to figure out which imports actually changed during reviews.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);

Review Comment:
   Can we use `OmMetadataManagerImpl` as a wrapper over the RocksDB instance? I don't think `OmMetadataManagerImpl` should actually need an `OzoneManager` instance that looks like a mistake introduced in the snapshot code.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",

Review Comment:
   Can we implement the container ID parsing similar to how it was done in #5659  for consistency?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},

Review Comment:
   I think there's been a move to try to make ozone flag names more standard, including not using multi character flags with a single dash.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +
+      "  \"keysProcessed\": 3,\n" +
+      "  \"containerKeys\": {\n" +
+      "    \"1\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 1,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": -123,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": -456,\n" +
+      "        \"keyName\": \"dir1/key1\",\n" +
+      "        \"parentId\": -789\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"2\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 2,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key2\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"3\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 3,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key3\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ]\n" +
+      "  }\n" +
+      "}\n";
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new RDBParser())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForConatainerIds() throws IOException {

Review Comment:
   I think the same typo is in the other test name too.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;

Review Comment:
   I'm not sure this command belongs under the `ldb` subcommand. Right now that command has generic utils that work on any Ozone RocksDB instance, and this is specific to OM. Maybe it hsould be its own top level debug subcommand?



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +
+      "  \"keysProcessed\": 3,\n" +
+      "  \"containerKeys\": {\n" +
+      "    \"1\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 1,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": -123,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": -456,\n" +
+      "        \"keyName\": \"dir1/key1\",\n" +
+      "        \"parentId\": -789\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"2\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 2,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key2\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"3\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 3,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key3\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ]\n" +
+      "  }\n" +
+      "}\n";
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new RDBParser())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForConatainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"--db", dbStore.getDbLocation().getAbsolutePath(), "ckscanner",
+            "-ids", "1,2,3"};
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertTrue(stdout.toString().contains(KEYS_FOUND_OUTPUT));
+
+    Assertions.assertTrue(stderr.toString().isEmpty());
+  }
+
+  @Test
+  void testWhenThereAreNotKeysForConatainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    createFile(volumeId, bucketId, "key1", -987L, bucketId, 4L);
+    createKey("key2", 5L);
+    createKey("key3", 6L);
+
+    String[] cmdArgs =
+        {"--db", dbStore.getDbLocation().getAbsolutePath(), "ckscanner",
+            "-ids", "1,2,3"};
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertTrue(stdout.toString().contains(KEYS_NOT_FOUND_OUTPUT));
+
+    Assertions.assertTrue(stderr.toString().isEmpty());
+  }
+
+  private void createFile(long volumeId, long bucketId, String keyName,
+                          long objectId, long parentId, long containerId)
+      throws IOException {
+    Table<byte[], byte[]> table = dbStore.getTable(FILE_TABLE);
+
+    // format: /volumeId/bucketId/parentId(bucketId)/keyName
+    String key =
+        "/" + volumeId + "/" + bucketId + "/" + parentId + "/" + keyName;
+
+    OmKeyInfo value =
+        getOmKeyInfo("vol1", "bucket1", keyName, containerId, objectId,
+            parentId);
+
+    table.put(key.getBytes(UTF_8),
+        value.getProtobuf(ClientVersion.CURRENT_VERSION).toByteArray());
+  }
+
+  private void createKey(String keyName, long containerId) throws IOException {
+    Table<byte[], byte[]> table = dbStore.getTable(KEY_TABLE);
+
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    // format: /volumeName/bucketName/keyName
+    String key = "/" + volumeName + "/" + bucketName + "/" + keyName;

Review Comment:
   Can we use `OzoneConsts.OM_KEY_PREFIX` here instead? unfortunately the usual builder for these types of things in `OmMetadataManagerImpl#getOzoneKey` is not static.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;

Review Comment:
   I think the directory table may be too large to require completely loading it in to memory.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, FILE_TABLE);
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, KEY_TABLE);
+    }
+    return new ContainerKeyInfoWrapper(keysProcessed, containerKeyInfos);
+  }
+
+  private long processTable(DBDefinition dbDefinition,
+                            List<ColumnFamilyHandle> columnFamilyHandles,
+                            ManagedRocksDB db,
+                            List<ContainerKeyInfo> containerKeyInfos,
+                            String tableName)
+      throws IOException {
+    long keysProcessed = 0;
+    DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+        dbDefinition.getColumnFamily(tableName);
+    if (columnFamilyDefinition == null) {
+      throw new IllegalStateException(
+          "Table with name" + tableName + " not found");
+    }
+
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+        columnFamilyDefinition.getName().getBytes(UTF_8),
+        columnFamilyHandles);
+    if (columnFamilyHandle == null) {
+      throw new IllegalStateException("columnFamilyHandle is null");
+    }
+
+    try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+        db.get().newIterator(columnFamilyHandle))) {
+      iterator.get().seekToFirst();
+      while (iterator.get().isValid()) {
+        OmKeyInfo value = ((OmKeyInfo) columnFamilyDefinition.getValueCodec()
+            .fromPersistedFormat(iterator.get().value()));
+        List<OmKeyLocationInfoGroup> keyLocationVersions =
+            value.getKeyLocationVersions();
+        if (Objects.isNull(keyLocationVersions)) {
+          iterator.get().next();
+          keysProcessed++;
+          continue;
+        }
+
+        long volumeId = 0;
+        long bucketId = 0;
+        // volumeId and bucketId are only applicable to file table
+        if (tableName.equals(FILE_TABLE)) {
+          String key = new String(iterator.get().key(), UTF_8);
+          String[] keyParts = key.split(OM_KEY_PREFIX);
+          volumeId = Long.parseLong(keyParts[1]);
+          bucketId = Long.parseLong(keyParts[2]);
+        }
+
+        for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+          for (List<OmKeyLocationInfo> locationInfos :
+              locationInfoGroup.getLocationVersionMap().values()) {
+            for (OmKeyLocationInfo locationInfo : locationInfos) {
+              if (containerIds.contains(locationInfo.getContainerID())) {
+                // Generate asbolute key path for FSO keys
+                StringBuilder keyName = new StringBuilder();
+                if (tableName.equals(FILE_TABLE)) {
+                  // Load directory table only after the first fso key is found
+                  // to reduce necessary load if there are not fso keys
+                  if (!isDirTableLoaded) {
+                    long start = System.currentTimeMillis();
+                    directoryTable = getDirectoryTableData(parent.getDbPath());
+                    long end = System.currentTimeMillis();
+                    out().println(
+                        "directoryTable loaded in " + (end - start) + " ms.");
+                    isDirTableLoaded = true;
+                  }
+                  keyName.append(getFsoKeyPrefix(volumeId, bucketId, value));

Review Comment:
   Looks like this is doing a full directory walk for every key found. I think this approach would be better:
   1. Scan the file table and save all the matching keys.
   2. If no keys in the file table match, we are done.
   3. Else, do one walk through the directory table to build the full paths for all of those keys.
   As it is this tool is storing all matches in memory either way. 
   
   A more robust approach would be to have the tool print json matches as it goes so that it works with any size input and output. Then we would walk the directory and file table in one iteration to check for matches.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1449027417


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},

Review Comment:
   Removed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1915280353

   > Thanks @mladjan-gadzic for working on this.  I think this tool will be very useful.
   
   Thank you for the review! Currently I am working on something else so I am going to put this PR in draft state, but I will be sure get back to this ASAP. 


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485857805


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoResponse.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Class for response for container key scanner.
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ContainerKeyInfoResponse {
+
+  private final long keysProcessed;
+  private final Map<Long, List<ContainerKeyInfo>> containerKeys;

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1448856062


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, FILE_TABLE);
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, KEY_TABLE);
+    }
+    return new ContainerKeyInfoWrapper(keysProcessed, containerKeyInfos);
+  }
+
+  private long processTable(DBDefinition dbDefinition,
+                            List<ColumnFamilyHandle> columnFamilyHandles,
+                            ManagedRocksDB db,
+                            List<ContainerKeyInfo> containerKeyInfos,
+                            String tableName)
+      throws IOException {
+    long keysProcessed = 0;
+    DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+        dbDefinition.getColumnFamily(tableName);
+    if (columnFamilyDefinition == null) {
+      throw new IllegalStateException(
+          "Table with name" + tableName + " not found");
+    }
+
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+        columnFamilyDefinition.getName().getBytes(UTF_8),
+        columnFamilyHandles);
+    if (columnFamilyHandle == null) {
+      throw new IllegalStateException("columnFamilyHandle is null");
+    }
+
+    try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+        db.get().newIterator(columnFamilyHandle))) {
+      iterator.get().seekToFirst();
+      while (iterator.get().isValid()) {
+        OmKeyInfo value = ((OmKeyInfo) columnFamilyDefinition.getValueCodec()
+            .fromPersistedFormat(iterator.get().value()));
+        List<OmKeyLocationInfoGroup> keyLocationVersions =
+            value.getKeyLocationVersions();
+        if (Objects.isNull(keyLocationVersions)) {
+          iterator.get().next();
+          keysProcessed++;
+          continue;
+        }
+
+        long volumeId = 0;
+        long bucketId = 0;
+        // volumeId and bucketId are only applicable to file table
+        if (tableName.equals(FILE_TABLE)) {
+          String key = new String(iterator.get().key(), UTF_8);
+          String[] keyParts = key.split(OM_KEY_PREFIX);
+          volumeId = Long.parseLong(keyParts[1]);
+          bucketId = Long.parseLong(keyParts[2]);
+        }
+
+        for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+          for (List<OmKeyLocationInfo> locationInfos :
+              locationInfoGroup.getLocationVersionMap().values()) {
+            for (OmKeyLocationInfo locationInfo : locationInfos) {
+              if (containerIds.contains(locationInfo.getContainerID())) {
+                // Generate asbolute key path for FSO keys
+                StringBuilder keyName = new StringBuilder();
+                if (tableName.equals(FILE_TABLE)) {
+                  // Load directory table only after the first fso key is found
+                  // to reduce necessary load if there are not fso keys
+                  if (!isDirTableLoaded) {
+                    long start = System.currentTimeMillis();
+                    directoryTable = getDirectoryTableData(parent.getDbPath());
+                    long end = System.currentTimeMillis();
+                    out().println(
+                        "directoryTable loaded in " + (end - start) + " ms.");
+                    isDirTableLoaded = true;
+                  }
+                  keyName.append(getFsoKeyPrefix(volumeId, bucketId, value));
+                }
+                keyName.append(value.getKeyName());
+                containerKeyInfos.add(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        value.getVolumeName(), volumeId, value.getBucketName(),
+                        bucketId, keyName.toString(),
+                        value.getParentObjectID()));
+              }
+            }
+          }
+        }
+        iterator.get().next();
+        keysProcessed++;
+      }
+      return keysProcessed;
+    } catch (RocksDBException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static String removeBeginningSlash(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      return path.substring(1);
+    }
+
+    return path;
+  }
+
+  private String getFsoKeyPrefix(long volumeId, long bucketId,
+                                 OmKeyInfo value) {
+    String prefix =
+        OM_KEY_PREFIX + volumeId + OM_KEY_PREFIX + bucketId +
+            OM_KEY_PREFIX;
+    Set<Long> dirObjIds = new HashSet<>();
+    dirObjIds.add(value.getParentObjectID());
+    Map<Long, Path> absolutePaths =
+        getAbsolutePathForObjectIDs(bucketId, prefix, Optional.of(dirObjIds));
+    Path path = absolutePaths.get(value.getParentObjectID());
+    String keyPath;
+    if (path.toString().equals(OM_KEY_PREFIX)) {
+      keyPath = path.toString();
+    } else {
+      keyPath = path + OM_KEY_PREFIX;
+    }
+
+    return removeBeginningSlash(keyPath);
+  }
+
+
+  private ColumnFamilyHandle getColumnFamilyHandle(
+      byte[] name, List<ColumnFamilyHandle> columnFamilyHandles) {
+    return columnFamilyHandles
+        .stream()
+        .filter(
+            handle -> {
+              try {
+                return Arrays.equals(handle.getName(), name);
+              } catch (Exception ex) {
+                throw new RuntimeException(ex);
+              }
+            })
+        .findAny()
+        .orElse(null);
+  }
+
+  private String removeTrailingSlashIfNeeded(String dbPath) {
+    if (dbPath.endsWith(OzoneConsts.OZONE_URI_DELIMITER)) {
+      dbPath = dbPath.substring(0, dbPath.length() - 1);
+    }
+    return dbPath;
+  }
+
+  private void printOutput(ContainerKeyInfoWrapper containerKeyInfoWrapper) {
+    List<ContainerKeyInfo> containerKeyInfos =
+        containerKeyInfoWrapper.getContainerKeyInfos();
+    if (containerKeyInfos.isEmpty()) {
+      out().println("No keys were found for container IDs: " + containerIds);
+      out().println(

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1915286908

   > @mladjan-gadzic thanks for working on the patch , but a quick question, why we want to implement this as a CLI feature ? We already have it in recon UI.
   
   As far as I know, complains about Recon UI that we got on this is that Recon is a little behind the real state in cluster even after refresh. Furthermore, AFAIK there is no way to check for a specific containers/keys, but UI output shows everything, and it can be hard to look through. 


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "devmadhuu (via GitHub)" <gi...@apache.org>.
devmadhuu commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1915912272

   > > @mladjan-gadzic thanks for working on the patch , but a quick question, why we want to implement this as a CLI feature ? We already have it in recon UI.
   > 
   > As far as I know, complains about Recon UI that we got on this is that Recon is a little behind the real state in cluster even after refresh. Furthermore, AFAIK there is no way to check for a specific containers/keys, but UI output shows everything, and it can be hard to look through.
   
   @mladjan-gadzic , Recon works on eventual consistency concept as it has periodic tasks to run , so there might be some delay,but you can search any container id in UI itself 


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485322122


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo, OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+
+    Pair<Long, Long> volumeAndBucketId = parseKey(key);
+    Long volumeId = volumeAndBucketId.getLeft();
+    Long bucketId = volumeAndBucketId.getRight();
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+            StringBuilder keyName = new StringBuilder();
+            if (!isDirTableLoaded) {
+              long start = System.currentTimeMillis();
+              directoryTable = getDirectoryTableData(omMetadataManager);
+              long end = System.currentTimeMillis();
+              LOG.info("directoryTable loaded in " + (end - start) + " ms.");
+              isDirTableLoaded = true;
+            }
+            keyName.append(getFsoKeyPrefix(volumeId, bucketId, keyInfo));
+            keyName.append(keyInfo.getKeyName());
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyName.toString(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private static String removeBeginningSlash(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      return path.substring(1);
+    }
+
+    return path;
+  }
+
+  private String getFsoKeyPrefix(long volumeId, long bucketId,
+                                 OmKeyInfo value) {
+    String prefix =

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485307063


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,252 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * This class tests `ozone debug ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON =
+      new GsonBuilder().setPrettyPrinting().create();
+  private static final ContainerKeyInfo KEY_ONE =
+      new ContainerKeyInfo(1L, "vol1", -123L, "bucket1", -456L, "dir1/key1",
+          -789L);
+  private static final ContainerKeyInfo KEY_TWO =
+      new ContainerKeyInfo(2L, "vol1", 0L, "bucket1", 0L, "key2", 0L);
+  private static final ContainerKeyInfo KEY_THREE =
+      new ContainerKeyInfo(3L, "vol1", 0L, "bucket1", 0L, "key3", 0L);
+
+  private static final Map<Long, List<ContainerKeyInfo>> CONTAINER_KEYS =
+      new HashMap<>();
+
+  static {
+    List<ContainerKeyInfo> list1 = new ArrayList<>();
+    list1.add(KEY_ONE);
+    List<ContainerKeyInfo> list2 = new ArrayList<>();
+    list2.add(KEY_TWO);
+    List<ContainerKeyInfo> list3 = new ArrayList<>();
+    list3.add(KEY_THREE);
+    CONTAINER_KEYS.put(1L, list1);
+    CONTAINER_KEYS.put(2L, list2);
+    CONTAINER_KEYS.put(3L, list3);
+  }
+
+  private static final ContainerKeyInfoResponse KEYS_FOUND_OUTPUT =
+      new ContainerKeyInfoResponse(3, CONTAINER_KEYS);
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);

Review Comment:
   Done. Furthermore, I am going to use assertj instead of junit assertions, because I do not like to mix those in a single test class.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1414333428


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Parse a list of container IDs"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2.(Separated by ',')")

Review Comment:
   Is it supposed to be `.` terminated?
   ```suggestion
             "their keys. Example-usage: 1,11,2 (Separated by ',')")
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Parse a list of container IDs"

Review Comment:
   ```suggestion
       description = "Find keys that reference a container"
   ```



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447537112


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =

Review Comment:
   I am not sure what exactly is meant by `default` here. Could you please explain?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447564439


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),

Review Comment:
   > I think you can directly pass DIRECTORY_TABLE.getBytes(UTF_8) rather than getting it all from DBDefinition and then DBColumnFamilyDefinition.
   
   You are right. However, `DBColumnFamilyDefinition`  is needed for getting value codecs later in the code (check like 156 - https://github.com/apache/ozone/pull/5724/files#diff-f755296101c0f607475f2fd53338934b7b0364339a491c2a76350a9c9b8520c5R156). Is that not correct?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485860450


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoResponse.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Class for response for container key scanner.
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ContainerKeyInfoResponse {
+
+  private final long keysProcessed;
+  private final Map<Long, List<ContainerKeyInfo>> containerKeys;
+
+  public ContainerKeyInfoResponse(
+      long keysProcessed, Map<Long, List<ContainerKeyInfo>> containerKeys) {
+    this.keysProcessed = keysProcessed;
+    this.containerKeys = containerKeys;
+  }
+
+  public long getKeysProcessed() {
+    return keysProcessed;
+  }
+
+  public Map<Long, List<ContainerKeyInfo>> getContainerKeys() {
+    return containerKeys;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1503337656


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestFindContainerKeys.java:
##########
@@ -0,0 +1,286 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.debug.container.ContainerCommands;
+import org.apache.hadoop.ozone.debug.container.ContainerKeyInfo;
+import org.apache.hadoop.ozone.debug.container.ContainerKeyInfoResponse;
+import org.apache.hadoop.ozone.debug.container.FindContainerKeys;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test class for {@link FindContainerKeys}.
+ */
+public class TestFindContainerKeys {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
+  private String[] cmdArgs;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerCommands())
+        .addSubcommand(new FindContainerKeys())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+
+    cmdArgs =
+        new String[]{"find-keys", "--om-db", dbStore.getDbLocation().getAbsolutePath(), "--container-ids", "1 2 3"};
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testFSO() throws Exception {
+    /*
+      Structure:
+      keyName (container id)
+
+      /vol1/bucet1
+        - key1 (1)
+        - dir1
+          - key2 (2)
+          - dir2
+            - key3 (3)
+            - key4 (3)
+        - key5 (4)
+     */
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId1 = -789L;
+    long dirObjectId2 = -788L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId1, "dir1");
+    createDirectory(volumeId, bucketId, dirObjectId1, dirObjectId2, "dir2");
+    createFile(volumeId, bucketId, "key1", -987L, bucketId, 1L);
+    createFile(volumeId, bucketId, "key2", -986L, dirObjectId1, 2L);
+    createFile(volumeId, bucketId, "key3", -985L, dirObjectId2, 3L);
+    createFile(volumeId, bucketId, "key4", -984L, dirObjectId2, 3L);
+    createFile(volumeId, bucketId, "key5", -983L, dirObjectId2, 4L);
+
+    closeDbStore();

Review Comment:
   Why do we need to close it in individual test when we are closing it in `@AfterEach`?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1915278399

   > Thanks @mladjan-gadzic for addressing previous review comment.
   > 
   > 
   > 
   > I left couple of questions and some cosmetic suggestions.
   > 
   > 
   > 
   > Also can you please add sample request and its output in the details? If there is any, you used for manual testing.
   
   Thank you for the second review round. I will be sure to do that! 


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485306749


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfo.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import java.util.Objects;
+
+/**
+ * Class that holds basic key data in relation to container it is in.
+ */
+public class ContainerKeyInfo {

Review Comment:
   Done.



##########
hadoop-ozone/tools/pom.xml:
##########
@@ -108,6 +108,21 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>hdds-test-utils</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+      <scope>test</scope>
+    </dependency>

Review Comment:
   Done.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,252 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * This class tests `ozone debug ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON =
+      new GsonBuilder().setPrettyPrinting().create();
+  private static final ContainerKeyInfo KEY_ONE =
+      new ContainerKeyInfo(1L, "vol1", -123L, "bucket1", -456L, "dir1/key1",
+          -789L);
+  private static final ContainerKeyInfo KEY_TWO =
+      new ContainerKeyInfo(2L, "vol1", 0L, "bucket1", 0L, "key2", 0L);
+  private static final ContainerKeyInfo KEY_THREE =
+      new ContainerKeyInfo(3L, "vol1", 0L, "bucket1", 0L, "key3", 0L);
+
+  private static final Map<Long, List<ContainerKeyInfo>> CONTAINER_KEYS =
+      new HashMap<>();
+
+  static {
+    List<ContainerKeyInfo> list1 = new ArrayList<>();
+    list1.add(KEY_ONE);
+    List<ContainerKeyInfo> list2 = new ArrayList<>();
+    list2.add(KEY_TWO);
+    List<ContainerKeyInfo> list3 = new ArrayList<>();
+    list3.add(KEY_THREE);
+    CONTAINER_KEYS.put(1L, list1);
+    CONTAINER_KEYS.put(2L, list2);
+    CONTAINER_KEYS.put(3L, list3);
+  }
+
+  private static final ContainerKeyInfoResponse KEYS_FOUND_OUTPUT =
+      new ContainerKeyInfoResponse(3, CONTAINER_KEYS);
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertEquals(
+        GSON.fromJson(stdout.toString(), ContainerKeyInfoResponse.class),
+        KEYS_FOUND_OUTPUT);
+
+    Assertions.assertTrue(stderr.toString().isEmpty());
+  }
+
+  /**
+   * Close db store because of the lock.
+   */
+  private void closeDbStore() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+  }
+
+  @Test
+  void testWhenThereAreNotKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    createFile(volumeId, bucketId, "key1", -987L, bucketId, 4L);
+    createKey("key2", 5L);
+    createKey("key3", 6L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertTrue(stderr.toString().contains(KEYS_NOT_FOUND_OUTPUT));

Review Comment:
   Done.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,252 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * This class tests `ozone debug ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON =
+      new GsonBuilder().setPrettyPrinting().create();
+  private static final ContainerKeyInfo KEY_ONE =
+      new ContainerKeyInfo(1L, "vol1", -123L, "bucket1", -456L, "dir1/key1",
+          -789L);
+  private static final ContainerKeyInfo KEY_TWO =
+      new ContainerKeyInfo(2L, "vol1", 0L, "bucket1", 0L, "key2", 0L);
+  private static final ContainerKeyInfo KEY_THREE =
+      new ContainerKeyInfo(3L, "vol1", 0L, "bucket1", 0L, "key3", 0L);
+
+  private static final Map<Long, List<ContainerKeyInfo>> CONTAINER_KEYS =
+      new HashMap<>();
+
+  static {
+    List<ContainerKeyInfo> list1 = new ArrayList<>();
+    list1.add(KEY_ONE);
+    List<ContainerKeyInfo> list2 = new ArrayList<>();
+    list2.add(KEY_TWO);
+    List<ContainerKeyInfo> list3 = new ArrayList<>();
+    list3.add(KEY_THREE);
+    CONTAINER_KEYS.put(1L, list1);
+    CONTAINER_KEYS.put(2L, list2);
+    CONTAINER_KEYS.put(3L, list3);
+  }
+
+  private static final ContainerKeyInfoResponse KEYS_FOUND_OUTPUT =
+      new ContainerKeyInfoResponse(3, CONTAINER_KEYS);
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertEquals(
+        GSON.fromJson(stdout.toString(), ContainerKeyInfoResponse.class),
+        KEYS_FOUND_OUTPUT);
+
+    Assertions.assertTrue(stderr.toString().isEmpty());

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1449057198


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1448853413


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +
+      "  \"keysProcessed\": 3,\n" +
+      "  \"containerKeys\": {\n" +
+      "    \"1\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 1,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": -123,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": -456,\n" +
+      "        \"keyName\": \"dir1/key1\",\n" +
+      "        \"parentId\": -789\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"2\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 2,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key2\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"3\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 3,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key3\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ]\n" +
+      "  }\n" +
+      "}\n";
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new RDBParser())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForConatainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"--db", dbStore.getDbLocation().getAbsolutePath(), "ckscanner",
+            "-ids", "1,2,3"};
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertTrue(stdout.toString().contains(KEYS_FOUND_OUTPUT));
+
+    Assertions.assertTrue(stderr.toString().isEmpty());
+  }
+
+  @Test
+  void testWhenThereAreNotKeysForConatainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    createFile(volumeId, bucketId, "key1", -987L, bucketId, 4L);
+    createKey("key2", 5L);
+    createKey("key3", 6L);
+
+    String[] cmdArgs =
+        {"--db", dbStore.getDbLocation().getAbsolutePath(), "ckscanner",
+            "-ids", "1,2,3"};
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertTrue(stdout.toString().contains(KEYS_NOT_FOUND_OUTPUT));
+
+    Assertions.assertTrue(stderr.toString().isEmpty());
+  }
+
+  private void createFile(long volumeId, long bucketId, String keyName,
+                          long objectId, long parentId, long containerId)
+      throws IOException {
+    Table<byte[], byte[]> table = dbStore.getTable(FILE_TABLE);
+
+    // format: /volumeId/bucketId/parentId(bucketId)/keyName
+    String key =
+        "/" + volumeId + "/" + bucketId + "/" + parentId + "/" + keyName;
+
+    OmKeyInfo value =
+        getOmKeyInfo("vol1", "bucket1", keyName, containerId, objectId,
+            parentId);
+
+    table.put(key.getBytes(UTF_8),
+        value.getProtobuf(ClientVersion.CURRENT_VERSION).toByteArray());
+  }
+
+  private void createKey(String keyName, long containerId) throws IOException {
+    Table<byte[], byte[]> table = dbStore.getTable(KEY_TABLE);
+
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    // format: /volumeName/bucketName/keyName
+    String key = "/" + volumeName + "/" + bucketName + "/" + keyName;

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1415941857


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Parse a list of container IDs"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2.(Separated by ',')")

Review Comment:
   No, that was more of a full stop, but I understand why it might be confusing. I'll remove it as suggested.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1463965437


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoResponse.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Class for response for container key scanner.
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ContainerKeyInfoResponse {
+
+  private final long keysProcessed;
+  private final Map<Long, List<ContainerKeyInfo>> containerKeys;
+
+  public ContainerKeyInfoResponse(
+      long keysProcessed, Map<Long, List<ContainerKeyInfo>> containerKeys) {
+    this.keysProcessed = keysProcessed;
+    this.containerKeys = containerKeys;
+  }
+
+  public long getKeysProcessed() {
+    return keysProcessed;
+  }
+
+  public Map<Long, List<ContainerKeyInfo>> getContainerKeys() {
+    return containerKeys;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {

Review Comment:
   nit: you can avoid null check if `instanceof` is used.
   ```suggestion
       if (!(o instanceof ContainerKeyInfoResponse)) {
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoResponse.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Class for response for container key scanner.
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ContainerKeyInfoResponse {
+
+  private final long keysProcessed;
+  private final Map<Long, List<ContainerKeyInfo>> containerKeys;

Review Comment:
   nit: can you please add java doc for this map and what's the key used? Or use better name which tells about key e.g.`XyzToContainerKeys`? Right now, I don't know what's key for this map unless I look where it is set.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo, OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+
+    Pair<Long, Long> volumeAndBucketId = parseKey(key);
+    Long volumeId = volumeAndBucketId.getLeft();
+    Long bucketId = volumeAndBucketId.getRight();
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+            StringBuilder keyName = new StringBuilder();
+            if (!isDirTableLoaded) {
+              long start = System.currentTimeMillis();
+              directoryTable = getDirectoryTableData(omMetadataManager);
+              long end = System.currentTimeMillis();
+              LOG.info("directoryTable loaded in " + (end - start) + " ms.");
+              isDirTableLoaded = true;
+            }
+            keyName.append(getFsoKeyPrefix(volumeId, bucketId, keyInfo));
+            keyName.append(keyInfo.getKeyName());
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyName.toString(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private static String removeBeginningSlash(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      return path.substring(1);
+    }
+
+    return path;
+  }
+
+  private String getFsoKeyPrefix(long volumeId, long bucketId,

Review Comment:
   nit: it returns keyWithPrefix not just keyPrefix if I'm not wrong.
   
   ```suggestion
     private String getFsoKeyWithPrefix(long volumeId, long bucketId,
                                        OmKeyInfo value) {
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo, OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+
+    Pair<Long, Long> volumeAndBucketId = parseKey(key);
+    Long volumeId = volumeAndBucketId.getLeft();
+    Long bucketId = volumeAndBucketId.getRight();
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+            StringBuilder keyName = new StringBuilder();
+            if (!isDirTableLoaded) {
+              long start = System.currentTimeMillis();
+              directoryTable = getDirectoryTableData(omMetadataManager);
+              long end = System.currentTimeMillis();
+              LOG.info("directoryTable loaded in " + (end - start) + " ms.");
+              isDirTableLoaded = true;
+            }
+            keyName.append(getFsoKeyPrefix(volumeId, bucketId, keyInfo));
+            keyName.append(keyInfo.getKeyName());
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyName.toString(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private static String removeBeginningSlash(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      return path.substring(1);
+    }
+
+    return path;
+  }
+
+  private String getFsoKeyPrefix(long volumeId, long bucketId,
+                                 OmKeyInfo value) {
+    String prefix =

Review Comment:
   I think it is time to move [this helper function](https://github.com/apache/ozone/blob/ad2414697e5073c8a76434c28b3d5d09fd6dab84/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java#L234) to OMMetadataManager.
   https://github.com/apache/ozone/pull/5465#discussion_r1372365796
   
   You can have two helper functions, one which take volumeName & bucketName and other takes volumeId & bucketId.
   
   e.g.:
   
   ```
     public static String getOzonePathKeyForFso(OMMetadataManager metadataManager,
                                                String volumeName,
                                                String bucketName)
         throws IOException {
       final long volumeId = metadataManager.getVolumeId(volumeName);
       final long bucketId = metadataManager.getBucketId(volumeName, bucketName);
       return getOzonePathKeyForFso(volumeId, bucketId);
     }
   
     public static String getOzonePathKeyForFso(long volumeId, long bucketId) {
       return OM_KEY_PREFIX + volumeId + OM_KEY_PREFIX + bucketId + OM_KEY_PREFIX;
     }
   ```
   
   



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();

Review Comment:
   nit:should this be called from final block?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",

Review Comment:
   QQ: do we need this to work for snapshot? If so, will it work with snapshot db dir?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(

Review Comment:
   ```suggestion 
    private void processKeyData(Map<Long, List<ContainerKeyInfo>> containerKeyInfos, OmKeyInfo keyInfo) {
       long volumeId = 0L;
       long bucketId = 0L;
   
       for (OmKeyLocationInfoGroup locationInfoGroup : keyInfo.getKeyLocationVersions()) {
         for (List<OmKeyLocationInfo> locationInfos : locationInfoGroup.getLocationVersionMap().values()) {
           for (OmKeyLocationInfo locationInfo : locationInfos) {
             if (!containerIds.contains(locationInfo.getContainerID())) {
               continue;
             }
   
             List<ContainerKeyInfo> containerKeyInfoList = new ArrayList<>();
             containerKeyInfoList.add(new ContainerKeyInfo(locationInfo.getContainerID(),
                 keyInfo.getVolumeName(), volumeId,
                 keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
                 keyInfo.getParentObjectID()));
   
             containerKeyInfos.merge(locationInfo.getContainerID(), containerKeyInfoList,
                 (existingList, newList) -> {
                   existingList.addAll(newList);
                   return existingList;
                 });
           }
         }
       }
     }
   ```
   
   qq: Should we use actual volume and bucket Ids here?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/

Review Comment:
   JFYI, this is valid only for the first dir or key without any dir.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(

Review Comment:
   nit: few suggestions
   1. We recently increased chars per line to 120. Please update your IDE setting if you have not.
   2. Use early return to avoid nesting.
   3. I think `getDirectoryTableData` can be loaded outside the loops and no need to use `isDirTableLoaded` to check if it is loaded or not. If it is to load only when needed, may be use some other way to load lazily.
    
   ```suggestion
     private void processFileData(Map<Long, List<ContainerKeyInfo>> containerKeyInfos, String key, OmKeyInfo keyInfo) {
       Pair<Long, Long> volumeAndBucketId = parseKey(key);
       Long volumeId = volumeAndBucketId.getLeft();
       Long bucketId = volumeAndBucketId.getRight();
   
       for (OmKeyLocationInfoGroup locationInfoGroup : keyInfo.getKeyLocationVersions()) {
         for (List<OmKeyLocationInfo> locationInfos : locationInfoGroup.getLocationVersionMap().values()) {
           for (OmKeyLocationInfo locationInfo : locationInfos) {
             if (!containerIds.contains(locationInfo.getContainerID())) {
               continue;
             }
   
             String keyName = getFsoKeyPrefix(volumeId, bucketId, keyInfo) + keyInfo.getKeyName();
             List<ContainerKeyInfo> containerKeyInfoList = new ArrayList<>();
             containerKeyInfoList.add(new ContainerKeyInfo(locationInfo.getContainerID(),
                 keyInfo.getVolumeName(), volumeId,
                 keyInfo.getBucketName(), bucketId, keyName,
                 keyInfo.getParentObjectID()));
   
             containerKeyInfos.merge(locationInfo.getContainerID(), containerKeyInfoList,
                 (existingList, newList) -> {
                   existingList.addAll(newList);
                   return existingList;
                 });
           }
         }
       }
     }
   ```



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485308147


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",

Review Comment:
   Initially it is not intended to work for snapshot. This can be extended in future if needed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1486006167


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {

Review Comment:
   I've managed to incorporate all of the above conditions into three tests.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1450343068


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;

Review Comment:
   Done. Moved it as subcommand to `debug` command.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1464137793


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =

Review Comment:
   Whenever you read RoaksDB, you need to pass default CF. More details: https://github.com/facebook/rocksdb/wiki/Column-Families
   
    I don't think this comment is valid anymore.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "devmadhuu (via GitHub)" <gi...@apache.org>.
devmadhuu commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1936118061

   > > > > @mladjan-gadzic thanks for working on the patch , but a quick question, why we want to implement this as a CLI feature ? We already have it in recon UI.
   > > > 
   > > > 
   > > > As far as I know, complains about Recon UI that we got on this is that Recon is a little behind the real state in cluster even after refresh. Furthermore, AFAIK there is no way to check for a specific containers/keys, but UI output shows everything, and it can be hard to look through.
   > > 
   > > 
   > > @mladjan-gadzic , Recon works on eventual consistency concept as it has periodic tasks to run , so there might be some delay,but you can search any container id in UI itself
   > 
   > Could you please point me to that page/endpoint? As I can remember, issue was that there was pagination, and in order to get to a particular key we'd need to know on which page it is excatly if there is large number of keys.
   
   **Recon UI currently search across all pages in pagination is under development. But you can still use api/v1/containers/<container_id>/keys**  using curl command 


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447564439


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),

Review Comment:
   > I think you can directly pass DIRECTORY_TABLE.getBytes(UTF_8) rather than getting it all from DBDefinition and then DBColumnFamilyDefinition.
   
   You are right. However, `DBColumnFamilyDefinition`  is needed for getting value codecs later in the code (check like 156 - https://github.com/apache/ozone/pull/5724/files#diff-f755296101c0f607475f2fd53338934b7b0364339a491c2a76350a9c9b8520c5R156). Is that not correct?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447599366


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447605796


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +
+      "  \"keysProcessed\": 3,\n" +
+      "  \"containerKeys\": {\n" +
+      "    \"1\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 1,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": -123,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": -456,\n" +
+      "        \"keyName\": \"dir1/key1\",\n" +
+      "        \"parentId\": -789\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"2\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 2,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key2\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ],\n" +
+      "    \"3\": [\n" +
+      "      {\n" +
+      "        \"containerID\": 3,\n" +
+      "        \"volumeName\": \"vol1\",\n" +
+      "        \"volumeId\": 0,\n" +
+      "        \"bucketName\": \"bucket1\",\n" +
+      "        \"bucketId\": 0,\n" +
+      "        \"keyName\": \"key3\",\n" +
+      "        \"parentId\": 0\n" +
+      "      }\n" +
+      "    ]\n" +
+      "  }\n" +
+      "}\n";
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new RDBParser())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForConatainerIds() throws IOException {

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1935915850

   > > > @mladjan-gadzic thanks for working on the patch , but a quick question, why we want to implement this as a CLI feature ? We already have it in recon UI.
   > > 
   > > 
   > > As far as I know, complains about Recon UI that we got on this is that Recon is a little behind the real state in cluster even after refresh. Furthermore, AFAIK there is no way to check for a specific containers/keys, but UI output shows everything, and it can be hard to look through.
   > 
   > @mladjan-gadzic , Recon works on eventual consistency concept as it has periodic tasks to run , so there might be some delay,but you can search any container id in UI itself
   
   Could you please point me to that page/endpoint? As I can remember, issue was that there was pagination, and in order to get to a particular key we'd need to know on which page it is excatly if there is large number of keys.


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1486008477


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(

Review Comment:
   > We recently increased chars per line to 120. Please update your IDE setting if you have not.
   
   Thanks for this! Somehow I've missed this update.
   
   > Use early return to avoid nesting.
   
   Fixed.
   
   > I think getDirectoryTableData can be loaded outside the loops and no need to use isDirTableLoaded to check if it is loaded or not.
   
   You are right, however, I used that to enable quick lazy loading mechanism.
   
   > If it is to load only when needed, may be use some other way to load lazily.
   
   Do you have any suggestions for 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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1422336283


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Parse a list of container IDs"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2.(Separated by ',')")

Review Comment:
   I think you can end the sentence with period.
   ```
   "their keys. Example-usage: 1,11,2 (Separated by ',').")
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfo.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import java.util.Objects;
+
+/**
+ * Class that holds basic key data in relation to container it is in.
+ */
+public class ContainerKeyInfo {
+
+  private final long containerID;
+  private final String volumeName;
+  private final long volumeId;
+  private final String bucketName;
+  private final long bucketId;
+  private final String keyName;
+  private final long parentId;
+
+  public ContainerKeyInfo(long containerID, String volumeName, long volumeId,
+                          String bucketName, long bucketId, String keyName,
+                          long parentId) {
+    this.containerID = containerID;
+    this.volumeName = volumeName;
+    this.volumeId = volumeId;
+    this.bucketName = bucketName;
+    this.bucketId = bucketId;
+    this.keyName = keyName;
+    this.parentId = parentId;
+  }
+
+  public long getContainerID() {
+    return containerID;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {

Review Comment:
   nit: you can use `instanceof` which will do null check as well.
   ```suggestion
       if (!(o instanceof ContainerKeyInfo)) {
   ```



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1423619403


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(

Review Comment:
   Similar to [comment](https://github.com/apache/ozone/pull/5724#discussion_r1423593884), I don't think dbDefinition is needed. You can directly access CFH.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1448914727


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoWrapper.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import java.util.List;
+
+/**
+ * Class for aggregation of collected data.
+ */
+public class ContainerKeyInfoWrapper {

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1449095864


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;

Review Comment:
   In some use cases it might be. I wasn't sure how to handle this. Reading from table multiple times is time consuming. Any chance you have an idea how this might be handled better?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1448832197


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,241 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * This class tests `ozone debug ldb ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private static final String KEY_TABLE = "keyTable";
+  private static final String FILE_TABLE = "fileTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+
+  private static final String KEYS_FOUND_OUTPUT = "{\n" +

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485279815


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485854708


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1503323330


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(

Review Comment:
   ObjectId (volumeId and BucketId) still should be there no matter their type OBS or FSO.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1450482233


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, FILE_TABLE);
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, KEY_TABLE);
+    }
+    return new ContainerKeyInfoWrapper(keysProcessed, containerKeyInfos);
+  }
+
+  private long processTable(DBDefinition dbDefinition,
+                            List<ColumnFamilyHandle> columnFamilyHandles,
+                            ManagedRocksDB db,
+                            List<ContainerKeyInfo> containerKeyInfos,
+                            String tableName)
+      throws IOException {
+    long keysProcessed = 0;
+    DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+        dbDefinition.getColumnFamily(tableName);
+    if (columnFamilyDefinition == null) {
+      throw new IllegalStateException(
+          "Table with name" + tableName + " not found");
+    }
+
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+        columnFamilyDefinition.getName().getBytes(UTF_8),
+        columnFamilyHandles);
+    if (columnFamilyHandle == null) {
+      throw new IllegalStateException("columnFamilyHandle is null");
+    }
+
+    try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+        db.get().newIterator(columnFamilyHandle))) {
+      iterator.get().seekToFirst();
+      while (iterator.get().isValid()) {
+        OmKeyInfo value = ((OmKeyInfo) columnFamilyDefinition.getValueCodec()
+            .fromPersistedFormat(iterator.get().value()));
+        List<OmKeyLocationInfoGroup> keyLocationVersions =
+            value.getKeyLocationVersions();
+        if (Objects.isNull(keyLocationVersions)) {
+          iterator.get().next();
+          keysProcessed++;
+          continue;
+        }
+
+        long volumeId = 0;
+        long bucketId = 0;
+        // volumeId and bucketId are only applicable to file table
+        if (tableName.equals(FILE_TABLE)) {
+          String key = new String(iterator.get().key(), UTF_8);
+          String[] keyParts = key.split(OM_KEY_PREFIX);
+          volumeId = Long.parseLong(keyParts[1]);
+          bucketId = Long.parseLong(keyParts[2]);
+        }
+
+        for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+          for (List<OmKeyLocationInfo> locationInfos :
+              locationInfoGroup.getLocationVersionMap().values()) {
+            for (OmKeyLocationInfo locationInfo : locationInfos) {
+              if (containerIds.contains(locationInfo.getContainerID())) {
+                // Generate asbolute key path for FSO keys
+                StringBuilder keyName = new StringBuilder();
+                if (tableName.equals(FILE_TABLE)) {
+                  // Load directory table only after the first fso key is found
+                  // to reduce necessary load if there are not fso keys
+                  if (!isDirTableLoaded) {
+                    long start = System.currentTimeMillis();
+                    directoryTable = getDirectoryTableData(parent.getDbPath());
+                    long end = System.currentTimeMillis();
+                    out().println(
+                        "directoryTable loaded in " + (end - start) + " ms.");
+                    isDirTableLoaded = true;
+                  }
+                  keyName.append(getFsoKeyPrefix(volumeId, bucketId, value));

Review Comment:
   There seems to be multiple approaches on how to print output. There is something like `ListVolumeHandler` for listing volumes which prints out proper JSON format and then there is something like `ListSubcommand` for listing containers which prints out JSON-like format but not true JSON. In order to have it printed out as true JSON format, data for printing needs to be kept in memory and accessed at the time of printing out.
   
   With above in mind would you like to see proper JSON format which is better formatted and easier for end user to read or JSON-like format which is faster performance-wise but uglier for end user?



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447603618


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()
+                  .fromPersistedFormat(iterator.get().value())));
+          iterator.get().next();
+        }
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return RDBParser.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoWrapper scanDBForContainerKeys(String dbPath)
+      throws RocksDBException, IOException {
+    List<ContainerKeyInfo> containerKeyInfos = new ArrayList<>();
+
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    long keysProcessed = 0;
+
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, FILE_TABLE);
+      keysProcessed +=
+          processTable(dbDefinition, columnFamilyHandles, db,
+              containerKeyInfos, KEY_TABLE);
+    }
+    return new ContainerKeyInfoWrapper(keysProcessed, containerKeyInfos);
+  }
+
+  private long processTable(DBDefinition dbDefinition,
+                            List<ColumnFamilyHandle> columnFamilyHandles,
+                            ManagedRocksDB db,
+                            List<ContainerKeyInfo> containerKeyInfos,
+                            String tableName)
+      throws IOException {
+    long keysProcessed = 0;
+    DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+        dbDefinition.getColumnFamily(tableName);
+    if (columnFamilyDefinition == null) {
+      throw new IllegalStateException(
+          "Table with name" + tableName + " not found");
+    }
+
+    ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+        columnFamilyDefinition.getName().getBytes(UTF_8),
+        columnFamilyHandles);
+    if (columnFamilyHandle == null) {
+      throw new IllegalStateException("columnFamilyHandle is null");
+    }
+
+    try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+        db.get().newIterator(columnFamilyHandle))) {
+      iterator.get().seekToFirst();
+      while (iterator.get().isValid()) {
+        OmKeyInfo value = ((OmKeyInfo) columnFamilyDefinition.getValueCodec()
+            .fromPersistedFormat(iterator.get().value()));
+        List<OmKeyLocationInfoGroup> keyLocationVersions =
+            value.getKeyLocationVersions();
+        if (Objects.isNull(keyLocationVersions)) {
+          iterator.get().next();
+          keysProcessed++;
+          continue;
+        }
+
+        long volumeId = 0;
+        long bucketId = 0;
+        // volumeId and bucketId are only applicable to file table
+        if (tableName.equals(FILE_TABLE)) {
+          String key = new String(iterator.get().key(), UTF_8);
+          String[] keyParts = key.split(OM_KEY_PREFIX);
+          volumeId = Long.parseLong(keyParts[1]);
+          bucketId = Long.parseLong(keyParts[2]);
+        }
+
+        for (OmKeyLocationInfoGroup locationInfoGroup : keyLocationVersions) {
+          for (List<OmKeyLocationInfo> locationInfos :
+              locationInfoGroup.getLocationVersionMap().values()) {
+            for (OmKeyLocationInfo locationInfo : locationInfos) {
+              if (containerIds.contains(locationInfo.getContainerID())) {
+                // Generate asbolute key path for FSO keys
+                StringBuilder keyName = new StringBuilder();
+                if (tableName.equals(FILE_TABLE)) {
+                  // Load directory table only after the first fso key is found
+                  // to reduce necessary load if there are not fso keys
+                  if (!isDirTableLoaded) {
+                    long start = System.currentTimeMillis();
+                    directoryTable = getDirectoryTableData(parent.getDbPath());
+                    long end = System.currentTimeMillis();
+                    out().println(
+                        "directoryTable loaded in " + (end - start) + " ms.");
+                    isDirTableLoaded = true;
+                  }
+                  keyName.append(getFsoKeyPrefix(volumeId, bucketId, value));
+                }
+                keyName.append(value.getKeyName());
+                containerKeyInfos.add(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        value.getVolumeName(), volumeId, value.getBucketName(),
+                        bucketId, keyName.toString(),
+                        value.getParentObjectID()));
+              }
+            }
+          }
+        }

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1447577555


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),

Review Comment:
   You are right. Removed those.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    // Get all table handles
+    try (ManagedRocksDB db = ManagedRocksDB.openReadOnly(dbPath,
+        columnFamilyDescriptors, columnFamilyHandles)) {
+      dbPath = removeTrailingSlashIfNeeded(dbPath);
+      DBDefinition dbDefinition = DBDefinitionFactory.getDefinition(
+          Paths.get(dbPath), new OzoneConfiguration());
+      if (dbDefinition == null) {
+        throw new IllegalStateException("Incorrect DB Path");
+      }
+
+      // Get directory table
+      DBColumnFamilyDefinition<?, ?> columnFamilyDefinition =
+          dbDefinition.getColumnFamily(DIRECTORY_TABLE);
+      if (columnFamilyDefinition == null) {
+        throw new IllegalStateException(
+            "Table with name" + DIRECTORY_TABLE + " not found");
+      }
+
+      // Get directory table handle
+      ColumnFamilyHandle columnFamilyHandle = getColumnFamilyHandle(
+          columnFamilyDefinition.getName().getBytes(UTF_8),
+          columnFamilyHandles);
+      if (columnFamilyHandle == null) {
+        throw new IllegalStateException("columnFamilyHandle is null");
+      }
+
+      // Get iterator for directory table
+      try (ManagedRocksIterator iterator = new ManagedRocksIterator(
+          db.get().newIterator(columnFamilyHandle))) {
+        iterator.get().seekToFirst();
+        while (iterator.get().isValid()) {
+          directoryTableData.put(new String(iterator.get().key(), UTF_8),
+              ((OmDirectoryInfo) columnFamilyDefinition.getValueCodec()

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1450286012


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,421 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition;
+import org.apache.hadoop.hdds.utils.db.DBDefinition;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB;
+import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner implements Callable<Void>,
+    SubcommandWithParent {
+
+  private static final String FILE_TABLE = "fileTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DIRECTORY_TABLE = "directoryTable";
+
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.ParentCommand
+  private RDBParser parent;
+
+  @CommandLine.Option(names = {"-ids", "--container-ids"},
+      split = ",",
+      paramLabel = "containerIDs",
+      required = true,
+      description = "Set of container IDs to be used for getting all " +
+          "their keys. Example-usage: 1,11,2 (Separated by ',').")
+  private Set<Long> containerIds;
+
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    ContainerKeyInfoWrapper containerKeyInfoWrapper =
+        scanDBForContainerKeys(parent.getDbPath());
+
+    printOutput(containerKeyInfoWrapper);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(String dbPath)
+      throws RocksDBException, IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    // Get all tables from RocksDB
+    List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        RocksDBUtils.getColumnFamilyDescriptors(dbPath);

Review Comment:
   Done.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1889285344

   @hemantk-12 @errose28 thank you for the review once more. I addressed all the review comments and left some inline questions. Please, let me know what do you think.


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1468492708


##########
hadoop-ozone/tools/pom.xml:
##########
@@ -108,6 +108,21 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>hdds-test-utils</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+      <scope>test</scope>
+    </dependency>

Review Comment:
   These dependencies are available via `hdds-hadoop-dependency-test`.
   
   ```suggestion
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,252 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * This class tests `ozone debug ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON =
+      new GsonBuilder().setPrettyPrinting().create();
+  private static final ContainerKeyInfo KEY_ONE =
+      new ContainerKeyInfo(1L, "vol1", -123L, "bucket1", -456L, "dir1/key1",
+          -789L);
+  private static final ContainerKeyInfo KEY_TWO =
+      new ContainerKeyInfo(2L, "vol1", 0L, "bucket1", 0L, "key2", 0L);
+  private static final ContainerKeyInfo KEY_THREE =
+      new ContainerKeyInfo(3L, "vol1", 0L, "bucket1", 0L, "key3", 0L);
+
+  private static final Map<Long, List<ContainerKeyInfo>> CONTAINER_KEYS =
+      new HashMap<>();
+
+  static {
+    List<ContainerKeyInfo> list1 = new ArrayList<>();
+    list1.add(KEY_ONE);
+    List<ContainerKeyInfo> list2 = new ArrayList<>();
+    list2.add(KEY_TWO);
+    List<ContainerKeyInfo> list3 = new ArrayList<>();
+    list3.add(KEY_THREE);
+    CONTAINER_KEYS.put(1L, list1);
+    CONTAINER_KEYS.put(2L, list2);
+    CONTAINER_KEYS.put(3L, list3);
+  }
+
+  private static final ContainerKeyInfoResponse KEYS_FOUND_OUTPUT =
+      new ContainerKeyInfoResponse(3, CONTAINER_KEYS);
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertEquals(
+        GSON.fromJson(stdout.toString(), ContainerKeyInfoResponse.class),
+        KEYS_FOUND_OUTPUT);
+
+    Assertions.assertTrue(stderr.toString().isEmpty());
+  }
+
+  /**
+   * Close db store because of the lock.
+   */
+  private void closeDbStore() throws IOException {
+    if (dbStore != null) {
+      dbStore.close();
+    }
+  }
+
+  @Test
+  void testWhenThereAreNotKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    createFile(volumeId, bucketId, "key1", -987L, bucketId, 4L);
+    createKey("key2", 5L);
+    createKey("key3", 6L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertTrue(stderr.toString().contains(KEYS_NOT_FOUND_OUTPUT));

Review Comment:
   similarly:
   
   ```suggestion
       assertThat(stderr.toString()).contains(KEYS_NOT_FOUND_OUTPUT);
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfoResponse.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Class for response for container key scanner.
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ContainerKeyInfoResponse {
+
+  private final long keysProcessed;
+  private final Map<Long, List<ContainerKeyInfo>> containerKeys;
+
+  public ContainerKeyInfoResponse(
+      long keysProcessed, Map<Long, List<ContainerKeyInfo>> containerKeys) {
+    this.keysProcessed = keysProcessed;
+    this.containerKeys = containerKeys;
+  }
+
+  public long getKeysProcessed() {
+    return keysProcessed;
+  }
+
+  public Map<Long, List<ContainerKeyInfo>> getContainerKeys() {
+    return containerKeys;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {

Review Comment:
   @hemantk-12 the class must also be `final` to be OK to use `instanceof` in `equals()`.  Otherwise equality with an instance of a subclass may not be symmetric (i.e. result may vary according to which instance is `equals(other)` invoked on.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,252 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * This class tests `ozone debug ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON =
+      new GsonBuilder().setPrettyPrinting().create();
+  private static final ContainerKeyInfo KEY_ONE =
+      new ContainerKeyInfo(1L, "vol1", -123L, "bucket1", -456L, "dir1/key1",
+          -789L);
+  private static final ContainerKeyInfo KEY_TWO =
+      new ContainerKeyInfo(2L, "vol1", 0L, "bucket1", 0L, "key2", 0L);
+  private static final ContainerKeyInfo KEY_THREE =
+      new ContainerKeyInfo(3L, "vol1", 0L, "bucket1", 0L, "key3", 0L);
+
+  private static final Map<Long, List<ContainerKeyInfo>> CONTAINER_KEYS =
+      new HashMap<>();
+
+  static {
+    List<ContainerKeyInfo> list1 = new ArrayList<>();
+    list1.add(KEY_ONE);
+    List<ContainerKeyInfo> list2 = new ArrayList<>();
+    list2.add(KEY_TWO);
+    List<ContainerKeyInfo> list3 = new ArrayList<>();
+    list3.add(KEY_THREE);
+    CONTAINER_KEYS.put(1L, list1);
+    CONTAINER_KEYS.put(2L, list2);
+    CONTAINER_KEYS.put(3L, list3);
+  }
+
+  private static final ContainerKeyInfoResponse KEYS_FOUND_OUTPUT =
+      new ContainerKeyInfoResponse(3, CONTAINER_KEYS);
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);
+
+    Assertions.assertEquals(
+        GSON.fromJson(stdout.toString(), ContainerKeyInfoResponse.class),
+        KEYS_FOUND_OUTPUT);
+
+    Assertions.assertTrue(stderr.toString().isEmpty());

Review Comment:
   nit: please use `assertThat` for better failure message
   
   ```suggestion
       assertThat(stderr.toString()).isEmpty();
   ```
   
   (needs `import static org.assertj.core.api.Assertions.assertThat`)



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyInfo.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import java.util.Objects;
+
+/**
+ * Class that holds basic key data in relation to container it is in.
+ */
+public class ContainerKeyInfo {

Review Comment:
   nit:
   
   ```suggestion
   public final class ContainerKeyInfo {
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestContainerKeyScanner.java:
##########
@@ -0,0 +1,252 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * This class tests `ozone debug ckscanner` CLI that reads from RocksDB
+ * and gets keys for container ids.
+ */
+public class TestContainerKeyScanner {
+  private DBStore dbStore;
+  @TempDir
+  private File tempDir;
+  private StringWriter stdout, stderr;
+  private PrintWriter pstdout, pstderr;
+  private CommandLine cmd;
+  private static final Gson GSON =
+      new GsonBuilder().setPrettyPrinting().create();
+  private static final ContainerKeyInfo KEY_ONE =
+      new ContainerKeyInfo(1L, "vol1", -123L, "bucket1", -456L, "dir1/key1",
+          -789L);
+  private static final ContainerKeyInfo KEY_TWO =
+      new ContainerKeyInfo(2L, "vol1", 0L, "bucket1", 0L, "key2", 0L);
+  private static final ContainerKeyInfo KEY_THREE =
+      new ContainerKeyInfo(3L, "vol1", 0L, "bucket1", 0L, "key3", 0L);
+
+  private static final Map<Long, List<ContainerKeyInfo>> CONTAINER_KEYS =
+      new HashMap<>();
+
+  static {
+    List<ContainerKeyInfo> list1 = new ArrayList<>();
+    list1.add(KEY_ONE);
+    List<ContainerKeyInfo> list2 = new ArrayList<>();
+    list2.add(KEY_TWO);
+    List<ContainerKeyInfo> list3 = new ArrayList<>();
+    list3.add(KEY_THREE);
+    CONTAINER_KEYS.put(1L, list1);
+    CONTAINER_KEYS.put(2L, list2);
+    CONTAINER_KEYS.put(3L, list3);
+  }
+
+  private static final ContainerKeyInfoResponse KEYS_FOUND_OUTPUT =
+      new ContainerKeyInfoResponse(3, CONTAINER_KEYS);
+
+  private static final String KEYS_NOT_FOUND_OUTPUT =
+      "No keys were found for container IDs: [1, 2, 3]\n" +
+          "Keys processed: 3\n";
+
+  @BeforeEach
+  public void setup() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    stdout = new StringWriter();
+    pstdout = new PrintWriter(stdout);
+    stderr = new StringWriter();
+    pstderr = new PrintWriter(stderr);
+
+    cmd = new CommandLine(new OzoneDebug())
+        .addSubcommand(new ContainerKeyScanner())
+        .setOut(pstdout)
+        .setErr(pstderr);
+
+    dbStore = DBStoreBuilder.newBuilder(conf).setName("om.db")
+        .setPath(tempDir.toPath()).addTable(KEY_TABLE).addTable(FILE_TABLE)
+        .addTable(DIRECTORY_TABLE)
+        .build();
+  }
+
+  @AfterEach
+  public void shutdown() throws IOException {
+    closeDbStore();
+    pstderr.close();
+    stderr.close();
+    pstdout.close();
+    stdout.close();
+  }
+
+  @Test
+  void testWhenThereAreKeysForContainerIds() throws IOException {
+
+    // create keys for tables
+    long volumeId = -123L;
+    long bucketId = -456L;
+    long dirObjectId = -789L;
+    createDirectory(volumeId, bucketId, bucketId, dirObjectId, "dir1");
+    createFile(volumeId, bucketId, "key1", -987L, dirObjectId, 1L);
+    createKey("key2", 2L);
+    createKey("key3", 3L);
+
+    String[] cmdArgs =
+        {"ckscanner", "--om-db", dbStore.getDbLocation().getAbsolutePath(),
+            "--container-ids", "1 2 3"};
+
+    closeDbStore();
+
+    int exitCode = cmd.execute(cmdArgs);
+    Assertions.assertEquals(0, exitCode);

Review Comment:
   nit: please `import static` assertion methods.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",

Review Comment:
   Please add it as a sub-command of `ozone debug container`, then it can be named something like `keys` (or `list-keys` / `find-keys`, whichever you prefer).
   
   see `hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/container/ContainerCommands.java`
   
   I'd avoid using the term `scan`, to avoid confusion with container scanners (data validation process in datanode).



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on PR #5724:
URL: https://github.com/apache/ozone/pull/5724#issuecomment-1850055856

   @kerneltime @hemantk-12 thanks for the review!


-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1503323330


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(

Review Comment:
   ObjectId (volumeId and bucketId) still should be there no matter their type OBS or FSO.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "mladjan-gadzic (via GitHub)" <gi...@apache.org>.
mladjan-gadzic commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1485308404


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(

Review Comment:
   If I am not mistaken, non FSO layouts does not have volume/bucket id (it is 0), so I'd keep it here as well.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo, OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+
+    Pair<Long, Long> volumeAndBucketId = parseKey(key);
+    Long volumeId = volumeAndBucketId.getLeft();
+    Long bucketId = volumeAndBucketId.getRight();
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+            StringBuilder keyName = new StringBuilder();
+            if (!isDirTableLoaded) {
+              long start = System.currentTimeMillis();
+              directoryTable = getDirectoryTableData(omMetadataManager);
+              long end = System.currentTimeMillis();
+              LOG.info("directoryTable loaded in " + (end - start) + " ms.");
+              isDirTableLoaded = true;
+            }
+            keyName.append(getFsoKeyPrefix(volumeId, bucketId, keyInfo));
+            keyName.append(keyInfo.getKeyName());
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyName.toString(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private static String removeBeginningSlash(String path) {
+    if (path.startsWith(OM_KEY_PREFIX)) {
+      return path.substring(1);
+    }
+
+    return path;
+  }
+
+  private String getFsoKeyPrefix(long volumeId, long bucketId,

Review Comment:
   Fixed.



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


Re: [PR] HDDS-9824. Provide CLI that scans keys for a matching container and lists keys [ozone]

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #5724:
URL: https://github.com/apache/ozone/pull/5724#discussion_r1504801851


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerKeyScanner.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import com.google.gson.GsonBuilder;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.kohsuke.MetaInfServices;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.ROOT_PATH;
+
+/**
+ * Parser for a list of container IDs, to scan for keys.
+ */
+@CommandLine.Command(
+    name = "ckscanner",
+    description = "Find keys that reference a container"
+)
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerKeyScanner
+    implements Callable<Void>, SubcommandWithParent {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerKeyScanner.class);
+  @CommandLine.Spec
+  private static CommandLine.Model.CommandSpec spec;
+  @CommandLine.Option(names = {"--om-db"},
+      paramLabel = "<OM DB path>",
+      required = true,
+      description = "Path to OM DB.")
+  private String dbPath;
+  @CommandLine.Option(names = {"--container-ids"},
+      split = " ",
+      paramLabel = "<container ID>",
+      required = true,
+      description = "One or more container IDs separated by spaces.")
+  private Set<Long> containerIds;
+  private static Map<String, OmDirectoryInfo> directoryTable;
+  private static boolean isDirTableLoaded = false;
+
+  @Override
+  public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set("ozone.om.db.dirs",
+        dbPath.substring(0, dbPath.lastIndexOf("/")));
+    OmMetadataManagerImpl omMetadataManager =
+        new OmMetadataManagerImpl(ozoneConfiguration, null);
+
+    ContainerKeyInfoResponse containerKeyInfoResponse =
+        scanDBForContainerKeys(omMetadataManager);
+
+    printOutput(containerKeyInfoResponse);
+
+    closeStdChannels();
+
+    return null;
+  }
+
+  private void closeStdChannels() {
+    out().close();
+    err().close();
+  }
+
+  private Map<String, OmDirectoryInfo> getDirectoryTableData(
+      OmMetadataManagerImpl metadataManager)
+      throws IOException {
+    Map<String, OmDirectoryInfo> directoryTableData = new HashMap<>();
+
+    try (
+        TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+            iterator = metadataManager.getDirectoryTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> next = iterator.next();
+        directoryTableData.put(next.getKey(), next.getValue());
+      }
+    }
+
+    return directoryTableData;
+  }
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  private static PrintWriter err() {
+    return spec.commandLine().getErr();
+  }
+
+  private static PrintWriter out() {
+    return spec.commandLine().getOut();
+  }
+
+  public Map<Long, Path> getAbsolutePathForObjectIDs(
+      long bucketId, String prefix, Optional<Set<Long>> dirObjIds) {
+    // Root of a bucket would always have the
+    // key as /volumeId/bucketId/bucketId/
+    if (!dirObjIds.isPresent() || dirObjIds.get().isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Set<Long> objIds = Sets.newHashSet(dirObjIds.get());
+    Map<Long, Path> objectIdPathMap = new HashMap<>();
+    Queue<Pair<Long, Path>> objectIdPathVals = new LinkedList<>();
+    Pair<Long, Path> root = Pair.of(bucketId, ROOT_PATH);
+    objectIdPathVals.add(root);
+    addToPathMap(root, objIds, objectIdPathMap);
+
+    while (!objectIdPathVals.isEmpty() && !objIds.isEmpty()) {
+      Pair<Long, Path> parentPair = objectIdPathVals.poll();
+      String subDir = prefix + parentPair.getKey() + OM_KEY_PREFIX;
+
+      Iterator<String> subDirIterator =
+          directoryTable.keySet().stream()
+              .filter(k -> k.startsWith(subDir))
+              .collect(Collectors.toList()).iterator();
+      while (!objIds.isEmpty() && subDirIterator.hasNext()) {
+        OmDirectoryInfo childDir =
+            directoryTable.get(subDirIterator.next());
+        Pair<Long, Path> pathVal = Pair.of(childDir.getObjectID(),
+            parentPair.getValue().resolve(childDir.getName()));
+        addToPathMap(pathVal, objIds, objectIdPathMap);
+        objectIdPathVals.add(pathVal);
+      }
+    }
+    // Invalid directory objectId which does not exist in the given bucket.
+    if (!objIds.isEmpty()) {
+      throw new IllegalArgumentException(
+          "Dir object Ids required but not found in bucket: " + objIds);
+    }
+    return objectIdPathMap;
+  }
+
+  private void addToPathMap(Pair<Long, Path> objectIDPath,
+                            Set<Long> dirObjIds, Map<Long, Path> pathMap) {
+    if (dirObjIds.contains(objectIDPath.getKey())) {
+      pathMap.put(objectIDPath.getKey(), objectIDPath.getValue());
+      dirObjIds.remove(objectIDPath.getKey());
+    }
+  }
+
+  private ContainerKeyInfoResponse scanDBForContainerKeys(
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    Map<Long, List<ContainerKeyInfo>> containerKeyInfos = new HashMap<>();
+
+    long keysProcessed = 0;
+
+    keysProcessed += processFileTable(containerKeyInfos, omMetadataManager);
+    keysProcessed += processKeyTable(containerKeyInfos, omMetadataManager);
+
+    return new ContainerKeyInfoResponse(keysProcessed, containerKeyInfos);
+  }
+
+  private long processKeyTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager) throws IOException {
+    long keysProcessed = 0L;
+
+    // Anything but not FSO bucket layout
+    Table<String, OmKeyInfo> fileTable = omMetadataManager.getKeyTable(
+        BucketLayout.DEFAULT);
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = fileTable.iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processKeyData(containerKeyInfos, next.getKey(), next.getValue());
+      }
+    }
+
+    return keysProcessed;
+  }
+
+
+  private long processFileTable(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      OmMetadataManagerImpl omMetadataManager)
+      throws IOException {
+    long keysProcessed = 0L;
+
+    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+             iterator = omMetadataManager.getFileTable().iterator()) {
+      while (iterator.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> next = iterator.next();
+        keysProcessed++;
+
+        if (Objects.isNull(next.getValue().getKeyLocationVersions())) {
+          continue;
+        }
+
+        processFileData(containerKeyInfos, next.getKey(), next.getValue(),
+            omMetadataManager);
+      }
+    }
+
+    return keysProcessed;
+  }
+
+  /**
+   * @param key file table key.
+   * @return Pair of volume id and bucket id.
+   */
+  private Pair<Long, Long> parseKey(String key) {
+    String[] keyParts = key.split(OM_KEY_PREFIX);
+    return Pair.of(Long.parseLong(keyParts[1]), Long.parseLong(keyParts[2]));
+  }
+
+  private void processKeyData(
+      Map<Long, List<ContainerKeyInfo>> containerKeyInfos,
+      String key, OmKeyInfo keyInfo) {
+    long volumeId = 0L;
+    long bucketId = 0L;
+
+    for (OmKeyLocationInfoGroup locationInfoGroup :
+        keyInfo.getKeyLocationVersions()) {
+      for (List<OmKeyLocationInfo> locationInfos :
+          locationInfoGroup.getLocationVersionMap().values()) {
+        for (OmKeyLocationInfo locationInfo : locationInfos) {
+          if (containerIds.contains(locationInfo.getContainerID())) {
+
+            containerKeyInfos.merge(locationInfo.getContainerID(),
+                new ArrayList<>(Collections.singletonList(
+                    new ContainerKeyInfo(locationInfo.getContainerID(),
+                        keyInfo.getVolumeName(), volumeId,
+                        keyInfo.getBucketName(), bucketId, keyInfo.getKeyName(),
+                        keyInfo.getParentObjectID()))),
+                (existingList, newList) -> {
+                  existingList.addAll(newList);
+                  return existingList;
+                });
+          }
+        }
+      }
+    }
+  }
+
+  private void processFileData(

Review Comment:
   My suggestion would be something similar to [Lazy_initialization](https://en.wikipedia.org/wiki/Singleton_pattern#Lazy_initialization).



-- 
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-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org