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 2022/09/28 18:32:42 UTC

[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3784: HDDS-6854. Implement List Snapshot API

GeorgeJahad commented on code in PR #3784:
URL: https://github.com/apache/ozone/pull/3784#discussion_r982739755


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -1125,6 +1125,100 @@ public List<RepeatedOmKeyInfo> listTrash(String volumeName, String bucketName,
     return deletedKeys;
   }
 
+  @Override
+  public List<SnapshotInfo> listSnapshot(String volumeName, String bucketName,
+      String startKey, String prefix) throws IOException {
+    if (Strings.isNullOrEmpty(volumeName)) {
+      throw new OMException("Volume name is required.",
+          ResultCodes.VOLUME_NOT_FOUND);
+    }
+
+    if (Strings.isNullOrEmpty(bucketName)) {
+      throw new OMException("Bucket name is required.",
+          ResultCodes.BUCKET_NOT_FOUND);
+    }
+
+    List<SnapshotInfo> snapshotInfos = new ArrayList<>();
+    TreeMap<String, SnapshotInfo> filteredSnapshotInfoMap = new TreeMap<>();
+    String seekStartKey;
+    boolean skipStartKey = false;
+    if (StringUtil.isNotBlank(startKey)) {
+      // Seek to the specified key.
+      seekStartKey = getOzoneKey(volumeName, bucketName, startKey);
+      skipStartKey = true;
+    } else {
+      // This allows us to seek directly to the first key with the right prefix.
+      seekStartKey = getOzoneKey(volumeName, bucketName,
+          StringUtil.isNotBlank(prefix) ? prefix : OM_KEY_PREFIX);
+    }
+
+    String seekPrefix;
+    if (StringUtil.isNotBlank(prefix)) {
+      seekPrefix = getOzoneKey(volumeName, bucketName, prefix);
+    } else {
+      seekPrefix = getBucketKey(volumeName, bucketName + OM_KEY_PREFIX);
+    }
+
+    filteredSnapshotInfoMap.putAll(
+        getSnapshotFromCache(seekStartKey, seekPrefix));
+    filteredSnapshotInfoMap.putAll(getSnapshotFromDB(seekStartKey, seekPrefix));
+
+    for (Map.Entry<String, SnapshotInfo> cacheKey : filteredSnapshotInfoMap
+        .entrySet()) {
+      if (cacheKey.getKey().equals(seekStartKey) && skipStartKey) {
+        continue;
+      }
+      snapshotInfos.add(cacheKey.getValue());
+    }
+
+    return snapshotInfos;
+  }
+
+  private TreeMap<String, SnapshotInfo> getSnapshotFromCache(
+      String seekStartKey, String seekPrefix) {
+    TreeMap<String, SnapshotInfo> result = new TreeMap<>();
+    Iterator<Map.Entry<CacheKey<String>, CacheValue<SnapshotInfo>>> iterator =
+        snapshotInfoTable.cacheIterator();
+    while (iterator.hasNext()) {
+      Map.Entry<CacheKey<String>, CacheValue<SnapshotInfo>> entry =
+          iterator.next();
+      String snapshotKey = entry.getKey().getCacheKey();
+      SnapshotInfo snapshotInfo = entry.getValue().getCacheValue();
+      if (snapshotInfo != null
+          && snapshotKey.startsWith(seekPrefix)
+          && snapshotKey.compareTo(seekStartKey) >= 0) {
+        result.put(snapshotKey, snapshotInfo);
+      }
+    }
+    return result;
+  }
+
+  private TreeMap<String, SnapshotInfo> getSnapshotFromDB(
+      String seekStartKey, String seekPrefix) throws IOException {
+    TreeMap<String, SnapshotInfo> result = new TreeMap<>();
+    try (TableIterator<String, ? extends KeyValue<String, SnapshotInfo>>
+             snapshotIter = snapshotInfoTable.iterator()) {
+      KeyValue< String, SnapshotInfo> snapshotinfo;
+      snapshotIter.seek(seekStartKey);
+      while (snapshotIter.hasNext()) {
+        snapshotinfo = snapshotIter.next();
+        if (snapshotinfo != null && snapshotinfo.getKey()
+            .startsWith(seekPrefix)) {
+          CacheValue<SnapshotInfo> cacheValue =
+              snapshotInfoTable.getCacheValue(
+                  new CacheKey<>(snapshotinfo.getKey()));
+          if (cacheValue == null || cacheValue.getCacheValue() != null) {

Review Comment:
   I don't understand this:
   ```
    cacheValue.getCacheValue() != null
   ```
   should it be this?:
   ```
    cacheValue.getCacheValue() == null
   ```



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