You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2023/01/17 07:12:05 UTC

[GitHub] [ozone] ArafatKhan2198 opened a new pull request, #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   ## What changes were proposed in this pull request?
   
   The container Endpoint and ContainerKeyMapperTask have been updated to support both legacy and file-system optimized (FSO) buckets. Previously, only the KeyTable for legacy buckets was being referenced, but now both the KeyTable and FileTable will be utilised to fetch metadata
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5463?page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel&focusedCommentId=17675819#comment-17675819
   
   ## How was this patch tested?
   
   Manually tested out the API
   <img width="392" alt="image" src="https://user-images.githubusercontent.com/98023601/212832742-32328797-b80f-4ea7-b064-c1c57e039b92.png">
   
   <img width="551" alt="image" src="https://user-images.githubusercontent.com/98023601/212832806-1f79d83f-3e51-4113-959d-f3b559e38431.png">
   
   
   


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


[GitHub] [ozone] aryangupta1998 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

Posted by GitBox <gi...@apache.org>.
aryangupta1998 commented on code in PR #4182:
URL: https://github.com/apache/ozone/pull/4182#discussion_r1081656654


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -160,8 +160,19 @@ public Response getKeysForContainer(
         // Directly calling get() on the Key table instead of iterating since
         // only full keys are supported now. When we change to using a prefix
         // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        // The first expression will try to get the OmKeyInfo object by calling
+        // getSkipCache on the BucketLayout.LEGACY table, passing in the
+        // keyPrefix as the key. If this expression returns null, meaning
+        // the key was not found in the LEGACY table, the second expression
+        // will be evaluated and the OmKeyInfo will be obtained from the
+        // BucketLayout.FILE_SYSTEM_OPTIMIZED table.

Review Comment:
   Can we have a simple comment like, 
   `Directly calling getSkipCache() on the table based on the bucket layout(file table for FSO buckets and key table for others) instead of iterating since only full keys are supported now.`



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)

Review Comment:
   Thank you for bringing this to my attention. I understand that directory keys won't be considered in the case of FSO, but as you rightly pointed out, since the endpoint name is "getKeysForContainers" and we are concerned only with the keys that carry data size, I believe these change is still relevant and helpful for debugging purposes.
   
   



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)

Review Comment:
   Hi @GeorgeJahad,
   
   I would like to confirm if the current approach for handling FSO paths is correct. Once we reach a consensus, I will proceed with writing the UT for it.



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)
+      throws IOException {
+    if (StringUtils.isEmpty(prevKeyPrefix)) {
+      return "";
+    }
+    // Normalize the path to remove duplicate slashes & make it easier to parse.
+    String normalizedPath = normalizePath(prevKeyPrefix);
+    String[] names = parseRequestPath(normalizedPath);
+
+    if (names.length < 3) {
+      LOG.error("Invalid path: {} path should contain a directory",
+          prevKeyPrefix);
+      return prevKeyPrefix;
+    }
+    // Extract the volume, bucket, and key names from the path.
+    String volumeName = names[0];
+    String bucketName = names[1];
+    String keyName = names[names.length - 1];
+
+    // Get the bucket handler for the given volume and bucket.
+    BucketHandler handler =
+        getBucketHandler(reconNamespaceSummaryManager, omMetadataManager,
+            reconSCM, volumeName, bucketName);
+
+    // Only keyPaths for FSO bucket need to be converted to
+    // their respective objectId's
+    if (handler.getBucketLayout() != BucketLayout.FILE_SYSTEM_OPTIMIZED) {
+      return prevKeyPrefix;
+    }
+
+    // Get the object IDs for the bucket, volume, and parent directory.
+    long bucketId, volumeId, parentId;
+    bucketId = handler.getBucketObjectId(names);
+    volumeId = handler.getVolumeObjectId(names);
+    parentId = handler.getDirObjectId(names, names.length - 1);

Review Comment:
   Good suggestion, Ill make the changes to handle these exceptions.



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


[GitHub] [ozone] ArafatKhan2198 commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   @GeorgeJahad @jojochuang can you please take a look!


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -90,18 +91,24 @@ public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
       reconContainerMetadataManager
               .reinitWithNewContainerDataFromOm(new HashMap<>());
 
-      Table<String, OmKeyInfo> omKeyInfoTable =
-          omMetadataManager.getKeyTable(getBucketLayout());
-      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
-               keyIter = omKeyInfoTable.iterator()) {
-        while (keyIter.hasNext()) {
-          Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
-          OmKeyInfo omKeyInfo = kv.getValue();
-          handlePutOMKeyEvent(kv.getKey(), omKeyInfo, containerKeyMap,
-              containerKeyCountMap, deletedKeyCountList);
-          omKeyCount++;
+      // loop over both key table and file table
+      for (BucketLayout layout : Arrays.asList(BucketLayout.DEFAULT,

Review Comment:
   Actually, I think LEGACY is much better than DEFAULT.  I believe the DEFAULT has changed a number of times already and when it does, that may invalidate that are using it as a synomym for LEGACY



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -229,13 +239,63 @@ public void setUp() throws Exception {
     OMMetadataManager omMetadataManagerMock = mock(OMMetadataManager.class);
     Table tableMock = mock(Table.class);
     when(tableMock.getName()).thenReturn("KeyTable");
-    when(omMetadataManagerMock.getKeyTable(getBucketLayout()))
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.LEGACY))
         .thenReturn(tableMock);
-    ContainerKeyMapperTask containerKeyMapperTask  =
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED))
+        .thenReturn(tableMock);

Review Comment:
   this mock returns keytable for 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


[GitHub] [ozone] jojochuang commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   Actually, getKeyTable() returns fileTable rocksdb column family if it's FSO and returns keyTable otherwise, so no need to distinguish between OBJECT_STORE or LEGACY.
   
   But good point on the test case. We need 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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -162,6 +176,7 @@ public Response getKeysForContainer(
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     long totalCount;
     try {
+      prevKeyPrefix = buildObjectPathForFileSystemBucket(prevKeyPrefix);

Review Comment:
   Thank you for your message, Sumit. However, in the code, we will iterate through the `KeyTable` and `FileTable` for each `containerKeyPrefix` present in the container. Each container contains a mixture of keys from both the KeyTable and the FileTable. Therefore, the `correctPathForLayout` method will only be called once at the beginning, and setting the layout to either `FSO` or `LEGACY` at the start will not display keys from the other table.
   
   



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -229,13 +239,63 @@ public void setUp() throws Exception {
     OMMetadataManager omMetadataManagerMock = mock(OMMetadataManager.class);
     Table tableMock = mock(Table.class);
     when(tableMock.getName()).thenReturn("KeyTable");
-    when(omMetadataManagerMock.getKeyTable(getBucketLayout()))
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.LEGACY))
         .thenReturn(tableMock);
-    ContainerKeyMapperTask containerKeyMapperTask  =
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED))
+        .thenReturn(tableMock);

Review Comment:
   this mock returns keytable for fso instead of filetable



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


[GitHub] [ozone] sumitagrawl commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)
+      throws IOException {
+    if (StringUtils.isEmpty(prevKeyPrefix)) {
+      return "";
+    }
+    // Normalize the path to remove duplicate slashes & make it easier to parse.
+    String normalizedPath = normalizePath(prevKeyPrefix);
+    String[] names = parseRequestPath(normalizedPath);
+
+    if (names.length < 3) {
+      LOG.error("Invalid path: {} path should contain a directory",
+          prevKeyPrefix);
+      return prevKeyPrefix;
+    }
+    // Extract the volume, bucket, and key names from the path.
+    String volumeName = names[0];
+    String bucketName = names[1];
+    String keyName = names[names.length - 1];
+
+    // Get the bucket handler for the given volume and bucket.
+    BucketHandler handler =
+        getBucketHandler(reconNamespaceSummaryManager, omMetadataManager,
+            reconSCM, volumeName, bucketName);
+
+    // Only keyPaths for FSO bucket need to be converted to
+    // their respective objectId's
+    if (handler.getBucketLayout() != BucketLayout.FILE_SYSTEM_OPTIMIZED) {
+      return prevKeyPrefix;
+    }
+
+    // Get the object IDs for the bucket, volume, and parent directory.
+    long bucketId, volumeId, parentId;
+    bucketId = handler.getBucketObjectId(names);
+    volumeId = handler.getVolumeObjectId(names);
+    parentId = handler.getDirObjectId(names, names.length - 1);
+
+    // Build the object path by concatenating the object IDs with the key name.
+    StringBuilder objectPathBuilder = new StringBuilder();
+    objectPathBuilder.append(OM_KEY_PREFIX).append(volumeId)
+        .append(OM_KEY_PREFIX).append(bucketId)
+        .append(OM_KEY_PREFIX).append(parentId)
+        .append(OM_KEY_PREFIX).append(keyName);
+
+    return objectPathBuilder.toString();
   }
+
+  /**
+   * Normalizes a key path by adding the OM_KEY_PREFIX at the beginning
+   * and removing duplicate slashes.
+   *
+   * @param path the key path
+   * @return the normalized key path
+   */
+  public static String normalizePath(String path) {
+    return OM_KEY_PREFIX + OmUtils.normalizeKey(path, false);

Review Comment:
   input with "/" is also an input for path (optional), check having additional slash have a problem or not



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -162,6 +176,7 @@ public Response getKeysForContainer(
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     long totalCount;
     try {
+      prevKeyPrefix = buildObjectPathForFileSystemBucket(prevKeyPrefix);

Review Comment:
   We are already getting FSO / Legacy inside, same can be retrieved outside, no need iterate for those table.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)
+      throws IOException {
+    if (StringUtils.isEmpty(prevKeyPrefix)) {
+      return "";
+    }
+    // Normalize the path to remove duplicate slashes & make it easier to parse.
+    String normalizedPath = normalizePath(prevKeyPrefix);
+    String[] names = parseRequestPath(normalizedPath);
+
+    if (names.length < 3) {
+      LOG.error("Invalid path: {} path should contain a directory",
+          prevKeyPrefix);
+      return prevKeyPrefix;
+    }
+    // Extract the volume, bucket, and key names from the path.
+    String volumeName = names[0];
+    String bucketName = names[1];
+    String keyName = names[names.length - 1];
+
+    // Get the bucket handler for the given volume and bucket.
+    BucketHandler handler =
+        getBucketHandler(reconNamespaceSummaryManager, omMetadataManager,
+            reconSCM, volumeName, bucketName);
+
+    // Only keyPaths for FSO bucket need to be converted to
+    // their respective objectId's
+    if (handler.getBucketLayout() != BucketLayout.FILE_SYSTEM_OPTIMIZED) {
+      return prevKeyPrefix;
+    }
+
+    // Get the object IDs for the bucket, volume, and parent directory.
+    long bucketId, volumeId, parentId;
+    bucketId = handler.getBucketObjectId(names);
+    volumeId = handler.getVolumeObjectId(names);
+    parentId = handler.getDirObjectId(names, names.length - 1);

Review Comment:
   name can have some invalid directory as user input, we need throw exception as not-exist and handle NPE



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


[GitHub] [ozone] sumitagrawl commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   IMO, pseudocode,
   1. Get all keys (FSO/OBS) with limit and prvKey as starting point
   2. For each Key,
   - get bucket Type (OBS/FSO)
   - If OBS, as old code, get KeyInfo and extract metadata
   - If FSO, generate key based on path
   -             use generated Key, get KeyInfo and extract metadata
   
   Few optimization, generate key based on path, can see avoid getting parentId again and again
   


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


[GitHub] [ozone] sadanand48 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)

Review Comment:
   Just pointing it out here that directory keys won't be considered in the case of FSO however I guess it should be okay as the endpoint name is "getKeysForContainers".  The change looks good otherwise.



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


[GitHub] [ozone] devmadhuu commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
             .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        if (omKeyInfo == null) {
+          omKeyInfo =
+              omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                  .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   Are we not supporting OBS buckets ? Have we tested with OBS buckets ?



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


[GitHub] [ozone] DaveTeng0 commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   Hey @ArafatKhan2198 ~ there were some new comments, please help take a look. 
    Thanks!


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


[GitHub] [ozone] GeorgeJahad commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   What about object store buckets? I know those are similar to legacy buckets but the way recon is coded they won't get handled, will they?  Is that a separate 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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
             .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        if (omKeyInfo == null) {
+          omKeyInfo =
+              omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)

Review Comment:
   Thank you @sumitagrawl  for pointing out the flaw in the PR submission. It would have been incomplete without it.
   The keys in fileTable and keyTable are stored slightly differently:
   ```
   FSO : /VolumeId/BucketId/ParentId/KeyName
   ObjectStore / Legacy : /VolumeName/BucketName/keyName
   ```
   To ensure that the prevkey parameter works correctly, we need to identify whether the path provided by the user is for FSO or OBS/Legacy. If the path is for OBS/Legacy, then the path should be fine because the OBS/Legacy format contains the volumeName and BucketName in its path.
   
   However, if the path is for FSO, we need to find the objectId for the respective volumeName, bucketName, and parentDirectoryName, and then use it to search the fileTable. The fileTable has the following structure:
   
   ```
   fileTable: /volumeId/bucketId/parentId/fileName -> KeyInfo
   ```
   So I have added a helper method in ContainerEndpoint that coverts the user path for FSO keys given by the user, to their respective objectID paths.
   



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


[GitHub] [ozone] ArafatKhan2198 commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   Therefore, to conclude, there is no need for any conversion from `object-ID` to vol/bucket Name. Our goal should be to ensure that both the FSO and OBS keys are stored in the ContainerKey table, as it serves as the source of information for this endpoint. For Example :- 
   
   <pre><code class="language-bash">ozone sh bucket create --layout FILE_SYSTEM_OPTIMIZED /s3v/fso-bucket
   ozone sh key put s3v/fso-bucket/key1-fso NOTICE.txt
   ozone sh key put s3v/fso-bucket/key2-fso NOTICE.txt
   
   ozone sh bucket create --layout OBJECT_STORE /s3v/obs-bucket
   ozone sh key put s3v/obs-bucket/key1-obs NOTICE.txt
   ozone sh key put s3v/obs-bucket/key2-obs NOTICE.txt
   
   ozone sh bucket create --layout LEGACY s3v/legacy-bucket
   ozone sh key put s3v/legacy-bucket/key1-legacy NOTICE.txt
   ozone sh key put s3v/legacy-bucket/key2-legacy NOTICE.txt
   </code></pre>
   
   The ContainerKey Table generated for this data would be the below table :- 
   In which the FSO key information would be displayed in the form of Object-ID's
   
   KeyPrefix | Container ID
   -- | --
   /-4611686018427388160/-9223372036854774528/-9223372036854774528/key1-fso | 1
   /s3v/obs-bucket/key2-obs | 1
   /-4611686018427388160/-9223372036854774528/-9223372036854774528/key2-fso | 2
   /s3v/legacy-bucket/key1-legacy | 2
   /s3v/legacy-bucket/key2-legacy | 3
   /s3v/obs-bucket/key1-obs | 3


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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
             .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        if (omKeyInfo == null) {
+          omKeyInfo =
+              omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                  .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   We do support OBS bucket for this also, I have manually tested the changes out as well!
   <img width="494" alt="image" src="https://user-images.githubusercontent.com/98023601/224228535-edc8775e-6b1b-4021-a9de-3f7a3cf501c6.png">
   



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


[GitHub] [ozone] GeorgeJahad commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   It would be good to add an fso test here as well: https://github.com/apache/ozone/blob/94598d1cf42fb4811f17fce4ddbfcd72cee00632/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java#L153
   


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -229,13 +239,63 @@ public void setUp() throws Exception {
     OMMetadataManager omMetadataManagerMock = mock(OMMetadataManager.class);
     Table tableMock = mock(Table.class);
     when(tableMock.getName()).thenReturn("KeyTable");
-    when(omMetadataManagerMock.getKeyTable(getBucketLayout()))
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.LEGACY))
         .thenReturn(tableMock);
-    ContainerKeyMapperTask containerKeyMapperTask  =
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED))
+        .thenReturn(tableMock);
+    reprocessContainerKeyMapper();
+  }
+
+  private void reprocessContainerKeyMapper() {
+    ContainerKeyMapperTask containerKeyMapperTask =
         new ContainerKeyMapperTask(reconContainerMetadataManager);
     containerKeyMapperTask.reprocess(reconOMMetadataManager);
   }
 
+  private void setUpMultiBlockKey() throws IOException {
+    OmKeyLocationInfoGroup locationInfoGroup =
+        getLocationInfoGroup1();
+
+    // add the multi-block key to Recon's OM
+    writeKeyToOm(reconOMMetadataManager,
+        FSO_KEY_NAME,
+        BUCKET_NAME,
+        VOLUME_NAME,
+        FILE_NAME,
+        KEY_LENGTH,
+        BLOCK_LENGTH,
+        KEY_SEQ_NO,
+        MODIFICATION_TIME,
+        Collections.singletonList(locationInfoGroup),
+        getBucketLayout(),

Review Comment:
   this layout returns default instead of 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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -157,11 +157,17 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+      // Directly calling getSkipCache() on the Key/FileTable table accordingly
+      // instead of iterating since only full keys are supported now. We will
+      // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+      // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
+            .getSkipCache(containerKeyPrefix.getKeyPrefix()) == null ?
+            omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                .getSkipCache(containerKeyPrefix.getKeyPrefix()) :
+            omMetadataManager.getKeyTable(BucketLayout.LEGACY)
+                .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   Thanks for the suggestion @dombizita I have made the changes, do take a look !!



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -162,6 +176,7 @@ public Response getKeysForContainer(
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     long totalCount;
     try {
+      prevKeyPrefix = buildObjectPathForFileSystemBucket(prevKeyPrefix);

Review Comment:
   Thank you for your message, @sumitagrawl . However, in the code, we will iterate through the `KeyTable` and `FileTable` for each `containerKeyPrefix` present in the container. Each container contains a mixture of keys from both the KeyTable and the FileTable. Therefore, the `correctPathForLayout` method will only be called once at the beginning, and setting the layout to either `FSO` or `LEGACY` at the start will not display keys from the other table.
   
   



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -162,6 +176,7 @@ public Response getKeysForContainer(
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     long totalCount;
     try {
+      prevKeyPrefix = correctPathForLayout(prevKeyPrefix);

Review Comment:
   Thanks for the analysis sumit, you were absolutely right on this one this one the prevKey Prefix was used for pagination purposes. 



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


[GitHub] [ozone] ChenSammi merged pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


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


[GitHub] [ozone] dombizita commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -157,11 +157,17 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+      // Directly calling getSkipCache() on the Key/FileTable table accordingly
+      // instead of iterating since only full keys are supported now. We will
+      // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+      // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
+            .getSkipCache(containerKeyPrefix.getKeyPrefix()) == null ?
+            omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                .getSkipCache(containerKeyPrefix.getKeyPrefix()) :
+            omMetadataManager.getKeyTable(BucketLayout.LEGACY)
+                .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   maybe we could avoid calling `omMetadataManager.getKeyTable(BucketLayout.LEGACY).getSkipCache(containerKeyPrefix.getKeyPrefix())` twice, either we could save the value to a variable and use that
   ```suggestion
           OmKeyInfo omKeyInfo = ((OmKeyInfo keyinfo =
           omMetadataManager.getKeyTable(BucketLayout.LEGACY).getSkipCache(
           containerKeyPrefix.getKeyPrefix())) == null) ?
           omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
           .getSkipCache(containerKeyPrefix.getKeyPrefix()) : keyinfo;
   ```
   or we could do this
   ```suggestion
           OmKeyInfo omKeyInfo = Optional.ofNullable(omMetadataManager.getKeyTable(BucketLayout.LEGACY)
               .getSkipCache(containerKeyPrefix.getKeyPrefix()))
               .orElse(omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
                   .getSkipCache(containerKeyPrefix.getKeyPrefix()));
   ```
   let me know if I misunderstood something or you don't think this is necessary, but maybe it makes the code slightly more readable and we avoid a duplicated calling. 



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
             .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        if (omKeyInfo == null) {
+          omKeyInfo =
+              omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                  .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   We do support OBS bucket for this also, I have manually tested the changes out as well!
   <img width="503" alt="image" src="[https://user-images.githubusercontent.com/98023601/224228270-fbc3ca5b-f2ee-4336-970a-0e73abbafd02.png](https://user-images.githubusercontent.com/98023601/224228270-fbc3ca5b-f2ee-4336-970a-0e73abbafd02.png)">



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)

Review Comment:
   Got it thanks !! Will make the 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


[GitHub] [ozone] GeorgeJahad commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   I don't see any smoketests for this api here: https://github.com/apache/ozone/tree/master/hadoop-ozone/dist/src/main/smoketest/recon  


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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -157,11 +157,17 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+      // Directly calling get() on the Key table instead of iterating since

Review Comment:
   Thanks! I have made the 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


[GitHub] [ozone] aryangupta1998 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -157,11 +157,17 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+      // Directly calling get() on the Key table instead of iterating since

Review Comment:
   Actually, we are calling 'getSkipCache()' not 'get()' and we aren't calling the function directly on the key table rather we are deciding the table i.e, file table or key table based on the bucket layout, maybe you can update the comment accordingly.



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


[GitHub] [ozone] dombizita commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -157,11 +157,17 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+      // Directly calling getSkipCache() on the Key/FileTable table accordingly
+      // instead of iterating since only full keys are supported now. We will
+      // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+      // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
+            .getSkipCache(containerKeyPrefix.getKeyPrefix()) == null ?
+            omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                .getSkipCache(containerKeyPrefix.getKeyPrefix()) :
+            omMetadataManager.getKeyTable(BucketLayout.LEGACY)
+                .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   thanks for addressing this, it looks good to me!



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -280,6 +340,34 @@ public void testGetKeysForContainer() {
     assertEquals(3, data.getTotalCount());
   }
 
+  @Test
+  public void testGetFileTableKeysForContainer() throws IOException {
+    // test to check if the ContainerEndpoint also reads the File table
+
+    // Set up test data for FSO keys
+    setUpMultiBlockKey();
+    // Reprocess the container key mapper to ensure the latest mapping is used
+    reprocessContainerKeyMapper();
+    Response response = containerEndpoint.getKeysForContainer(20L, -1, "");
+
+    // Ensure that the expected number of keys is returned
+    KeysResponse data = (KeysResponse) response.getEntity();
+    Collection<KeyMetadata> keyMetadataList = data.getKeys();
+
+    assertEquals(1, data.getTotalCount());
+    assertEquals(1, keyMetadataList.size());
+
+    // Retrieve the first key from the list and verify its metadata
+    Iterator<KeyMetadata> iterator = keyMetadataList.iterator();
+    KeyMetadata keyMetadata = iterator.next();
+    assertEquals("dir1/file7", keyMetadata.getKey());
+    assertEquals(1, keyMetadata.getVersions().size());
+    assertEquals(1, keyMetadata.getBlockIds().size());
+    Map<Long, List<KeyMetadata.ContainerBlockMetadata>> blockIds =
+        keyMetadata.getBlockIds();
+    assertEquals(0, blockIds.get(0L).iterator().next().getLocalID());

Review Comment:
   Please create a static final for the LOCAL_ID.  Also "iterator().next()" seems convoluted, doesn't it?  couldn't it just be "get(0)"?



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)
+      throws IOException {
+    if (StringUtils.isEmpty(prevKeyPrefix)) {
+      return "";
+    }
+    // Normalize the path to remove duplicate slashes & make it easier to parse.
+    String normalizedPath = normalizePath(prevKeyPrefix);
+    String[] names = parseRequestPath(normalizedPath);
+
+    if (names.length < 3) {
+      LOG.error("Invalid path: {} path should contain a directory",
+          prevKeyPrefix);
+      return prevKeyPrefix;
+    }
+    // Extract the volume, bucket, and key names from the path.
+    String volumeName = names[0];
+    String bucketName = names[1];
+    String keyName = names[names.length - 1];
+
+    // Get the bucket handler for the given volume and bucket.
+    BucketHandler handler =
+        getBucketHandler(reconNamespaceSummaryManager, omMetadataManager,
+            reconSCM, volumeName, bucketName);
+
+    // Only keyPaths for FSO bucket need to be converted to
+    // their respective objectId's
+    if (handler.getBucketLayout() != BucketLayout.FILE_SYSTEM_OPTIMIZED) {
+      return prevKeyPrefix;
+    }
+
+    // Get the object IDs for the bucket, volume, and parent directory.
+    long bucketId, volumeId, parentId;
+    bucketId = handler.getBucketObjectId(names);
+    volumeId = handler.getVolumeObjectId(names);
+    parentId = handler.getDirObjectId(names, names.length - 1);
+
+    // Build the object path by concatenating the object IDs with the key name.
+    StringBuilder objectPathBuilder = new StringBuilder();
+    objectPathBuilder.append(OM_KEY_PREFIX).append(volumeId)
+        .append(OM_KEY_PREFIX).append(bucketId)
+        .append(OM_KEY_PREFIX).append(parentId)
+        .append(OM_KEY_PREFIX).append(keyName);
+
+    return objectPathBuilder.toString();
   }
+
+  /**
+   * Normalizes a key path by adding the OM_KEY_PREFIX at the beginning
+   * and removing duplicate slashes.
+   *
+   * @param path the key path
+   * @return the normalized key path
+   */
+  public static String normalizePath(String path) {
+    return OM_KEY_PREFIX + OmUtils.normalizeKey(path, false);

Review Comment:
   The `normalizePath()` method was added to ensure that the user provides the correct input for fso. However, it's important for users to know what input they're giving, and randomly changing their input is not a good practice. Therefore, I have removed the normalizePath() method as it is irrelevant to what `parseRequestPath()` does.
   
   
   
   



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


[GitHub] [ozone] sumitagrawl commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -162,6 +176,7 @@ public Response getKeysForContainer(
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     long totalCount;
     try {
+      prevKeyPrefix = correctPathForLayout(prevKeyPrefix);

Review Comment:
   IMO, we do not need correctPathForLayout, as this input is part of pagination.
   ie. - with pagesize 10,
   - get list with pageSize 10, and key as empty. -- list first 10 element
   - get list with pagesize 10, and key as last element -- list next 10 element
   



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


[GitHub] [ozone] ArafatKhan2198 commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

Posted by GitBox <gi...@apache.org>.
ArafatKhan2198 commented on PR #4182:
URL: https://github.com/apache/ozone/pull/4182#issuecomment-1385027040

   @aryangupta1998 @devmadhuu @dombizita @sadanand48 Can you please take a look at 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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -280,6 +340,34 @@ public void testGetKeysForContainer() {
     assertEquals(3, data.getTotalCount());
   }
 
+  @Test
+  public void testGetFileTableKeysForContainer() throws IOException {
+    // test to check if the ContainerEndpoint also reads the File table
+
+    // Set up test data for FSO keys
+    setUpMultiBlockKey();
+    // Reprocess the container key mapper to ensure the latest mapping is used
+    reprocessContainerKeyMapper();
+    Response response = containerEndpoint.getKeysForContainer(20L, -1, "");

Review Comment:
   please create a static final for the CONTAINER_ID



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


[GitHub] [ozone] dombizita commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java:
##########
@@ -90,18 +91,24 @@ public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
       reconContainerMetadataManager
               .reinitWithNewContainerDataFromOm(new HashMap<>());
 
-      Table<String, OmKeyInfo> omKeyInfoTable =
-          omMetadataManager.getKeyTable(getBucketLayout());
-      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
-               keyIter = omKeyInfoTable.iterator()) {
-        while (keyIter.hasNext()) {
-          Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
-          OmKeyInfo omKeyInfo = kv.getValue();
-          handlePutOMKeyEvent(kv.getKey(), omKeyInfo, containerKeyMap,
-              containerKeyCountMap, deletedKeyCountList);
-          omKeyCount++;
+      // loop over both key table and file table
+      for (BucketLayout layout : Arrays.asList(BucketLayout.DEFAULT,

Review Comment:
   why did you choose to use `BucketLayout.DEFAULT` here instead of `BucketLayout.LEGACY` (like in `ContainerEndpoint` on [line 177](https://github.com/apache/ozone/blob/f6f42207ba0a6bf18a58145d352c4904250a32ef/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java#L177))? later the default bucket layout could change (I believe). I think we should use `BucketLayout.LEGACY` consistently (I am fine with `BucketLayout.DEFAULT` too, but it should be consistent) 



##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -229,13 +239,63 @@ public void setUp() throws Exception {
     OMMetadataManager omMetadataManagerMock = mock(OMMetadataManager.class);
     Table tableMock = mock(Table.class);
     when(tableMock.getName()).thenReturn("KeyTable");
-    when(omMetadataManagerMock.getKeyTable(getBucketLayout()))
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.DEFAULT))

Review Comment:
   same applies here as above



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


[GitHub] [ozone] ArafatKhan2198 commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   @GeorgeJahad @ChenSammi @jojochuang Could you kindly review this patch? It has been open for some time. Please let me know if there are any additional changes required to expedite the merge process. Thank you.
   
   
   
   


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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -162,6 +176,7 @@ public Response getKeysForContainer(
     Map<String, KeyMetadata> keyMetadataMap = new LinkedHashMap<>();
     long totalCount;
     try {
+      prevKeyPrefix = correctPathForLayout(prevKeyPrefix);

Review Comment:
   Thanks for the analysis sumit, you were absolutely right on this one the prevKey Prefix was used for pagination purposes. 



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -278,10 +410,41 @@ public void testGetKeysForContainer() {
     keyMetadataList = data.getKeys();
     assertEquals(1, keyMetadataList.size());
     assertEquals(3, data.getTotalCount());
+
+    // Now to check if the ContainerEndpoint also reads the File table
+    // Set up test data for FSO keys
+    setUpFSOData();

Review Comment:
   Thanks for the comment @devmadhuu I have added separate UT's for the scenario where we pass a non-empty `prevKeyPrefix` 



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)

Review Comment:
   I don't see this method being tested anywhere.  Am I missing it.
   
   Also the name sounds awkward because it sounds like it is only return fso paths.  I would probably just call it correctPathForLayout() or something like that.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -397,7 +419,81 @@ private List<ContainerBlockMetadata> getBlocks(
     return blockIds;
   }
 
-  private BucketLayout getBucketLayout() {
-    return BucketLayout.DEFAULT;
+  /**
+   * Builds an object path for a file system optimized bucket.
+   *
+   * @param prevKeyPrefix the previous key prefix of the object path
+   * @return the object path for the file system optimized bucket
+   * @throws IOException if an IO error occurs
+   */
+  private String buildObjectPathForFileSystemBucket(String prevKeyPrefix)

Review Comment:
   I don't see this method being tested anywhere.  Am I missing it?
   
   Also the name sounds awkward because it sounds like it is only return fso paths.  I would probably just call it correctPathForLayout() or something like 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


[GitHub] [ozone] sumitagrawl commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
             .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        if (omKeyInfo == null) {
+          omKeyInfo =
+              omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)

Review Comment:
   Check keyPrefix, what is the format used here for Key and file and converted to required format?
   FSO: /vol/bucket/ParentId/KeyName
   ObjectStore: /vol/bucket/keyName
   



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)

Review Comment:
   Here we are trying to check both key and file table and functionally will work. But Legacy can work both FSO and ObjectStore based on property in Ozone. I think better use Object Store than Legacy word.



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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -229,13 +239,63 @@ public void setUp() throws Exception {
     OMMetadataManager omMetadataManagerMock = mock(OMMetadataManager.class);
     Table tableMock = mock(Table.class);
     when(tableMock.getName()).thenReturn("KeyTable");
-    when(omMetadataManagerMock.getKeyTable(getBucketLayout()))
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.DEFAULT))

Review Comment:
   Thanks for the comment @dombizita since both Legacy and Default serve the same purpose, I might have jumbled them up in places. I have made the required changes do take a look !!



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -229,13 +239,63 @@ public void setUp() throws Exception {
     OMMetadataManager omMetadataManagerMock = mock(OMMetadataManager.class);
     Table tableMock = mock(Table.class);
     when(tableMock.getName()).thenReturn("KeyTable");
-    when(omMetadataManagerMock.getKeyTable(getBucketLayout()))
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.LEGACY))
         .thenReturn(tableMock);
-    ContainerKeyMapperTask containerKeyMapperTask  =
+    when(omMetadataManagerMock.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED))
+        .thenReturn(tableMock);
+    reprocessContainerKeyMapper();
+  }
+
+  private void reprocessContainerKeyMapper() {
+    ContainerKeyMapperTask containerKeyMapperTask =
         new ContainerKeyMapperTask(reconContainerMetadataManager);
     containerKeyMapperTask.reprocess(reconOMMetadataManager);
   }
 
+  private void setUpMultiBlockKey() throws IOException {
+    OmKeyLocationInfoGroup locationInfoGroup =
+        getLocationInfoGroup1();
+
+    // add the multi-block key to Recon's OM
+    writeKeyToOm(reconOMMetadataManager,
+        FSO_KEY_NAME,
+        BUCKET_NAME,
+        VOLUME_NAME,
+        FILE_NAME,
+        KEY_LENGTH,
+        BLOCK_LENGTH,
+        KEY_SEQ_NO,
+        MODIFICATION_TIME,
+        Collections.singletonList(locationInfoGroup),
+        getBucketLayout(),

Review Comment:
   this layout returns default for 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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

Posted by GitBox <gi...@apache.org>.
ArafatKhan2198 commented on code in PR #4182:
URL: https://github.com/apache/ozone/pull/4182#discussion_r1082291351


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -160,8 +160,19 @@ public Response getKeysForContainer(
         // Directly calling get() on the Key table instead of iterating since
         // only full keys are supported now. When we change to using a prefix
         // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
-            .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        // The first expression will try to get the OmKeyInfo object by calling
+        // getSkipCache on the BucketLayout.LEGACY table, passing in the
+        // keyPrefix as the key. If this expression returns null, meaning
+        // the key was not found in the LEGACY table, the second expression
+        // will be evaluated and the OmKeyInfo will be obtained from the
+        // BucketLayout.FILE_SYSTEM_OPTIMIZED table.

Review Comment:
   Thanks for the review. I have trimmed down the comment !!



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


[GitHub] [ozone] GeorgeJahad commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   Is there a reason why no tests have been added to confirm this fix?
   


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


[GitHub] [ozone] kerneltime commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   @GeorgeJahad can you please take a look as well.


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


[GitHub] [ozone] ArafatKhan2198 commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java:
##########
@@ -170,11 +170,18 @@ public Response getKeysForContainer(
       for (ContainerKeyPrefix containerKeyPrefix : containerKeyPrefixMap
           .keySet()) {
 
-        // Directly calling get() on the Key table instead of iterating since
-        // only full keys are supported now. When we change to using a prefix
-        // of the key, this needs to change to prefix seek.
-        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout())
+        // Directly calling getSkipCache() on the Key/FileTable table
+        // instead of iterating since only full keys are supported now. We will
+        // try to get the OmKeyInfo object by searching the KEY_TABLE table with
+        // the key prefix. If it's not found, we will then search the FILE_TABLE
+        OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable(BucketLayout.LEGACY)
             .getSkipCache(containerKeyPrefix.getKeyPrefix());
+        if (omKeyInfo == null) {
+          omKeyInfo =
+              omMetadataManager.getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED)
+                  .getSkipCache(containerKeyPrefix.getKeyPrefix());

Review Comment:
   Yes we do support OBS buckets also :- https://github.com/apache/ozone/pull/4182#issuecomment-1439041949



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


[GitHub] [ozone] ArafatKhan2198 commented on pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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

   While working on this patch, I had a doubt regarding the endpoint **`GET /api/v1/containers/:id/keys`**, which is handled by the **`getKeysForContainer()`** method in the ContainerEndpoint class. This endpoint supports two optional query parameters, **`prevKey` and `limit`**.
   
   Initially, we assumed that the **`prevKey`** parameter was used to filter the keys in a container based on a certain prefix (specified by **`prevKey`**) in their name, and only the keys with names that started with this prefix would be returned, while those that did not have this prefix would be excluded from the response. 
   
   So after a discussion this is what we have understood about the `prevKey` parameter of the `ContainerEndpoint` in Recon :-
   
   - The method `getKeysForContainers()` can be used by the **RECON-UI** for pagination by making use of the "`prev-key`" query parameter and the "`limit`" parameter.
   - The "`prev-key`" parameter is used to specify the last key seen on the previous page. The method then retrieves all the keys starting from the key that comes after the "prev-key" parameter up to the limit specified by the "`limit`" parameter.
   - For example, if the UI wants to retrieve the first 10 keys in a container, it will call this method with a limit of 10 and no "prev-key" parameter. The method will then retrieve the first 10 keys in the container and return them to the UI along with the last key seen (which will be the 10th key).
   - If the UI wants to retrieve the next 10 keys, it will call the method again with the same limit of 10 and the "prev-key" parameter set to the last key seen on the previous page (i.e., the 10th key). The method will then retrieve the next 10 keys in the container starting from the key that comes after the 10th key and return them to the UI along with the last key seen (which will be the 20th key).
   - This process can be repeated for as many pages as the UI wants to retrieve. The UI can also change the limit parameter to retrieve more or fewer keys per page.


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


[GitHub] [ozone] devmadhuu commented on a diff in pull request #4182: HDDS-5463. [FSO] Recon Container API does not work correctly with FSO.

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


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java:
##########
@@ -278,10 +410,41 @@ public void testGetKeysForContainer() {
     keyMetadataList = data.getKeys();
     assertEquals(1, keyMetadataList.size());
     assertEquals(3, data.getTotalCount());
+
+    // Now to check if the ContainerEndpoint also reads the File table
+    // Set up test data for FSO keys
+    setUpFSOData();

Review Comment:
   @ArafatKhan2198 - Thanks for working on this patch. As discussed, pls add 2 test cases separately for testing prevKeyPrefix behavior for FSO bucket keys. Add 4 FSO keys on 4 separate paths.
   1. prevKeyPrefix - empty
   2. prevKeyPrefix - Pass 2nd path as prevKeyPrefix and assert for 3rd path in API response.



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