hadoop-ozone-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From adorosz...@apache.org
Subject [hadoop-ozone] branch master updated: HDDS-2750. OzoneFSInputStream to support StreamCapabilities (#379)
Date Wed, 15 Jan 2020 11:27:33 GMT
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/hadoop-ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new becceb6  HDDS-2750. OzoneFSInputStream to support StreamCapabilities (#379)
becceb6 is described below

commit becceb6fe551e1ae9de7f44564f3d9cb98c7d444
Author: Yi Sheng <lianp964@gmail.com>
AuthorDate: Wed Jan 15 19:27:25 2020 +0800

    HDDS-2750. OzoneFSInputStream to support StreamCapabilities (#379)
---
 .../hadoop/fs/ozone/BasicOzoneFileSystem.java      |  9 +++--
 .../hadoop/fs/ozone/CapableOzoneFSInputStream.java | 42 ++++++++++++++++++++++
 .../apache/hadoop/fs/ozone/OzoneFSInputStream.java |  2 +-
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java    |  6 ++++
 .../hadoop/fs/ozone/OzoneStreamCapabilities.java   | 38 ++++++++++++++++++++
 .../hadoop/fs/ozone/TestOzoneFSInputStream.java    | 12 +++++++
 6 files changed, 106 insertions(+), 3 deletions(-)

diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java
b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java
index cac6660..b2cc9af 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.ozone;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.EnumSet;
@@ -215,8 +216,12 @@ public class BasicOzoneFileSystem extends FileSystem {
     statistics.incrementReadOps(1);
     LOG.trace("open() path:{}", f);
     final String key = pathToKey(f);
-    return new FSDataInputStream(
-        new OzoneFSInputStream(adapter.readFile(key), statistics));
+    InputStream inputStream = adapter.readFile(key);
+    return new FSDataInputStream(createFSInputStream(inputStream));
+  }
+
+  protected InputStream createFSInputStream(InputStream inputStream) {
+    return new OzoneFSInputStream(inputStream, statistics);
   }
 
   protected void incrementCounter(Statistic statistic) {
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java
b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java
new file mode 100644
index 0000000..cef6a58
--- /dev/null
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java
@@ -0,0 +1,42 @@
+/**
+ * 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.fs.ozone;
+
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.InputStream;
+
+final class CapableOzoneFSInputStream extends OzoneFSInputStream
+    implements StreamCapabilities {
+
+  CapableOzoneFSInputStream(InputStream inputStream, Statistics statistics) {
+    super(inputStream, statistics);
+  }
+
+  @Override
+  public boolean hasCapability(String capability) {
+    switch (StringUtils.toLowerCase(capability)) {
+    case OzoneStreamCapabilities.READBYTEBUFFER:
+      return true;
+    default:
+      return false;
+    }
+  }
+}
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java
b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java
index 9ffd7fe..d7a413d 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.fs.Seekable;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public final class OzoneFSInputStream extends FSInputStream
+public class OzoneFSInputStream extends FSInputStream
     implements ByteBufferReadable {
 
   private final InputStream inputStream;
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
index 0514bd7..ffda690 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.ozone;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.URI;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -103,4 +104,9 @@ public class OzoneFileSystem extends BasicOzoneFileSystem
           volumeStr, bucketStr, storageStatistics);
     }
   }
+
+  @Override
+  protected InputStream createFSInputStream(InputStream inputStream) {
+    return new CapableOzoneFSInputStream(inputStream, statistics);
+  }
 }
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneStreamCapabilities.java
b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneStreamCapabilities.java
new file mode 100644
index 0000000..db90cd9
--- /dev/null
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneStreamCapabilities.java
@@ -0,0 +1,38 @@
+/**
+ * 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.fs.ozone;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Utility class to query streams for supported capabilities of Ozone.
+ * Capability strings must be in lower case.
+ */
+final class OzoneStreamCapabilities {
+
+  private OzoneStreamCapabilities() {
+  }
+
+  /**
+   * Stream read(ByteBuffer) capability implemented by
+   * {@link OzoneFSInputStream#read(ByteBuffer)}.
+   *
+   * TODO: If Hadoop dependency is upgraded, this string can be removed.
+   */
+  static final String READBYTEBUFFER = "in:readbytebuffer";
+}
diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
index f06f907..5e14abd 100644
--- a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
+++ b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
@@ -32,6 +32,7 @@ import java.util.function.IntFunction;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests for {@link OzoneFSInputStream}.
@@ -113,6 +114,17 @@ public class TestOzoneFSInputStream {
     }
   }
 
+  @Test
+  public void testStreamCapability() {
+    final OzoneFSInputStream subject = createTestSubject(emptyStream());
+    final CapableOzoneFSInputStream capableOzoneFSInputStream =
+        new CapableOzoneFSInputStream(subject,
+            new FileSystem.Statistics("test"));
+
+    assertTrue(capableOzoneFSInputStream.
+        hasCapability(OzoneStreamCapabilities.READBYTEBUFFER));
+  }
+
   private static OzoneFSInputStream createTestSubject(InputStream input) {
     return new OzoneFSInputStream(input,
         new FileSystem.Statistics("test"));


---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-commits-help@hadoop.apache.org


Mime
View raw message