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 a01676a9af HDDS-8029. [hsync] Outputstream in encrypted buckets do not
return the correct stream capabilities. (#4316)
a01676a9af is described below
commit a01676a9afccf4d7c4c80066d999a087973a9def
Author: Wei-Chiu Chuang <[email protected]>
AuthorDate: Fri Mar 3 11:14:41 2023 -0800
HDDS-8029. [hsync] Outputstream in encrypted buckets do not return the
correct stream capabilities. (#4316)
---
.../java/org/apache/hadoop/fs/ozone/TestHSync.java | 41 ++++++++++++++++++++++
.../fs/ozone/CapableOzoneFSOutputStream.java | 13 ++++++-
2 files changed, 53 insertions(+), 1 deletion(-)
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
index 3e5ba5aa32..1985449e49 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
@@ -20,9 +20,14 @@ package org.apache.hadoop.fs.ozone;
import java.io.Closeable;
import java.io.IOException;
+import java.security.GeneralSecurityException;
import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.crypto.Encryptor;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -38,6 +43,9 @@ import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.TestDataUtil;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.KeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.ozone.test.tag.Flaky;
@@ -57,6 +65,8 @@ import static
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
/**
* Test HSync.
@@ -219,6 +229,8 @@ public class TestHSync {
assertTrue(os.hasCapability(StreamCapabilities.HSYNC),
"KeyOutputStream should support hsync()!");
}
+
+ testEncryptedStreamCapabilities(false);
}
@Test
@@ -251,5 +263,34 @@ public class TestHSync {
assertFalse(os.hasCapability(StreamCapabilities.HSYNC),
"ECKeyOutputStream should not support hsync()!");
}
+ testEncryptedStreamCapabilities(true);
+ }
+
+ private void testEncryptedStreamCapabilities(boolean isEC) throws
IOException,
+ GeneralSecurityException {
+ KeyOutputStream kos;
+ if (isEC) {
+ kos = mock(ECKeyOutputStream.class);
+ } else {
+ kos = mock(KeyOutputStream.class);
+ }
+ CryptoCodec codec = mock(CryptoCodec.class);
+ when(codec.getCipherSuite()).thenReturn(CipherSuite.AES_CTR_NOPADDING);
+ when(codec.getConf()).thenReturn(CONF);
+ Encryptor encryptor = mock(Encryptor.class);
+ when(codec.createEncryptor()).thenReturn(encryptor);
+ CryptoOutputStream cos =
+ new CryptoOutputStream(kos, codec, new byte[0], new byte[0]);
+ OzoneOutputStream oos = new OzoneOutputStream(cos);
+ OzoneFSOutputStream ofso = new OzoneFSOutputStream(oos);
+
+ try (CapableOzoneFSOutputStream cofsos =
+ new CapableOzoneFSOutputStream(ofso)) {
+ if (isEC) {
+ assertFalse(cofsos.hasCapability(StreamCapabilities.HFLUSH));
+ } else {
+ assertTrue(cofsos.hasCapability(StreamCapabilities.HFLUSH));
+ }
+ }
}
}
diff --git
a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSOutputStream.java
b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSOutputStream.java
index 7cc76fcb25..689fde7637 100644
---
a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSOutputStream.java
+++
b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSOutputStream.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.fs.ozone;
+import org.apache.hadoop.crypto.CryptoOutputStream;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
@@ -44,6 +45,16 @@ public class CapableOzoneFSOutputStream extends
OzoneFSOutputStream
@Override
public boolean hasCapability(String capability) {
OutputStream os = getWrappedOutputStream().getOutputStream();
+
+ if (os instanceof CryptoOutputStream) {
+ OutputStream wrapped = ((CryptoOutputStream) os).getWrappedStream();
+ return hasWrappedCapability(wrapped, capability);
+ }
+ return hasWrappedCapability(os, capability);
+ }
+
+ private static boolean hasWrappedCapability(OutputStream os,
+ String capability) {
if (os instanceof ECKeyOutputStream) {
return false;
} else if (os instanceof KeyOutputStream) {
@@ -55,7 +66,7 @@ public class CapableOzoneFSOutputStream extends
OzoneFSOutputStream
return false;
}
}
- // deal with CryptoOutputStream
+ // this is unexpected. try last resort
return StoreImplementationUtils.hasCapability(os, capability);
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]