You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "hemantk-12 (via GitHub)" <gi...@apache.org> on 2023/07/27 23:49:16 UTC

[GitHub] [ozone] hemantk-12 commented on a diff in pull request #5115: HDDS-9079. Implement a lightweight listKeys API

hemantk-12 commented on code in PR #5115:
URL: https://github.com/apache/ozone/pull/5115#discussion_r1276910693


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java:
##########
@@ -1004,6 +1006,46 @@ public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
 
   }
 
+  /**
+   * Lightweight listKeys implementation.
+   */
+  @Override
+  public List<BasicOmKeyInfo> listKeysLight(String volumeName,
+      String bucketName, String startKey, String prefix,
+      int maxKeys) throws IOException {
+    List<BasicOmKeyInfo> keys = new ArrayList<>();
+    ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
+    reqBuilder.setVolumeName(volumeName);
+    reqBuilder.setBucketName(bucketName);
+    reqBuilder.setCount(maxKeys);
+
+    if (startKey != null) {

Review Comment:
   nit: should we add empty check too? If yes, `StringUtils.isNotEmpty()` or `StringUtils.isNotBlank()` can be used for that.



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java:
##########
@@ -1004,6 +1006,46 @@ public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
 
   }
 
+  /**
+   * Lightweight listKeys implementation.
+   */
+  @Override
+  public List<BasicOmKeyInfo> listKeysLight(String volumeName,
+      String bucketName, String startKey, String prefix,
+      int maxKeys) throws IOException {
+    List<BasicOmKeyInfo> keys = new ArrayList<>();
+    ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
+    reqBuilder.setVolumeName(volumeName);
+    reqBuilder.setBucketName(bucketName);
+    reqBuilder.setCount(maxKeys);
+
+    if (startKey != null) {
+      reqBuilder.setStartKey(startKey);
+    }
+
+    if (prefix != null) {
+      reqBuilder.setPrefix(prefix);
+    }
+
+    ListKeysRequest req = reqBuilder.build();
+
+    OMRequest omRequest = createOMRequest(Type.ListKeysLight)
+            .setListKeysRequest(req)
+            .build();
+
+    ListKeysLightResponse resp =
+            handleError(submitRequest(omRequest)).getListKeysLightResponse();
+    List<BasicOmKeyInfo> list = new ArrayList<>();

Review Comment:
   Why did you create another `list` object and not directly used `keys`?



##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1072,6 +1074,19 @@ message KeyInfo {
     optional bool isFile = 19;
 }
 
+message BasicKeyInfo {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    required string keyName = 3;
+    required uint64 dataSize = 4;
+    required uint64 creationTime = 5;
+    required uint64 modificationTime = 6;
+    required hadoop.hdds.ReplicationType type = 7;
+    optional hadoop.hdds.ReplicationFactor factor = 8;
+    optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 9;
+    optional bool isFile = 10;

Review Comment:
   qq: how is it useful for client? Isn't `keyName` enough to tell if it is key or dir? If I'm not wrong dir ends with `/`.



##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1347,6 +1362,10 @@ message ListKeysResponse {
     repeated KeyInfo keyInfo = 2;
 }
 
+message ListKeysLightResponse {
+    repeated BasicKeyInfo basicKeyInfo = 2;

Review Comment:
   QQ: Is it intentionally to set `basicKeyInfo` to 2 or just copy/paste thing.
   



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -2838,6 +2839,32 @@ public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
     }
   }
 
+  @Override
+  public List<BasicOmKeyInfo> listKeysLight(String volumeName,
+                                            String bucketName,
+                                            String startKey, String keyPrefix,
+                                            int maxKeys) throws IOException {
+    // Call the existing listKeys method to get the list of OmKeyInfo

Review Comment:
   Same as previous comment.



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BasicOmKeyInfo.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.helpers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BasicKeyInfo;
+
+/**
+ * Lightweight OmKeyInfo class.
+ */
+public class BasicOmKeyInfo {
+
+  private String volumeName;
+  private String bucketName;
+  private String keyName;
+  private long dataSize;
+  private long creationTime;
+  private long modificationTime;
+  private ReplicationConfig replicationConfig;
+  private boolean isFile;
+
+  @SuppressWarnings("parameternumber")
+  public BasicOmKeyInfo(String volumeName, String bucketName, String keyName,
+                        long dataSize, long creationTime, long modificationTime,
+                        ReplicationConfig replicationConfig, boolean isFile) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.keyName = keyName;
+    this.dataSize = dataSize;
+    this.creationTime = creationTime;
+    this.modificationTime = modificationTime;
+    this.replicationConfig = replicationConfig;
+    this.isFile = isFile;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  public long getDataSize() {
+    return dataSize;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public long getModificationTime() {
+    return modificationTime;
+  }
+
+  public ReplicationConfig getReplicationConfig() {
+    return replicationConfig;
+  }
+
+  public boolean isFile() {
+    return isFile;
+  }
+
+  /**
+   * Builder of BasicOmKeyInfo.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private String keyName;
+    private long dataSize;
+    private long creationTime;
+    private long modificationTime;
+    private ReplicationConfig replicationConfig;
+    private boolean isFile;
+
+    public Builder setVolumeName(String volumeName) {
+      this.volumeName = volumeName;
+      return this;
+    }
+
+    public Builder setBucketName(String bucketName) {
+      this.bucketName = bucketName;
+      return this;
+    }
+
+    public Builder setKeyName(String keyName) {
+      this.keyName = keyName;
+      return this;
+    }
+
+    public Builder setDataSize(long dataSize) {
+      this.dataSize = dataSize;
+      return this;
+    }
+
+    public Builder setCreationTime(long creationTime) {
+      this.creationTime = creationTime;
+      return this;
+    }
+
+    public Builder setModificationTime(long modificationTime) {
+      this.modificationTime = modificationTime;
+      return this;
+    }
+
+    public Builder setReplicationConfig(ReplicationConfig replicationConfig) {
+      this.replicationConfig = replicationConfig;
+      return this;
+    }
+
+    public Builder setIsFile(boolean isFile) {
+      this.isFile = isFile;
+      return this;
+    }
+
+    public BasicOmKeyInfo build() {
+      return new BasicOmKeyInfo(volumeName, bucketName, keyName, dataSize,
+              creationTime, modificationTime, replicationConfig, isFile);
+    }
+  }
+
+  public BasicKeyInfo getProtobuf() {
+    BasicKeyInfo.Builder builder = BasicKeyInfo.newBuilder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setDataSize(dataSize)
+            .setCreationTime(creationTime)
+            .setModificationTime(modificationTime)
+            .setType(replicationConfig.getReplicationType());
+    if (replicationConfig instanceof ECReplicationConfig) {
+      builder.setEcReplicationConfig(
+              ((ECReplicationConfig) replicationConfig).toProto());
+    } else {
+      builder.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+    }
+    builder.setIsFile(isFile);
+
+    return builder.build();
+  }
+
+  public static BasicOmKeyInfo getFromProtobuf(BasicKeyInfo basicKeyInfo)
+          throws IOException {
+    if (basicKeyInfo == null) {
+      return null;
+    }
+
+    Builder builder = new Builder()

Review Comment:
   Same as above.
   
   ```suggestion
       Builder builder = new Builder()
           .setVolumeName(basicKeyInfo.getVolumeName())
           .setBucketName(basicKeyInfo.getBucketName())
           .setKeyName(basicKeyInfo.getKeyName())
           .setDataSize(basicKeyInfo.getDataSize())
           .setCreationTime(basicKeyInfo.getCreationTime())
           .setModificationTime(basicKeyInfo.getModificationTime())
           .setReplicationConfig(ReplicationConfig.fromProto(
               basicKeyInfo.getType(),
               basicKeyInfo.getFactor(),
               basicKeyInfo.getEcReplicationConfig()));
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshot.java:
##########
@@ -141,6 +142,31 @@ public List<OmKeyInfo> listKeys(String vname, String bname,
         .collect(Collectors.toList());
   }
 
+  @Override
+  public List<BasicOmKeyInfo> listKeysLight(String volName,

Review Comment:
   Same as previous comment.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataReader.java:
##########
@@ -339,6 +341,32 @@ public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
     }
   }
 
+  @Override

Review Comment:
   nit: 
   1. I think it would be nicer if you create a helper function that transformation `OmKeyInfo` to `BasicOmKeyInfo` and use the same in `OzoneManager` and `OmSnapshot`.  You can add this helper to `BasicOmKeyInfo` class itself.
   
   ```suggestion
     @Override
     public List<BasicOmKeyInfo> listKeysLight(String volumeName,
                                               String bucketName,
                                               String startKey,
                                               String keyPrefix,
                                               int maxKeys) throws IOException {
   
       List<OmKeyInfo> keys = listKeys(volumeName, bucketName, startKey, keyPrefix,
           maxKeys);
   
       return keys.stream().map(BasicOmKeyInfo::fromOmKeyInfo)
           .collect(Collectors.toList());
     }
   ```
   
   And something like this in `BasicOmKeyInfo`:
   
   ```
     public static BasicOmKeyInfo fromOmKeyInfo(OmKeyInfo omKeyInfo) {
       return new BasicOmKeyInfo(
           omKeyInfo.getVolumeName(),
           omKeyInfo.getBucketName(),
           omKeyInfo.getKeyName(),
           omKeyInfo.getDataSize(),
           omKeyInfo.getCreationTime(),
           omKeyInfo.getModificationTime(),
           omKeyInfo.getReplicationConfig(),
           omKeyInfo.isFile());
     }
   ```
   
   2. Also there is no need of comments you added. It is just repeating what code is coding. Code comment should be added only to state why something is done in particular way. And I don't think `why` is needed here because it is pretty straightforward.



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BasicOmKeyInfo.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.helpers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BasicKeyInfo;
+
+/**
+ * Lightweight OmKeyInfo class.
+ */
+public class BasicOmKeyInfo {
+
+  private String volumeName;
+  private String bucketName;
+  private String keyName;
+  private long dataSize;
+  private long creationTime;
+  private long modificationTime;
+  private ReplicationConfig replicationConfig;
+  private boolean isFile;
+
+  @SuppressWarnings("parameternumber")
+  public BasicOmKeyInfo(String volumeName, String bucketName, String keyName,
+                        long dataSize, long creationTime, long modificationTime,
+                        ReplicationConfig replicationConfig, boolean isFile) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.keyName = keyName;
+    this.dataSize = dataSize;
+    this.creationTime = creationTime;
+    this.modificationTime = modificationTime;
+    this.replicationConfig = replicationConfig;
+    this.isFile = isFile;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  public long getDataSize() {
+    return dataSize;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public long getModificationTime() {
+    return modificationTime;
+  }
+
+  public ReplicationConfig getReplicationConfig() {
+    return replicationConfig;
+  }
+
+  public boolean isFile() {
+    return isFile;
+  }
+
+  /**
+   * Builder of BasicOmKeyInfo.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private String keyName;
+    private long dataSize;
+    private long creationTime;
+    private long modificationTime;
+    private ReplicationConfig replicationConfig;
+    private boolean isFile;
+
+    public Builder setVolumeName(String volumeName) {
+      this.volumeName = volumeName;
+      return this;
+    }
+
+    public Builder setBucketName(String bucketName) {
+      this.bucketName = bucketName;
+      return this;
+    }
+
+    public Builder setKeyName(String keyName) {
+      this.keyName = keyName;
+      return this;
+    }
+
+    public Builder setDataSize(long dataSize) {
+      this.dataSize = dataSize;
+      return this;
+    }
+
+    public Builder setCreationTime(long creationTime) {
+      this.creationTime = creationTime;
+      return this;
+    }
+
+    public Builder setModificationTime(long modificationTime) {
+      this.modificationTime = modificationTime;
+      return this;
+    }
+
+    public Builder setReplicationConfig(ReplicationConfig replicationConfig) {
+      this.replicationConfig = replicationConfig;
+      return this;
+    }
+
+    public Builder setIsFile(boolean isFile) {
+      this.isFile = isFile;
+      return this;
+    }
+
+    public BasicOmKeyInfo build() {
+      return new BasicOmKeyInfo(volumeName, bucketName, keyName, dataSize,
+              creationTime, modificationTime, replicationConfig, isFile);
+    }
+  }
+
+  public BasicKeyInfo getProtobuf() {

Review Comment:
   nit: not properly indented with 4 spaces for continuation of line.
   ```suggestion
     public BasicKeyInfo getProtobuf() {
       BasicKeyInfo.Builder builder = BasicKeyInfo.newBuilder()
           .setVolumeName(volumeName)
           .setBucketName(bucketName)
           .setKeyName(keyName)
           .setDataSize(dataSize)
           .setCreationTime(creationTime)
           .setModificationTime(modificationTime)
           .setType(replicationConfig.getReplicationType());
       if (replicationConfig instanceof ECReplicationConfig) {
         builder.setEcReplicationConfig(
             ((ECReplicationConfig) replicationConfig).toProto());
       } else {
         builder.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
       }
       builder.setIsFile(isFile);
   
       return builder.build();
     }
   ```
   
   Not sure if your IDE is configured with [ozone-coding-style](https://github.com/apache/ozone/blob/master/hadoop-ozone/dev-support/intellij/ozone-style.xml). If not, please add this to your IDE's code style.



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