This is an automated email from the ASF dual-hosted git repository.
weichiu 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 41d70590c0 HDDS-11322. [hsync] Block ECKeyOutputStream from calling
hsync and hflush (#7098)
41d70590c0 is described below
commit 41d70590c0e62f4bb0add068c2f6e708d60d0cf6
Author: Siyao Meng <[email protected]>
AuthorDate: Tue Aug 20 11:02:48 2024 -0700
HDDS-11322. [hsync] Block ECKeyOutputStream from calling hsync and hflush
(#7098)
---
.../hadoop/ozone/client/io/ECKeyOutputStream.java | 11 +++++++++++
.../client/rpc/AbstractTestECKeyOutputStream.java | 18 ++++++++++++++++++
2 files changed, 29 insertions(+)
diff --git
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
index 0cb3973e04..0de61f8485 100644
---
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
+++
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.client.io;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
@@ -420,6 +421,16 @@ public final class ECKeyOutputStream extends
KeyOutputStream
LOG.debug("ECKeyOutputStream does not support flush.");
}
+ @Override
+ public void hflush() {
+ throw new NotImplementedException("ECKeyOutputStream does not support
hflush.");
+ }
+
+ @Override
+ public void hsync() {
+ throw new NotImplementedException("ECKeyOutputStream does not support
hsync.");
+ }
+
private void closeCurrentStreamEntry()
throws IOException {
final ECBlockOutputStreamEntryPool blockOutputStreamEntryPool =
getBlockOutputStreamEntryPool();
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
index b40b0bbcc6..766ed09bcc 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
@@ -16,6 +16,7 @@
*/
package org.apache.hadoop.ozone.client.rpc;
+import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
@@ -67,6 +68,7 @@ import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.fail;
/**
@@ -123,6 +125,8 @@ abstract class AbstractTestECKeyOutputStream {
conf.setBoolean(OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED,
zeroCopyEnabled);
conf.setInt(ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT, 10);
+ // "Enable" hsync to verify that hsync would be blocked by
ECKeyOutputStream
+ conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
.setBlockSize(blockSize)
@@ -469,4 +473,18 @@ abstract class AbstractTestECKeyOutputStream {
return inputData;
}
+ @Test
+ public void testBlockedHflushAndHsync() throws Exception {
+ // Expect ECKeyOutputStream hflush and hsync calls to throw exception
+ try (OzoneOutputStream oOut = TestHelper.createKey(
+ keyString, new ECReplicationConfig(3, 2,
ECReplicationConfig.EcCodec.RS, chunkSize),
+ inputSize, objectStore, volumeName, bucketName)) {
+ assertInstanceOf(ECKeyOutputStream.class, oOut.getOutputStream());
+ KeyOutputStream kOut = (KeyOutputStream) oOut.getOutputStream();
+
+ assertThrows(NotImplementedException.class, () -> kOut.hflush());
+ assertThrows(NotImplementedException.class, () -> kOut.hsync());
+ }
+ }
+
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]