This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new d2314a5762 HDDS-10025. Add more function for Freon OmMetadataGenerator 
(#6037)
d2314a5762 is described below

commit d2314a5762cbad4bc81b4e97e4d50f1ec1597eeb
Author: XiChen <[email protected]>
AuthorDate: Wed Jan 31 16:52:59 2024 +0800

    HDDS-10025. Add more function for Freon OmMetadataGenerator (#6037)
---
 .../main/smoketest/freon/metadata-generate.robot   |  75 +++++++++
 .../hadoop/ozone/freon/OmMetadataGenerator.java    | 176 +++++++++++----------
 2 files changed, 166 insertions(+), 85 deletions(-)

diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/metadata-generate.robot 
b/hadoop-ozone/dist/src/main/smoketest/freon/metadata-generate.robot
new file mode 100644
index 0000000000..acbcb0a151
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/freon/metadata-generate.robot
@@ -0,0 +1,75 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation       Test freon ommg command
+Resource            ../ozone-lib/freon.robot
+Test Timeout        5 minutes
+
+*** Variables ***
+${PREFIX}    ${EMPTY}
+${n}    100
+${VOLUME}       volume1
+${BUCKET_FSO}       bucket-fso
+${BUCKET_OBJ}       bucket-obj
+
+*** Test Cases ***
+[Setup] Create Volume and Buckets
+    ${result} =     Execute             ozone sh volume create /${VOLUME}
+                    Should not contain  ${result}       Failed
+    ${result} =     Execute             ozone sh bucket create 
/${VOLUME}/${BUCKET_FSO} -l FILE_SYSTEM_OPTIMIZED
+                    Should not contain  ${result}       Failed
+    ${result} =     Execute             ozone sh bucket create 
/${VOLUME}/${BUCKET_OBJ} -l OBJECT_STORE
+                    Should not contain  ${result}       Failed
+
+[Read] Bucket Information
+    ${result} =        Execute          ozone freon ommg --operation 
INFO_BUCKET -n ${n} --bucket ${BUCKET_FSO}
+                       Should contain   ${result}   Successful executions: ${n}
+
+[Create] File in FILE_SYSTEM_OPTIMIZED Bucket
+    ${result} =        Execute          ozone freon ommg --operation 
CREATE_FILE -n ${n} --size 4096 --volume ${VOLUME} --bucket ${BUCKET_FSO}
+                       Should contain   ${result}   Successful executions: ${n}
+
+[Read] File in FILE_SYSTEM_OPTIMIZED Bucket
+    ${result} =        Execute          ozone freon ommg --operation READ_FILE 
-n ${n} --volume ${VOLUME} --bucket ${BUCKET_FSO} --size 4096
+                       Should contain   ${result}   Successful executions: ${n}
+
+[List] File Status in FILE_SYSTEM_OPTIMIZED Bucket
+    ${result} =        Execute          ozone freon ommg --operation 
LIST_STATUS -n 1 --volume ${VOLUME} --bucket ${BUCKET_FSO} --batch-size ${n}
+                       Should contain   ${result}   Successful executions: 1
+
+[List] light File status in FILE_SYSTEM_OPTIMIZED Bucket
+    ${result} =        Execute          ozone freon ommg --operation 
LIST_STATUS_LIGHT -n 1 --volume ${VOLUME} --bucket ${BUCKET_FSO} --batch-size 
${n}
+                       Should contain   ${result}   Successful executions: 1
+
+[Create] Key in OBJECT_STORE Bucket
+    ${result} =        Execute          ozone freon ommg --operation 
CREATE_KEY -n ${n} --size 4096 --volume ${VOLUME} --bucket ${BUCKET_OBJ}
+                       Should contain   ${result}   Successful executions: ${n}
+
+[Read] Key in OBJECT_STORE Bucket
+    ${result} =        Execute          ozone freon ommg --operation READ_KEY 
-n ${n} --volume ${VOLUME} --bucket ${BUCKET_OBJ} --size 4096
+                       Should contain   ${result}   Successful executions: ${n}
+
+[List] Keys in OBJECT_STORE Bucket
+    ${result} =        Execute          ozone freon ommg --operation LIST_KEYS 
-n 1 --volume ${VOLUME} --bucket ${BUCKET_OBJ} --batch-size ${n}
+                       Should contain   ${result}   Successful executions: 1
+
+[List] Light Keys in OBJECT_STORE Bucket
+    ${result} =        Execute          ozone freon ommg --operation 
LIST_KEYS_LIGHT -n 1 --volume ${VOLUME} --bucket ${BUCKET_OBJ} --batch-size ${n}
+                       Should contain   ${result}   Successful executions: 1
+
+[Get] Key Information in OBJECT_STORE Bucket
+    ${result} =        Execute          ozone freon ommg --operation 
GET_KEYINFO -n ${n} --volume ${VOLUME} --bucket ${BUCKET_OBJ}
+                       Should contain   ${result}   Successful executions: ${n}
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmMetadataGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmMetadataGenerator.java
index c94048e00d..60af701e14 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmMetadataGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmMetadataGenerator.java
@@ -17,6 +17,7 @@
 package org.apache.hadoop.ozone.freon;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.file.NoSuchFileException;
 import java.time.Duration;
@@ -41,20 +42,24 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.StorageSize;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.om.helpers.BasicOmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs.Builder;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
 import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatusLight;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 
 import com.codahale.metrics.Timer;
 import org.apache.hadoop.security.UserGroupInformation;
 import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
 import picocli.CommandLine.Option;
 
 import static 
org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
+import static java.util.Collections.emptyMap;
+
 
 /**
  * Data generator tool test om performance.
@@ -70,15 +75,19 @@ public class OmMetadataGenerator extends BaseFreonGenerator
 
   enum Operation {
     CREATE_FILE,
+    CREATE_STREAM_FILE,
     LOOKUP_FILE,
     READ_FILE,
     LIST_STATUS,
+    LIST_STATUS_LIGHT,
     CREATE_KEY,
+    CREATE_STREAM_KEY,
     LOOKUP_KEY,
     GET_KEYINFO,
     HEAD_KEY,
     READ_KEY,
     LIST_KEYS,
+    LIST_KEYS_LIGHT,
     INFO_BUCKET,
     INFO_VOLUME,
     MIXED,
@@ -145,6 +154,9 @@ public class OmMetadataGenerator extends BaseFreonGenerator
   )
   private String omServiceID;
 
+  @Mixin
+  private FreonReplicationOptions replication;
+
   private OzoneManagerProtocol ozoneManagerClient;
 
   private ThreadLocal<OmKeyArgs.Builder> omKeyArgsBuilder;
@@ -171,7 +183,7 @@ public class OmMetadataGenerator extends BaseFreonGenerator
     contentGenerator = new ContentGenerator(dataSize.toBytes(), bufferSize);
     omKeyArgsBuilder = ThreadLocal.withInitial(this::createKeyArgsBuilder);
     OzoneConfiguration conf = createOzoneConfiguration();
-    replicationConfig = ReplicationConfig.getDefault(conf);
+    replicationConfig = replication.fromParamsOrConfig(conf);
 
     try (OzoneClient rpcClient = createOzoneClient(omServiceID, conf)) {
       ensureVolumeAndBucketExist(rpcClient, volumeName, bucketName);
@@ -309,145 +321,113 @@ public class OmMetadataGenerator extends 
BaseFreonGenerator
     };
   }
 
-  @SuppressWarnings("checkstyle:EmptyBlock")
   private void applyOperation(long counter) throws Exception {
     OmKeyArgs keyArgs;
-    String keyName;
-    long threadSeqId;
+    final long threadSeqId = getThreadSequenceId();
     String startKeyName;
     if (mixedOperation) {
-      threadSeqId = getThreadSequenceId();
       operation = operations[(int)threadSeqId];
     }
     if (randomOp) {
       counter = ThreadLocalRandom.current().nextLong(getTestNo());
     }
+    final String keyName = getPath(counter);
     switch (operation) {
     case CREATE_KEY:
-      keyName = getPath(counter);
-      getMetrics().timer(operation.name()).time(() -> {
-        try (OutputStream stream = bucket.createStreamKey(keyName,
-            dataSize.toBytes(), replicationConfig, new HashMap<>())) {
-          contentGenerator.write(stream);
-        }
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> performWriteOperation(() 
->
+          bucket.createKey(keyName, dataSize.toBytes(), replicationConfig, 
emptyMap()), contentGenerator));
+      break;
+    case CREATE_STREAM_KEY:
+      getMetrics().timer(operation.name()).time(() -> performWriteOperation(() 
->
+          bucket.createStreamKey(keyName, dataSize.toBytes(), 
replicationConfig, emptyMap()), contentGenerator));
       break;
     case LOOKUP_KEY:
-      keyName = getPath(counter);
       keyArgs = omKeyArgsBuilder.get().setKeyName(keyName).build();
-      getMetrics().timer(operation.name()).time(() -> {
-        ozoneManagerClient.lookupKey(keyArgs);
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> 
ozoneManagerClient.lookupKey(keyArgs));
       break;
     case GET_KEYINFO:
-      keyName = getPath(counter);
       keyArgs = omKeyArgsBuilder.get().setKeyName(keyName).build();
-      getMetrics().timer(operation.name()).time(() -> {
-        ozoneManagerClient.getKeyInfo(keyArgs, false);
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> 
ozoneManagerClient.getKeyInfo(keyArgs, false));
       break;
     case HEAD_KEY:
-      keyName = getPath(counter);
       keyArgs = omKeyArgsBuilder.get()
           .setKeyName(keyName).setHeadOp(true).build();
-      getMetrics().timer(operation.name()).time(() -> {
-        ozoneManagerClient.getKeyInfo(keyArgs, false);
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> 
ozoneManagerClient.getKeyInfo(keyArgs, false));
       break;
     case READ_KEY:
-      keyName = getPath(counter);
-      getMetrics().timer(operation.name()).time(() -> {
-        try (OzoneInputStream stream = bucket.readKey(keyName)) {
-          while ((stream.read(readBuffer)) >= 0) {
-          }
-        }
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> performReadOperation(() 
-> bucket.readKey(keyName), readBuffer));
       break;
     case READ_FILE:
-      keyName = getPath(counter);
-      getMetrics().timer(operation.name()).time(() -> {
-        try (OzoneInputStream stream = bucket.readFile(keyName)) {
-          while ((stream.read(readBuffer)) >= 0) {
-          }
-        }
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> performReadOperation(() 
-> bucket.readFile(keyName), readBuffer));
       break;
     case CREATE_FILE:
-      keyName = getPath(counter);
-      getMetrics().timer(operation.name()).time(() -> {
-        try (
-            OutputStream stream = bucket.createFile(keyName, 
dataSize.toBytes(),
-                replicationConfig, true, false)) {
-          contentGenerator.write(stream);
-        }
-        return null;
-      });
+      getMetrics().timer(operation.name()).time(() -> performWriteOperation(() 
->
+          bucket.createFile(keyName, dataSize.toBytes(), replicationConfig, 
true, false), contentGenerator));
+      break;
+    case CREATE_STREAM_FILE:
+      getMetrics().timer(operation.name()).time(() -> performWriteOperation(() 
->
+          bucket.createStreamFile(keyName, dataSize.toBytes(), 
replicationConfig, true, false), contentGenerator));
       break;
     case LOOKUP_FILE:
-      keyName = getPath(counter);
       keyArgs = omKeyArgsBuilder.get().setKeyName(keyName).build();
+      getMetrics().timer(operation.name()).time(() -> 
ozoneManagerClient.lookupFile(keyArgs));
+      break;
+    case LIST_KEYS:
+      startKeyName = getPath(threadSeqId * batchSize);
       getMetrics().timer(operation.name()).time(() -> {
-        ozoneManagerClient.lookupFile(keyArgs);
+        List<OmKeyInfo> keyInfoList =
+            ozoneManagerClient.listKeys(volumeName, bucketName, startKeyName, 
"", batchSize).getKeys();
+        if (keyInfoList.size() + 1 < batchSize) {
+          throw new NoSuchFileException("There are not enough keys for testing 
you should use "
+                  + "CREATE_KEY to create at least batch-size * threads = " + 
batchSize * getThreadNo());
+        }
         return null;
       });
       break;
-    case LIST_KEYS:
-      threadSeqId = getThreadSequenceId();
+    case LIST_KEYS_LIGHT:
       startKeyName = getPath(threadSeqId * batchSize);
       getMetrics().timer(operation.name()).time(() -> {
-        List<OmKeyInfo> keyInfoList =
-            ozoneManagerClient.listKeys(volumeName, bucketName, startKeyName,
-                "", batchSize).getKeys();
+        List<BasicOmKeyInfo> keyInfoList =
+            ozoneManagerClient.listKeysLight(volumeName, bucketName, 
startKeyName, "", batchSize).getKeys();
         if (keyInfoList.size() + 1 < batchSize) {
-          throw new NoSuchFileException(
-              "There are not enough files for testing you should use "
-                  + "CREATE_FILE to create at least batch-size * threads = "
-                  + batchSize * getThreadNo());
+          throw new NoSuchFileException("There are not enough keys for testing 
you should use "
+                  + "CREATE_KEY to create at least batch-size * threads = " + 
batchSize * getThreadNo());
         }
         return null;
       });
       break;
     case LIST_STATUS:
-      threadSeqId = getThreadSequenceId();
       startKeyName = getPath(threadSeqId * batchSize);
       keyArgs = omKeyArgsBuilder.get().setKeyName("").build();
       getMetrics().timer(operation.name()).time(() -> {
         List<OzoneFileStatus> fileStatusList = ozoneManagerClient.listStatus(
             keyArgs, false, startKeyName, batchSize);
         if (fileStatusList.size() + 1 < batchSize) {
-          throw new NoSuchFileException(
-              "There are not enough files for testing you should use "
-                  + "CREATE_FILE to create at least batch-size * threads = "
-                  + batchSize * getThreadNo());
+          throw new NoSuchFileException("There are not enough files for 
testing you should use "
+                  + "CREATE_FILE to create at least batch-size * threads = " + 
batchSize * getThreadNo());
         }
         return null;
       });
       break;
-    case INFO_BUCKET:
+    case LIST_STATUS_LIGHT:
+      startKeyName = getPath(threadSeqId * batchSize);
+      keyArgs = omKeyArgsBuilder.get().setKeyName("").build();
       getMetrics().timer(operation.name()).time(() -> {
-            try {
-              ozoneManagerClient.getBucketInfo(volumeName, bucketName);
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-          }
+        List<OzoneFileStatusLight> fileStatusList = 
ozoneManagerClient.listStatusLight(
+            keyArgs, false, startKeyName, batchSize, false);
+        if (fileStatusList.size() + 1 < batchSize) {
+          throw new NoSuchFileException("There are not enough files for 
testing you should use "
+              + "CREATE_FILE to create at least batch-size * threads = " + 
batchSize * getThreadNo());
+        }
+        return null;
+      });
+      break;
+    case INFO_BUCKET:
+      getMetrics().timer(operation.name()).time(() -> 
ozoneManagerClient.getBucketInfo(volumeName, bucketName)
       );
       break;
     case INFO_VOLUME:
-      getMetrics().timer(operation.name()).time(() -> {
-            try {
-              ozoneManagerClient.getVolumeInfo(volumeName);
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-          }
-      );
+      getMetrics().timer(operation.name()).time(() -> 
ozoneManagerClient.getVolumeInfo(volumeName));
       break;
     default:
       throw new IllegalStateException("Unrecognized write command " +
@@ -455,4 +435,30 @@ public class OmMetadataGenerator extends BaseFreonGenerator
     }
   }
 
+  @FunctionalInterface
+  interface WriteOperation {
+    OutputStream createStream() throws IOException;
+  }
+
+  @FunctionalInterface
+  interface ReadOperation {
+    InputStream createStream() throws IOException;
+  }
+
+  private Void performWriteOperation(WriteOperation writeOp, ContentGenerator 
contentGen) throws IOException {
+    try (OutputStream stream = writeOp.createStream()) {
+      contentGen.write(stream);
+    }
+    return null;
+  }
+
+  @SuppressWarnings("checkstyle:EmptyBlock")
+  private Void performReadOperation(ReadOperation readOp, byte[] buffer) 
throws IOException {
+    try (InputStream stream = readOp.createStream()) {
+      while (stream.read(buffer) >= 0) {
+      }
+      return null;
+    }
+  }
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to