You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "smengcl (via GitHub)" <gi...@apache.org> on 2024/01/04 15:48:47 UTC

[PR] HDDS-8830. [hsync] Add CLI to list open files [ozone]

smengcl opened a new pull request, #5920:
URL: https://github.com/apache/ozone/pull/5920

   WIP. Basic functionality works. Needs further polishing. See TODOs inline.
   
   ## What changes were proposed in this pull request?
   
   Add Ozone Admin CLI `listopenfiles` as a debug command for admins and devs.
   
   ```bash
   ozone admin om listopenfiles
   ```
   
   - [x] Filter by bucket or key prefix
   - [x] Pagination support
   - [ ] Documentation
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-8830
   
   ## How was this patch tested?
   
   - [ ] Integration test addition - WIP.
   - [ ] Unit test addition?
   - [ ] Test manually in Docker.


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java:
##########
@@ -428,10 +430,18 @@ public void incNumListS3Buckets() {
   }
 
   public void incNumListS3BucketsFails() {
-    numBucketOps.incr();

Review Comment:
   Correct. Haven't bothered to file a new jira to do this one-line fix.
   
   `Fails` metric isn't supposed to increase `Ops`.



-- 
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-8830. Add admin CLI to list open files [ozone]

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

   Filed HDDS-10077 for hsync metadata tag improvement.


-- 
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-8830. Add admin CLI to list open files [ozone]

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

   Looks like branch HDDS-7593 needs a master branch merge before I can cleanly base this PR on to that one.


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java:
##########
@@ -558,6 +561,128 @@ public void testListKeysWithFewDeleteEntriesInCache() throws Exception {
 
   }
 
+  @Test

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-8830. Add admin CLI to list open files [ozone]

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

   Right now this is returning object IDs in the path of the keys, which were previously only internal to OM. What is the expected use of this CLI? If it is to do some sort of file close/reclamation based on key name the current paths returned will not help. Alternatively, providing the actual full paths may be problematic because keys can remain open after their parent directories or buckets are deleted, and only the final commit will fail.


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java:
##########
@@ -558,6 +561,128 @@ public void testListKeysWithFewDeleteEntriesInCache() throws Exception {
 
   }
 
+  @Test

Review Comment:
   These three test cases can be parameterized using JUnit5.



-- 
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-8830. Add admin CLI to list open files [ozone]

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

   Thanks @jojochuang for merging this to the HDDS-7593 feature branch. And thanks @adoroszlai and @errose28 for the reviews and comments.
   
   Will post a PR for HDDS-10077 that depends on this one.


-- 
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-8830. Add admin CLI to list open files [ozone]

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

   > Right now this is returning object IDs in the path of the keys, which were previously only internal to OM. What is the expected use of this CLI? If it is to do some sort of file close/reclamation based on key name the current paths returned will not help. Alternatively, providing the actual full paths may be problematic because keys can remain open after their parent directories or buckets are deleted, and only the final commit will fail.
   
   At the beginning I figure this command is just supposed to return what is there in the `OpenKey(File)Table`. Then it gets complicated.
   
   I think just like hdfs's, this command is mostly for admins to get a rough idea of how many files are open, and which ones. Thus human-readable output is important. JSON output might be used for scripting. The `OmKeyInfo` is there tho I think most of the fields may not be very useful, but its what `OpenKey(File)Table` has.
   
   We could also return the `dbKey` of `OpenKey(File)Table` if that becomes useful. @jojochuang 


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java:
##########
@@ -549,6 +555,134 @@ public void testOzoneAdminCmdList() throws UnsupportedEncodingException {
     execute(ozoneAdminShell, args);
   }
 
+  @Test
+  public void testOzoneAdminCmdListOpenFiles()
+      throws IOException, InterruptedException {
+
+    OzoneConfiguration conf = cluster.getConf();
+    final String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + omServiceId;
+
+    OzoneConfiguration clientConf = getClientConfForOFS(hostPrefix, conf);
+    clientConf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
+    FileSystem fs = FileSystem.get(clientConf);
+
+    assertNotEquals(fs.getConf().get(OZONE_FS_HSYNC_ENABLED),
+        "false", OZONE_FS_HSYNC_ENABLED + " is set to false " +
+            "by external force. Must be true to allow hsync to function");
+
+    final String volumeName = "volume-lof";
+    final String bucketName = "buck1";
+
+    String dir1 = hostPrefix + OM_KEY_PREFIX + volumeName + OM_KEY_PREFIX +
+        bucketName + OM_KEY_PREFIX + "dir1";
+    // Create volume, bucket, dir
+    assertTrue(fs.mkdirs(new Path(dir1)));
+    String keyPrefix = OM_KEY_PREFIX + "key";
+
+    final int numKeys = 5;
+    String[] keys = new String[numKeys];
+
+    for (int i = 0; i < numKeys; i++) {
+      keys[i] = dir1 + keyPrefix + i;
+    }
+
+    int pageSize = 3;
+
+    FSDataOutputStream[] streams = new FSDataOutputStream[numKeys];
+    // Create multiple keys and hold them open
+    for (int i = 0; i < numKeys; i++) {
+      streams[i] = fs.create(new Path(keys[i]));
+      streams[i].write(1);
+    }

Review Comment:
   Move into the `try` block below.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om list-open-files command.
+ */
+@CommandLine.Command(
+    name = "list-open-files",
+    aliases = {"list-open-keys", "lof", "lok"},
+    description = "Lists open files (keys) in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},

Review Comment:
   Please do not add `-id` and `-host` (see HDDS-6736).



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java:
##########
@@ -1451,7 +1585,7 @@ public void testKeyDeleteWhenTrashEnableOBS()
   }
 
   @Test
-  public void testRecursiveBucketDelete()
+  public void testZRecursiveBucketDelete()

Review Comment:
   If you intend to run this last, renaming it may not achieve that, since default order is "obscure":
   
   > By default, test classes and methods will be ordered using an algorithm that is deterministic but intentionally nonobvious.
   
   Please use [JUnit5 ordering](https://junit.org/junit5/docs/current/user-guide/#writing-tests-test-execution-order-methods).
   
   Example:
   
   https://github.com/apache/ozone/blob/46b6f3def1d84ca769affb4d3f0d84dece6e8567/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithStoppedNodes.java#L77-L78
   
   and
   
   https://github.com/apache/ozone/blob/46b6f3def1d84ca769affb4d3f0d84dece6e8567/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithStoppedNodes.java#L241-L243



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java:
##########
@@ -558,6 +561,128 @@ public void testListKeysWithFewDeleteEntriesInCache() throws Exception {
 
   }
 
+  @Test
+  public void testListOpenFilesFSO() throws Exception {
+    testListOpenFiles(BucketLayout.FILE_SYSTEM_OPTIMIZED);
+  }
+
+  @Test
+  public void testListOpenFilesOBS() throws Exception {
+    testListOpenFiles(BucketLayout.OBJECT_STORE);
+  }
+
+  @Test
+  public void testListOpenFilesLegacy() throws Exception {
+    // OBS and LEGACY should share the same internal structure for the most part
+    // still, testing both here for the sake of completeness
+    testListOpenFiles(BucketLayout.LEGACY);
+  }
+
+  /**
+   * Tests inner impl of listOpenFiles with different bucket types with and
+   * without pagination. NOTE: This UT does NOT test hsync in this since hsync
+   * status check is done purely on the client side.
+   * @param bucketLayout BucketLayout
+   */
+  public void testListOpenFiles(BucketLayout bucketLayout) throws Exception {

Review Comment:
   Let's use `@ParameterizedTest` for the same.
   
   ```suggestion
     /**
      * Tests inner impl of listOpenFiles with different bucket types with and
      * without pagination. NOTE: This UT does NOT test hsync in this since hsync
      * status check is done purely on the client side.
      */
     @ParameterizedTest
     @EnumSource
     void testListOpenFiles(BucketLayout bucketLayout) throws Exception {
   ```
   
   (additional import needed for `EnumSource`)



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java:
##########
@@ -549,6 +555,134 @@ public void testOzoneAdminCmdList() throws UnsupportedEncodingException {
     execute(ozoneAdminShell, args);
   }
 
+  @Test
+  public void testOzoneAdminCmdListOpenFiles()
+      throws IOException, InterruptedException {
+
+    OzoneConfiguration conf = cluster.getConf();
+    final String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + omServiceId;
+
+    OzoneConfiguration clientConf = getClientConfForOFS(hostPrefix, conf);
+    clientConf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
+    FileSystem fs = FileSystem.get(clientConf);
+
+    assertNotEquals(fs.getConf().get(OZONE_FS_HSYNC_ENABLED),
+        "false", OZONE_FS_HSYNC_ENABLED + " is set to false " +
+            "by external force. Must be true to allow hsync to function");
+
+    final String volumeName = "volume-lof";
+    final String bucketName = "buck1";
+
+    String dir1 = hostPrefix + OM_KEY_PREFIX + volumeName + OM_KEY_PREFIX +
+        bucketName + OM_KEY_PREFIX + "dir1";
+    // Create volume, bucket, dir
+    assertTrue(fs.mkdirs(new Path(dir1)));
+    String keyPrefix = OM_KEY_PREFIX + "key";
+
+    final int numKeys = 5;
+    String[] keys = new String[numKeys];
+
+    for (int i = 0; i < numKeys; i++) {
+      keys[i] = dir1 + keyPrefix + i;
+    }
+
+    int pageSize = 3;
+
+    FSDataOutputStream[] streams = new FSDataOutputStream[numKeys];
+    // Create multiple keys and hold them open
+    for (int i = 0; i < numKeys; i++) {
+      streams[i] = fs.create(new Path(keys[i]));
+      streams[i].write(1);
+    }
+
+    String path = "/" +  volumeName + "/" + bucketName;
+    try {
+      // Wait for DB flush
+      cluster.getOzoneManager().awaitDoubleBufferFlush();
+
+      String[] args = new String[] {"om", "lof",
+          "-id", omServiceId,
+          "-l", String.valueOf(numKeys + 1),  // pagination
+          "-p", path};
+      // Run listopenfiles
+      execute(ozoneAdminShell, args);
+      String cmdRes = getStdOut();
+      // Should have retrieved all 5 open keys
+      for (int i = 0; i < numKeys; i++) {
+        assertTrue(cmdRes.contains(keyPrefix + i));
+      }
+
+      // Try pagination
+      args = new String[] {"om", "lof",
+          "-id", omServiceId,
+          "-l", String.valueOf(pageSize),  // pagination
+          "-p", path};
+      execute(ozoneAdminShell, args);
+      cmdRes = getStdOut();
+
+      // Should have retrieved the 1st page only (3 keys)
+      for (int i = 0; i < pageSize; i++) {
+        assertTrue(cmdRes.contains(keyPrefix + i));
+      }
+      for (int i = pageSize; i < numKeys; i++) {
+        assertFalse(cmdRes.contains(keyPrefix + i));
+      }
+      // No hsync'ed file/key at this point
+      assertFalse(cmdRes.contains("\tYes\t"));
+
+      // Get last line of the output which has the continuation token
+      String[] lines = cmdRes.split("\n");
+      String nextCmd = lines[lines.length - 1].trim();
+      String kw = "--start=";
+      String contToken =
+          nextCmd.substring(nextCmd.lastIndexOf(kw) + kw.length());
+
+      args = new String[] {"om", "lof",
+          "-id", omServiceId,
+          "-l", String.valueOf(pageSize),  // pagination
+          "-p", path,
+          "-s", contToken};
+      execute(ozoneAdminShell, args);
+      cmdRes = getStdOut();
+
+      // Should have retrieved the 2nd page only (2 keys)
+      for (int i = 0; i < pageSize - 1; i++) {
+        assertFalse(cmdRes.contains(keyPrefix + i));
+      }
+      // Note: key2 is shown in the continuation token prompt
+      for (int i = pageSize - 1; i < numKeys; i++) {
+        assertTrue(cmdRes.contains(keyPrefix + i));
+      }
+
+      // hsync last key
+      streams[numKeys - 1].hsync();
+      // Wait for flush
+      cluster.getOzoneManager().awaitDoubleBufferFlush();
+
+      execute(ozoneAdminShell, args);
+      cmdRes = getStdOut();
+
+      // Verify that only one key is hsync'ed
+      assertTrue(cmdRes.contains("\tYes\t"), "One key should be hsync'ed");
+      assertTrue(cmdRes.contains("\tNo\t"), "One key should not be hsync'ed");
+    } finally {
+      // Cleanup
+      for (int i = 0; i < numKeys; i++) {
+        streams[i].close();
+      }

Review Comment:
   If `streams[i]` assignment is moved into the `try` block as suggested above, we need to check for `null` element here.  Direct `close()` call can be replaced with `IOUtils.closeQuietly` for that.
   
   What's more, it can handle all streams in one call.
   
   ```suggestion
         IOUtils.closeQuietly(streams);
   ```



-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3187,14 +3192,96 @@ public ServiceInfoEx getServiceInfo() throws IOException {
   }
 
   @Override
-  public void transferLeadership(String newLeaderId)
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           int maxKeys,
+                                           String contToken)
       throws IOException {
-    final UserGroupInformation ugi = getRemoteUser();
-    if (!isAdmin(ugi)) {
-      throw new OMException(
-          "Only Ozone admins are allowed to transfer raft leadership.",
-          PERMISSION_DENIED);
+
+    metrics.incNumListOpenFiles();
+    checkAdminUserPrivilege("list open files.");
+
+    // Using final to make sure they are assigned once and only once in
+    // every branch.
+    final String dbOpenKeyPrefix, dbContTokenPrefix;
+    final String volumeName, bucketName;
+    final BucketLayout bucketLayout;
+
+    // Process path prefix
+    if (path == null || path.isEmpty() || path.equals(OM_KEY_PREFIX)) {
+      // path is root
+      dbOpenKeyPrefix = "";
+      volumeName = "";
+      bucketName = "";
+      // default to FSO's OpenFileTable. TODO: client option to pass OBS/LEGACY?
+      bucketLayout = BucketLayout.FILE_SYSTEM_OPTIMIZED;
+    } else {
+      // path is bucket or key prefix, break it down to volume, bucket, prefix
+      StringTokenizer tokenizer = new StringTokenizer(path, OM_KEY_PREFIX);
+      // Validate path to avoid NoSuchElementException
+      if (tokenizer.countTokens() < 2) {
+        metrics.incNumListOpenFilesFails();
+        throw new OMException("Invalid path: " + path + ". " +
+            "Only root level or bucket level path is supported at this time",
+            INVALID_PATH);
+      }
+
+      volumeName = tokenizer.nextToken();
+      bucketName = tokenizer.nextToken();
+
+      OmBucketInfo bucketInfo;
+      try {
+        // as expected, getBucketInfo throws if volume or bucket does not exist
+        bucketInfo = getBucketInfo(volumeName, bucketName);
+      } catch (Exception ex) {
+        metrics.incNumListOpenFilesFails();
+        throw ex;

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-8830. [hsync] Add CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3186,6 +3192,132 @@ public ServiceInfoEx getServiceInfo() throws IOException {
     return serviceInfo.provide();
   }
 
+  @Override
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           long maxKeys,
+                                           String contToken)
+      throws IOException {
+
+//    metrics.incListOpenFiles();  // TODO: Do we want a counter for this?

Review Comment:
   Donee



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3186,6 +3192,132 @@ public ServiceInfoEx getServiceInfo() throws IOException {
     return serviceInfo.provide();
   }
 
+  @Override
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           long maxKeys,
+                                           String contToken)
+      throws IOException {
+
+//    metrics.incListOpenFiles();  // TODO: Do we want a counter for this?

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-8830. [hsync] Add CLI to list open files [ozone]

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


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om listopenfiles command.
+ */
+@CommandLine.Command(
+    name = "listopenfiles",
+    aliases = {"listopenkeys", "lof", "lok"},
+    description = "Lists open keys in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},
+      description = "Ozone Manager Host. If OM HA is enabled, use -id instead. "
+          + "If insists on using -host with OM HA, this must point directly "
+          + "to the leader OM. "
+          + "This option is required when -id is not provided or "
+          + "when HA is not enabled."
+  )
+  private String omHost;
+
+  @CommandLine.Option(names = { "--json" },
+      defaultValue = "false",
+      description = "Format output as JSON")
+  private boolean json;
+
+  @CommandLine.Option(
+      names = {"-p", "--path"},
+      description = "Show only open files under this path.",
+      defaultValue = "/",
+      hidden = false
+  )
+  private String path;
+
+  @CommandLine.Option(
+      names = {"-n"},
+      description = "Numerical limit of open files/keys to return.",
+      defaultValue = "1000",
+      hidden = false
+  )
+  private long count;
+
+  @CommandLine.Option(
+      names = {"-t", "--token"},  // TODO: Conform to Ozone CLI convention
+      description = "Previous/last file/key path as the continuation token.",

Review Comment:
   Users of this command won't have to remember the usage of this option ideally, as I have added `getCmdForNextBatch()` to prompt the command to fetch the next batch. User could just copy, paste and run.
   
   Markdown doc to be added later.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om listopenfiles command.
+ */
+@CommandLine.Command(
+    name = "listopenfiles",
+    aliases = {"listopenkeys", "lof", "lok"},
+    description = "Lists open keys in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},
+      description = "Ozone Manager Host. If OM HA is enabled, use -id instead. "
+          + "If insists on using -host with OM HA, this must point directly "
+          + "to the leader OM. "
+          + "This option is required when -id is not provided or "
+          + "when HA is not enabled."
+  )
+  private String omHost;
+
+  @CommandLine.Option(names = { "--json" },
+      defaultValue = "false",
+      description = "Format output as JSON")
+  private boolean json;
+
+  @CommandLine.Option(
+      names = {"-p", "--path"},
+      description = "Show only open files under this path.",
+      defaultValue = "/",
+      hidden = false
+  )
+  private String path;
+
+  @CommandLine.Option(
+      names = {"-n"},
+      description = "Numerical limit of open files/keys to return.",
+      defaultValue = "1000",
+      hidden = false
+  )
+  private long count;
+
+  @CommandLine.Option(
+      names = {"-t", "--token"},  // TODO: Conform to Ozone CLI convention
+      description = "Previous/last file/key path as the continuation token.",

Review Comment:
   Users of this command won't have to remember the usage of this option ideally, as I have added `getCmdForNextBatch()` that prints the command to fetch the next batch. User could just copy, paste and run.
   
   Markdown doc to be added later.



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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

   You can check out OmMetadataManagerImpl.getExpiredOpenKeys()   https://github.com/apache/ozone/blob/f4a3b0e64c0cb3f2debaded68dcc0b7573602837/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L1808
   to see how we tell a key has hsync or not. You shouldn't need to walk through the keyTable.


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om list-open-files command.
+ */
+@CommandLine.Command(
+    name = "list-open-files",
+    aliases = {"list-open-keys", "lof", "lok"},
+    description = "Lists open files (keys) in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},

Review Comment:
   Yup I did realized the naming convention is not conformant when I copied this chunk from `UpdateRangerSubcommand`. Thought we might one day do a batch correction on those?
   
   Let me remove the short hand here at least.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om list-open-files command.
+ */
+@CommandLine.Command(
+    name = "list-open-files",
+    aliases = {"list-open-keys", "lof", "lok"},
+    description = "Lists open files (keys) in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},

Review Comment:
   Yup I did realize the naming convention is not conformant when I copied this chunk from `UpdateRangerSubcommand`. Thought we might one day do a batch correction on those?
   
   Let me remove the short hand here at least.



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3186,6 +3192,132 @@ public ServiceInfoEx getServiceInfo() throws IOException {
     return serviceInfo.provide();
   }
 
+  @Override
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           long maxKeys,
+                                           String contToken)
+      throws IOException {
+
+//    metrics.incListOpenFiles();  // TODO: Do we want a counter for this?
+
+    final UserGroupInformation ugi = getRemoteUser();
+    if (!isAdmin(ugi)) {
+      final OMException omEx = new OMException(
+          "Only Ozone admins are allowed to list open files.",
+          PERMISSION_DENIED);
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(
+          OMAction.LIST_OPEN_FILES, new LinkedHashMap<>(), omEx));
+      throw omEx;
+    }
+
+    final String dbOpenKeyPrefix;
+    final Table<String, OmKeyInfo> openKeyTable;
+    if (path == null || path.isEmpty() || path.equals(OM_KEY_PREFIX)) {
+      // path is root
+      dbOpenKeyPrefix = "";
+      // default to FSO. TODO: Add client option to pass OBS/LEGACY?
+      openKeyTable =
+          metadataManager.getOpenKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED);

Review Comment:
   I think it depends on whether or not we want to export only keys visible in the namespace. If so, then this semantics is only useful for FSO.  If not, there could be use cases where it makes sense for legacy/OBS



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om listopenfiles command.
+ */
+@CommandLine.Command(
+    name = "listopenfiles",
+    aliases = {"listopenkeys", "lof", "lok"},
+    description = "Lists open keys in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},
+      description = "Ozone Manager Host. If OM HA is enabled, use -id instead. "
+          + "If insists on using -host with OM HA, this must point directly "
+          + "to the leader OM. "
+          + "This option is required when -id is not provided or "
+          + "when HA is not enabled."
+  )
+  private String omHost;
+
+  @CommandLine.Option(names = { "--json" },
+      defaultValue = "false",
+      description = "Format output as JSON")
+  private boolean json;
+
+  @CommandLine.Option(
+      names = {"-p", "--path"},
+      description = "Show only open files under this path.",
+      defaultValue = "/",
+      hidden = false
+  )
+  private String path;
+
+  @CommandLine.Option(
+      names = {"-n"},
+      description = "Numerical limit of open files/keys to return.",
+      defaultValue = "1000",
+      hidden = false
+  )
+  private long count;
+
+  @CommandLine.Option(
+      names = {"-t", "--token"},  // TODO: Conform to Ozone CLI convention
+      description = "Previous/last file/key path as the continuation token.",

Review Comment:
   Add an example?



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java:
##########
@@ -549,6 +551,44 @@ public void testOzoneAdminCmdList() throws UnsupportedEncodingException {
     execute(ozoneAdminShell, args);
   }
 
+  @Test
+  public void testOzoneAdminCmdListOpenFiles()
+      throws IOException, InterruptedException {
+    final String volumeName = "volumelof";
+
+    final String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + omServiceId;
+//    final String hostPrefix = OZONE_OFS_URI_SCHEME + "://localhost:9862";
+    OzoneConfiguration clientConf =
+        getClientConfForOFS(hostPrefix, cluster.getConf());
+//        getClientConfForOFS(hostPrefix, new OzoneConfiguration());
+    FileSystem fs = FileSystem.get(clientConf);
+
+    String dir1 = hostPrefix + OM_KEY_PREFIX + volumeName + OM_KEY_PREFIX +
+        "buck1" + OM_KEY_PREFIX + "dir1";
+    assertTrue(fs.mkdirs(new Path(dir1)));
+    String key1 = dir1 + OM_KEY_PREFIX + "key1";
+
+    // Create key1
+    try (FSDataOutputStream out = fs.create(new Path(key1))) {
+      out.write(1);
+      // Wait for flush
+//      cluster.getOzoneManager().awaitDoubleBufferFlush();
+
+      // Hold the file open, run listopenfiles
+      String[] args = new String[] {"om", "listopenfiles", "-id", omServiceId,
+          "-p", "/volumelof/buck1"};
+      // TODO: Possible to make this work under volume level as well?
+      execute(ozoneAdminShell, args);

Review Comment:
   please run listopenfiles again after out.hsync()



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3186,6 +3192,132 @@ public ServiceInfoEx getServiceInfo() throws IOException {
     return serviceInfo.provide();
   }
 
+  @Override
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           long maxKeys,
+                                           String contToken)
+      throws IOException {
+
+//    metrics.incListOpenFiles();  // TODO: Do we want a counter for this?
+
+    final UserGroupInformation ugi = getRemoteUser();
+    if (!isAdmin(ugi)) {
+      final OMException omEx = new OMException(
+          "Only Ozone admins are allowed to list open files.",
+          PERMISSION_DENIED);
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(
+          OMAction.LIST_OPEN_FILES, new LinkedHashMap<>(), omEx));
+      throw omEx;
+    }
+
+    final String dbOpenKeyPrefix;
+    final Table<String, OmKeyInfo> openKeyTable;
+    if (path == null || path.isEmpty() || path.equals(OM_KEY_PREFIX)) {

Review Comment:
   IMO it is more flexible (in terms of not having to break protobuf msg) to parse it on the server side.



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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

   Thanks @errose28 for the comment.
   
   > * I think we should follow standard kebab-case conventions for commands going forwards.
   > * We should avoid combining multiple words in one command since it makes it less flexible for future use. Instead of `ozone admin om listopenfiles` we could do something like `ozone admin om open-files ls` in case we need to provide more open file operations in the future.
   
   +1 this. But in this way we'd also need to rename all other commands (e.g. `getserviceroles`, `cancelprepare`). I'd like @jojochuang and @adoroszlai 's opinions as well.
   
   > * I'm not sure whether we want to use `open-files` or `open-keys` as the term here. Even though this is being implemented for hsync which is a filesystem operation, the command is under `ozone admin` and therefore outside of ofs. In this area we usually refer to objects as `keys` like `ozone admin key list`.
   
   This is basically a command that enumerates `OpenKeyTable` (or `OpenFileTable`). It is not just limited to hsync'ed keys but all open keys. I should remove `[hsync]` from the jira title.
   
   I do like the idea of `ozone admin key list --open`.
   
   > * Related to the above, can you clarify the scope of this command? Is it operating on only open files, open keys as well? Does it work for all bucket layouts? Do you pass it a bucket name to limit the scope?
   
   Yup it is supposed to work with FSO/OBS/LEGACY. Path to a bucket (or just a key prefix) can be optionally passed to limit the scope. But without it (or when passing root as the path) it just iterates over two `OpenKey(File)Table`s.
   
   > * Can you provide an example of expected CLI parameters and output in the PR description? This makes it easier for others to quickly give feedback on the UI portion without reviewing implementation details.
   
   I planned to. But it is not finalized yet so I will do that later. Plus I will add it to the markdown doc as well.
   
   > * Since the output will likely be verbose, I think it makes sense to only have this command print json, similar to `ozone sh key list`. In the current draft there is a --json flag that defaults to false.
   
   I think we need both. 


-- 
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-8830. Add admin CLI to list open files [ozone]

Posted by "jojochuang (via GitHub)" <gi...@apache.org>.
jojochuang merged PR #5920:
URL: https://github.com/apache/ozone/pull/5920


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java:
##########
@@ -549,6 +555,134 @@ public void testOzoneAdminCmdList() throws UnsupportedEncodingException {
     execute(ozoneAdminShell, args);
   }
 
+  @Test
+  public void testOzoneAdminCmdListOpenFiles()
+      throws IOException, InterruptedException {
+
+    OzoneConfiguration conf = cluster.getConf();
+    final String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + omServiceId;
+
+    OzoneConfiguration clientConf = getClientConfForOFS(hostPrefix, conf);
+    clientConf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
+    FileSystem fs = FileSystem.get(clientConf);
+
+    assertNotEquals(fs.getConf().get(OZONE_FS_HSYNC_ENABLED),
+        "false", OZONE_FS_HSYNC_ENABLED + " is set to false " +
+            "by external force. Must be true to allow hsync to function");
+
+    final String volumeName = "volume-lof";
+    final String bucketName = "buck1";
+
+    String dir1 = hostPrefix + OM_KEY_PREFIX + volumeName + OM_KEY_PREFIX +
+        bucketName + OM_KEY_PREFIX + "dir1";
+    // Create volume, bucket, dir
+    assertTrue(fs.mkdirs(new Path(dir1)));
+    String keyPrefix = OM_KEY_PREFIX + "key";
+
+    final int numKeys = 5;
+    String[] keys = new String[numKeys];
+
+    for (int i = 0; i < numKeys; i++) {
+      keys[i] = dir1 + keyPrefix + i;
+    }
+
+    int pageSize = 3;
+
+    FSDataOutputStream[] streams = new FSDataOutputStream[numKeys];
+    // Create multiple keys and hold them open
+    for (int i = 0; i < numKeys; i++) {
+      streams[i] = fs.create(new Path(keys[i]));
+      streams[i].write(1);
+    }
+
+    String path = "/" +  volumeName + "/" + bucketName;
+    try {
+      // Wait for DB flush
+      cluster.getOzoneManager().awaitDoubleBufferFlush();
+
+      String[] args = new String[] {"om", "lof",
+          "-id", omServiceId,
+          "-l", String.valueOf(numKeys + 1),  // pagination
+          "-p", path};
+      // Run listopenfiles
+      execute(ozoneAdminShell, args);
+      String cmdRes = getStdOut();
+      // Should have retrieved all 5 open keys
+      for (int i = 0; i < numKeys; i++) {
+        assertTrue(cmdRes.contains(keyPrefix + i));
+      }
+
+      // Try pagination
+      args = new String[] {"om", "lof",
+          "-id", omServiceId,
+          "-l", String.valueOf(pageSize),  // pagination
+          "-p", path};
+      execute(ozoneAdminShell, args);
+      cmdRes = getStdOut();
+
+      // Should have retrieved the 1st page only (3 keys)
+      for (int i = 0; i < pageSize; i++) {
+        assertTrue(cmdRes.contains(keyPrefix + i));
+      }
+      for (int i = pageSize; i < numKeys; i++) {
+        assertFalse(cmdRes.contains(keyPrefix + i));
+      }
+      // No hsync'ed file/key at this point
+      assertFalse(cmdRes.contains("\tYes\t"));
+
+      // Get last line of the output which has the continuation token
+      String[] lines = cmdRes.split("\n");
+      String nextCmd = lines[lines.length - 1].trim();
+      String kw = "--start=";
+      String contToken =
+          nextCmd.substring(nextCmd.lastIndexOf(kw) + kw.length());
+
+      args = new String[] {"om", "lof",
+          "-id", omServiceId,
+          "-l", String.valueOf(pageSize),  // pagination
+          "-p", path,
+          "-s", contToken};
+      execute(ozoneAdminShell, args);
+      cmdRes = getStdOut();
+
+      // Should have retrieved the 2nd page only (2 keys)
+      for (int i = 0; i < pageSize - 1; i++) {
+        assertFalse(cmdRes.contains(keyPrefix + i));
+      }
+      // Note: key2 is shown in the continuation token prompt
+      for (int i = pageSize - 1; i < numKeys; i++) {
+        assertTrue(cmdRes.contains(keyPrefix + i));
+      }
+
+      // hsync last key
+      streams[numKeys - 1].hsync();
+      // Wait for flush
+      cluster.getOzoneManager().awaitDoubleBufferFlush();
+
+      execute(ozoneAdminShell, args);
+      cmdRes = getStdOut();
+
+      // Verify that only one key is hsync'ed
+      assertTrue(cmdRes.contains("\tYes\t"), "One key should be hsync'ed");
+      assertTrue(cmdRes.contains("\tNo\t"), "One key should not be hsync'ed");
+    } finally {
+      // Cleanup
+      for (int i = 0; i < numKeys; i++) {
+        streams[i].close();
+      }

Review Comment:
   nice!



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3186,6 +3192,132 @@ public ServiceInfoEx getServiceInfo() throws IOException {
     return serviceInfo.provide();
   }
 
+  @Override
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           long maxKeys,
+                                           String contToken)
+      throws IOException {
+
+//    metrics.incListOpenFiles();  // TODO: Do we want a counter for this?
+
+    final UserGroupInformation ugi = getRemoteUser();
+    if (!isAdmin(ugi)) {
+      final OMException omEx = new OMException(
+          "Only Ozone admins are allowed to list open files.",
+          PERMISSION_DENIED);
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(
+          OMAction.LIST_OPEN_FILES, new LinkedHashMap<>(), omEx));
+      throw omEx;
+    }
+
+    final String dbOpenKeyPrefix;
+    final Table<String, OmKeyInfo> openKeyTable;
+    if (path == null || path.isEmpty() || path.equals(OM_KEY_PREFIX)) {

Review Comment:
   Should the parsing of path be done at client side or OM side? Looking at code it is usually done at server side.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3186,6 +3192,132 @@ public ServiceInfoEx getServiceInfo() throws IOException {
     return serviceInfo.provide();
   }
 
+  @Override
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           long maxKeys,
+                                           String contToken)
+      throws IOException {
+
+//    metrics.incListOpenFiles();  // TODO: Do we want a counter for this?

Review Comment:
   yeah we should because this call may be potentially one of those that destroy OM performance.



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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

   Just a few comments on the CLI portion of this change:
   
   - I think we should follow standard kebab-case conventions for commands going forwards.
   -  We should avoid combining multiple words in one command since it makes it less flexible for future use. Instead of `ozone admin om listopenfiles` we could do something like `ozone admin om open-files ls` in case we need to provide more open file operations in the future. 
   - I'm not sure whether we want to use `open-files` or `open-keys` as the term here. Even though this is being implemented for hsync which is a filesystem operation, the command is under `ozone admin` and therefore outside of ofs. In this area we usually refer to objects as `keys` like `ozone admin key list`.
   - Related to the above, can you clarify the scope of this command? Is it operating on only open files, open keys as well? Does it work for all bucket layouts? Do you pass it a bucket name to limit the scope?
   - Can you provide an example of expected CLI parameters and output in the PR description? This makes it easier for others to quickly give feedback on the UI portion without reviewing implementation details.
   - Since the output will likely be verbose, I think it makes sense to only have this command print json, similar to `ozone sh key list`. In the current draft there is a --json flag that defaults to false.


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1523,6 +1529,24 @@ message CancelPrepareResponse {
 
 }
 
+message ListOpenFilesRequest {
+  optional string path = 1;

Review Comment:
   It is for proto3 migration at some point. Same idea as this:
   
   https://github.com/apache/ozone/pull/3824#discussion_r1011376445



-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -3187,14 +3192,96 @@ public ServiceInfoEx getServiceInfo() throws IOException {
   }
 
   @Override
-  public void transferLeadership(String newLeaderId)
+  public ListOpenFilesResult listOpenFiles(String path,
+                                           int maxKeys,
+                                           String contToken)
       throws IOException {
-    final UserGroupInformation ugi = getRemoteUser();
-    if (!isAdmin(ugi)) {
-      throw new OMException(
-          "Only Ozone admins are allowed to transfer raft leadership.",
-          PERMISSION_DENIED);
+
+    metrics.incNumListOpenFiles();
+    checkAdminUserPrivilege("list open files.");
+
+    // Using final to make sure they are assigned once and only once in
+    // every branch.
+    final String dbOpenKeyPrefix, dbContTokenPrefix;
+    final String volumeName, bucketName;
+    final BucketLayout bucketLayout;
+
+    // Process path prefix
+    if (path == null || path.isEmpty() || path.equals(OM_KEY_PREFIX)) {
+      // path is root
+      dbOpenKeyPrefix = "";
+      volumeName = "";
+      bucketName = "";
+      // default to FSO's OpenFileTable. TODO: client option to pass OBS/LEGACY?
+      bucketLayout = BucketLayout.FILE_SYSTEM_OPTIMIZED;
+    } else {
+      // path is bucket or key prefix, break it down to volume, bucket, prefix
+      StringTokenizer tokenizer = new StringTokenizer(path, OM_KEY_PREFIX);
+      // Validate path to avoid NoSuchElementException
+      if (tokenizer.countTokens() < 2) {
+        metrics.incNumListOpenFilesFails();
+        throw new OMException("Invalid path: " + path + ". " +
+            "Only root level or bucket level path is supported at this time",
+            INVALID_PATH);
+      }
+
+      volumeName = tokenizer.nextToken();
+      bucketName = tokenizer.nextToken();
+
+      OmBucketInfo bucketInfo;
+      try {
+        // as expected, getBucketInfo throws if volume or bucket does not exist
+        bucketInfo = getBucketInfo(volumeName, bucketName);
+      } catch (Exception ex) {
+        metrics.incNumListOpenFilesFails();
+        throw ex;

Review Comment:
   consider wrapping it in an OMException.



-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om list-open-files command.
+ */
+@CommandLine.Command(
+    name = "list-open-files",
+    aliases = {"list-open-keys", "lof", "lok"},
+    description = "Lists open files (keys) in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},

Review Comment:
   Thanks.  Yes, we should fix them sometime.  The problem is that existing commands/options need deprecation etc.  That's why I try to prevent any new commands introduce options that should be deprecated right away...



-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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

   > Looks like this would list both visible and invisible (keys that are written but no hsync() is called at it) keys. If that's the intend it would be good to distinguish them in the output.
   
   That makes sense. Added in the latest revision. Thanks for taking a look at the draft.


-- 
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-8830. [hsync] Add CLI to list open files [ozone]

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


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.admin.om;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Handler of ozone admin om listopenfiles command.
+ */
+@CommandLine.Command(
+    name = "listopenfiles",
+    aliases = {"listopenkeys", "lof", "lok"},
+    description = "Lists open keys in Ozone Manager.",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class
+)
+public class ListOpenFilesSubCommand implements Callable<Void> {
+
+  @CommandLine.ParentCommand
+  private OMAdmin parent;
+
+  @CommandLine.Option(
+      names = {"-id", "--service-id"},
+      description = "Ozone Manager Service ID",
+      required = false
+  )
+  private String omServiceId;
+
+  @CommandLine.Option(
+      names = {"-host", "--service-host"},
+      description = "Ozone Manager Host. If OM HA is enabled, use -id instead. "
+          + "If insists on using -host with OM HA, this must point directly "
+          + "to the leader OM. "
+          + "This option is required when -id is not provided or "
+          + "when HA is not enabled."
+  )
+  private String omHost;
+
+  @CommandLine.Option(names = { "--json" },
+      defaultValue = "false",
+      description = "Format output as JSON")
+  private boolean json;
+
+  @CommandLine.Option(
+      names = {"-p", "--path"},
+      description = "Show only open files under this path.",
+      defaultValue = "/",
+      hidden = false
+  )
+  private String path;
+
+  @CommandLine.Option(
+      names = {"-n"},
+      description = "Numerical limit of open files/keys to return.",
+      defaultValue = "1000",
+      hidden = false
+  )
+  private long count;
+
+  @CommandLine.Option(
+      names = {"-t", "--token"},  // TODO: Conform to Ozone CLI convention
+      description = "Previous/last file/key path as the continuation token.",

Review Comment:
   Users of this command won't have to remember the usage of this option's usage ideally, as I have added `getCmdForNextBatch()` to prompt the command to fetch the next batch. User could just copy, paste and run.
   
   Markdown doc to be added later.



-- 
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-8830. Add admin CLI to list open files [ozone]

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

   > You can check out OmMetadataManagerImpl.getExpiredOpenKeys()
   > 
   > https://github.com/apache/ozone/blob/f4a3b0e64c0cb3f2debaded68dcc0b7573602837/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L1808
   > 
   > 
   > to see how we tell a key has hsync or not. You shouldn't need to walk through the keyTable.
   
   As I can see it is using the same approach as mine to check whether a key is hsync'ed or not. AFAIK the metadata tag is the only way to do it at the moment. Plus `getExpiredOpenKeys` looks to be iterating over the entire `KeyTable`/`FileTable`.
   
   Again, the problem is that `OpenKeyTable` doen't have this metadata tag at all. If we can add that it will be much more efficient (both for `getExpiredOpenKeys` and this PR).


-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java:
##########
@@ -428,10 +430,18 @@ public void incNumListS3Buckets() {
   }
 
   public void incNumListS3BucketsFails() {
-    numBucketOps.incr();

Review Comment:
   Unrelated change?



-- 
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-8830. Add admin CLI to list open files [ozone]

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1523,6 +1529,24 @@ message CancelPrepareResponse {
 
 }
 
+message ListOpenFilesRequest {
+  optional string path = 1;

Review Comment:
   is it really optional ...? As far as I can tell the OM code expects these three parameter all the time.



-- 
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