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 7da2a57e1c HDDS-9131. [hsync] Add --sync option in `freon dfsg` to 
test hsync/hflush performance (#5225)
7da2a57e1c is described below

commit 7da2a57e1cb9d7393e4d686446d986289b12c757
Author: Wei-Chiu Chuang <[email protected]>
AuthorDate: Wed Oct 4 07:22:59 2023 -0700

    HDDS-9131. [hsync] Add --sync option in `freon dfsg` to test hsync/hflush 
performance (#5225)
---
 .../hadoop/ozone/freon/ContentGenerator.java       | 43 ++++++++++++++++++++
 .../hadoop/ozone/freon/HadoopFsGenerator.java      |  8 +++-
 .../hadoop/ozone/freon/TestContentGenerator.java   | 47 ++++++++++++++++++++--
 3 files changed, 93 insertions(+), 5 deletions(-)

diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ContentGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ContentGenerator.java
index b01c12f6b3..da16026210 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ContentGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ContentGenerator.java
@@ -23,6 +23,9 @@ import java.nio.charset.StandardCharsets;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.impl.StoreImplementationUtils;
 import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput;
 
 /**
@@ -51,6 +54,14 @@ public class ContentGenerator {
 
   private final byte[] buffer;
 
+  private SyncOptions flushOrSync;
+
+  enum SyncOptions {
+    NONE,
+    HFLUSH,
+    HSYNC
+  }
+
   ContentGenerator(long keySize, int bufferSize) {
     this(keySize, bufferSize, bufferSize);
   }
@@ -61,6 +72,13 @@ public class ContentGenerator {
     this.copyBufferSize = copyBufferSize;
     buffer = RandomStringUtils.randomAscii(bufferSize)
         .getBytes(StandardCharsets.UTF_8);
+    this.flushOrSync = SyncOptions.NONE;
+  }
+
+  ContentGenerator(long keySize, int bufferSize, int copyBufferSize,
+      SyncOptions flushOrSync) {
+    this(keySize, bufferSize, copyBufferSize);
+    this.flushOrSync = flushOrSync;
   }
 
   /**
@@ -73,16 +91,41 @@ public class ContentGenerator {
       if (copyBufferSize == 1) {
         for (int i = 0; i < curSize; i++) {
           outputStream.write(buffer[i]);
+          doFlushOrSync(outputStream);
         }
       } else {
         for (int i = 0; i < curSize; i += copyBufferSize) {
           outputStream.write(buffer, i,
               Math.min(copyBufferSize, curSize - i));
+          doFlushOrSync(outputStream);
         }
       }
     }
   }
 
+  private void doFlushOrSync(OutputStream outputStream) throws IOException {
+    switch (flushOrSync) {
+    case NONE:
+      // noop
+      break;
+    case HFLUSH:
+      if (StoreImplementationUtils.hasCapability(
+          outputStream, StreamCapabilities.HSYNC)) {
+        ((Syncable)outputStream).hflush();
+      }
+      break;
+    case HSYNC:
+      if (StoreImplementationUtils.hasCapability(
+          outputStream, StreamCapabilities.HSYNC)) {
+        ((Syncable)outputStream).hsync();
+      }
+      break;
+    default:
+      throw new IllegalArgumentException("Unsupported sync option"
+          + flushOrSync);
+    }
+  }
+
   /**
    * Write the required bytes to the streaming output stream.
    */
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java
index 755c57ac92..7dd66937db 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/HadoopFsGenerator.java
@@ -63,6 +63,12 @@ public class HadoopFsGenerator extends BaseFreonGenerator
       defaultValue = "4096")
   private int copyBufferSize;
 
+  @Option(names = {"--sync"},
+      description = "Type of operation to execute after a write. Supported " +
+      "options include NONE (default), HFLUSH and HSYNC",
+      defaultValue = "NONE")
+  private static ContentGenerator.SyncOptions flushOrSync;
+
   private ContentGenerator contentGenerator;
 
   private Timer timer;
@@ -89,7 +95,7 @@ public class HadoopFsGenerator extends BaseFreonGenerator
     }
 
     contentGenerator =
-        new ContentGenerator(fileSize, bufferSize, copyBufferSize);
+        new ContentGenerator(fileSize, bufferSize, copyBufferSize, 
flushOrSync);
 
     timer = getMetrics().timer("file-create");
 
diff --git 
a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestContentGenerator.java
 
b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestContentGenerator.java
index e8de083f04..c3d28021b5 100644
--- 
a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestContentGenerator.java
+++ 
b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestContentGenerator.java
@@ -1,7 +1,3 @@
-package org.apache.hadoop.ozone.freon;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with this
@@ -18,10 +14,21 @@ import java.io.IOException;
  * License for the specific language governing permissions and limitations 
under
  * the License.
  */
+package org.apache.hadoop.ozone.freon;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
+
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 /**
  * Tests for the ContentGenerator class of Freon.
  */
@@ -78,4 +85,36 @@ public class TestContentGenerator {
     System.arraycopy(buffer, 0, expected, 16, 4);
     Assert.assertArrayEquals(expected, output.toByteArray());
   }
+
+  @Test
+  public void writeWithHsync() throws IOException {
+    class SyncableByteArrayOutputStream extends ByteArrayOutputStream
+        implements Syncable, StreamCapabilities {
+      @Override
+      public void hflush() throws IOException {
+      }
+      @Override
+      public void hsync() throws IOException {
+      }
+
+      @Override
+      public boolean hasCapability(String capability) {
+        return true;
+      }
+    }
+    ContentGenerator generator = new ContentGenerator(20, 8, 3,
+        ContentGenerator.SyncOptions.HSYNC);
+
+    SyncableByteArrayOutputStream syncable =
+        new SyncableByteArrayOutputStream();
+    SyncableByteArrayOutputStream spySyncable = spy(syncable);
+
+    generator.write(spySyncable);
+    verify(spySyncable, times(8)).hsync();
+
+    generator = new ContentGenerator(20, 8, 3);
+    spySyncable = spy(syncable);
+    generator.write(spySyncable);
+    verify(spySyncable, times(0)).hsync();
+  }
 }


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

Reply via email to