1
0
Selaa lähdekoodia

HADOOP-16859: ABFS: Add unbuffer support to ABFS connector.

Contributed by Sahil Takiar
Sahil Takiar 5 vuotta sitten
vanhempi
commit
42dfd270a1

+ 49 - 20
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractUnbufferTest.java

@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.fs.contract;
 
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
-
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
@@ -34,21 +35,22 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 public abstract class AbstractContractUnbufferTest extends AbstractFSContractTestBase {
 
   private Path file;
+  private byte[] fileBytes;
 
   @Override
   public void setup() throws Exception {
     super.setup();
     skipIfUnsupported(SUPPORTS_UNBUFFER);
     file = path("unbufferFile");
-    createFile(getFileSystem(), file, true,
-            dataset(TEST_FILE_LEN, 0, 255));
+    fileBytes = dataset(TEST_FILE_LEN, 0, 255);
+    createFile(getFileSystem(), file, true, fileBytes);
   }
 
   @Test
   public void testUnbufferAfterRead() throws IOException {
     describe("unbuffer a file after a single read");
     try (FSDataInputStream stream = getFileSystem().open(file)) {
-      assertEquals(128, stream.read(new byte[128]));
+      validateFullFileContents(stream);
       unbuffer(stream);
     }
   }
@@ -58,15 +60,14 @@ public abstract class AbstractContractUnbufferTest extends AbstractFSContractTes
     describe("unbuffer a file before a read");
     try (FSDataInputStream stream = getFileSystem().open(file)) {
       unbuffer(stream);
-      assertEquals(128, stream.read(new byte[128]));
+      validateFullFileContents(stream);
     }
   }
 
   @Test
   public void testUnbufferEmptyFile() throws IOException {
     Path emptyFile = path("emptyUnbufferFile");
-    createFile(getFileSystem(), emptyFile, true,
-            dataset(TEST_FILE_LEN, 0, 255));
+    getFileSystem().create(emptyFile, true).close();
     describe("unbuffer an empty file");
     try (FSDataInputStream stream = getFileSystem().open(emptyFile)) {
       unbuffer(stream);
@@ -79,13 +80,15 @@ public abstract class AbstractContractUnbufferTest extends AbstractFSContractTes
     FSDataInputStream stream = null;
     try {
       stream = getFileSystem().open(file);
-      assertEquals(128, stream.read(new byte[128]));
+      validateFullFileContents(stream);
     } finally {
       if (stream != null) {
         stream.close();
       }
     }
-    unbuffer(stream);
+    if (stream != null) {
+      unbuffer(stream);
+    }
   }
 
   @Test
@@ -94,32 +97,58 @@ public abstract class AbstractContractUnbufferTest extends AbstractFSContractTes
     try (FSDataInputStream stream = getFileSystem().open(file)) {
       unbuffer(stream);
       unbuffer(stream);
-      assertEquals(128, stream.read(new byte[128]));
+      validateFullFileContents(stream);
       unbuffer(stream);
       unbuffer(stream);
     }
   }
 
-   @Test
+  @Test
   public void testUnbufferMultipleReads() throws IOException {
     describe("unbuffer a file multiple times");
     try (FSDataInputStream stream = getFileSystem().open(file)) {
       unbuffer(stream);
-      assertEquals(128, stream.read(new byte[128]));
+      validateFileContents(stream, TEST_FILE_LEN / 8, 0);
       unbuffer(stream);
-      assertEquals(128, stream.read(new byte[128]));
-      assertEquals(128, stream.read(new byte[128]));
+      validateFileContents(stream, TEST_FILE_LEN / 8, TEST_FILE_LEN / 8);
+      validateFileContents(stream, TEST_FILE_LEN / 4, TEST_FILE_LEN / 4);
       unbuffer(stream);
-      assertEquals(128, stream.read(new byte[128]));
-      assertEquals(128, stream.read(new byte[128]));
-      assertEquals(128, stream.read(new byte[128]));
+      validateFileContents(stream, TEST_FILE_LEN / 2, TEST_FILE_LEN / 2);
       unbuffer(stream);
+      assertEquals("stream should be at end of file", TEST_FILE_LEN,
+              stream.getPos());
     }
   }
 
   private void unbuffer(FSDataInputStream stream) throws IOException {
     long pos = stream.getPos();
     stream.unbuffer();
-    assertEquals(pos, stream.getPos());
+    assertEquals("unbuffer unexpectedly changed the stream position", pos,
+            stream.getPos());
+  }
+
+  protected void validateFullFileContents(FSDataInputStream stream)
+          throws IOException {
+    validateFileContents(stream, TEST_FILE_LEN, 0);
+  }
+
+  protected void validateFileContents(FSDataInputStream stream, int length,
+                                      int startIndex)
+          throws IOException {
+    byte[] streamData = new byte[length];
+    assertEquals("failed to read expected number of bytes from "
+            + "stream", length, stream.read(streamData));
+    byte[] validateFileBytes;
+    if (startIndex == 0 && length == fileBytes.length) {
+      validateFileBytes = fileBytes;
+    } else {
+      validateFileBytes = Arrays.copyOfRange(fileBytes, startIndex,
+              startIndex + length);
+    }
+    assertArrayEquals("invalid file contents", validateFileBytes, streamData);
+  }
+
+  protected Path getFile() {
+    return file;
   }
 }

+ 26 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java

@@ -25,16 +25,22 @@ import java.net.HttpURLConnection;
 
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 
+import static org.apache.hadoop.util.StringUtils.toLowerCase;
+
 /**
  * The AbfsInputStream for AbfsClient.
  */
-public class AbfsInputStream extends FSInputStream {
+public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
+        StreamCapabilities {
+
   private final AbfsClient client;
   private final Statistics statistics;
   private final String path;
@@ -390,4 +396,23 @@ public class AbfsInputStream extends FSInputStream {
   public boolean markSupported() {
     return false;
   }
+
+  @Override
+  public synchronized void unbuffer() {
+    buffer = null;
+    // Preserve the original position returned by getPos()
+    fCursor = fCursor - limit + bCursor;
+    fCursorAfterLastRead = -1;
+    bCursor = 0;
+    limit = 0;
+  }
+
+  @Override
+  public boolean hasCapability(String capability) {
+    return StreamCapabilities.UNBUFFER.equals(toLowerCase(capability));
+  }
+
+  byte[] getBuffer() {
+    return buffer;
+  }
 }

+ 77 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractUnbuffer.java

@@ -0,0 +1,77 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contract;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.contract.AbstractContractUnbufferTest;
+
+/**
+ * Contract test for unbuffer operation.
+ */
+public class ITestAbfsContractUnbuffer extends AbstractContractUnbufferTest {
+  private final boolean isSecure;
+  private final ABFSContractTestBinding binding;
+
+  public ITestAbfsContractUnbuffer() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
+  }
+
+  @Override
+  public void setup() throws Exception {
+    binding.setup();
+    super.setup();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return binding.getRawConfiguration();
+  }
+
+  @Override
+  protected AbfsFileSystemContract createContract(Configuration conf) {
+    return new AbfsFileSystemContract(conf, isSecure);
+  }
+
+  /**
+   * {@link org.apache.hadoop.fs.azurebfs.services.AbfsInputStream} does not
+   * allow calling {@link org.apache.hadoop.fs.Seekable#getPos()} on a closed
+   * stream, so this test needs to be overridden so that it does not call
+   * getPos() after the stream has been closed.
+   */
+  @Override
+  public void testUnbufferOnClosedFile() throws IOException {
+    describe("unbuffer a file before a read");
+    FSDataInputStream stream = null;
+    try {
+      stream = getFileSystem().open(getFile());
+      validateFullFileContents(stream);
+    } finally {
+      if (stream != null) {
+        stream.close();
+      }
+    }
+    if (stream != null) {
+      stream.unbuffer();
+    }
+  }
+}

+ 84 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsUnbuffer.java

@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+/**
+ * Integration test for calling
+ * {@link org.apache.hadoop.fs.CanUnbuffer#unbuffer} on {@link AbfsInputStream}.
+ * Validates that the underlying stream's buffer is null.
+ */
+public class ITestAbfsUnbuffer extends AbstractAbfsIntegrationTest {
+
+  private Path dest;
+
+  public ITestAbfsUnbuffer() throws Exception {
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    dest = path("ITestAbfsUnbuffer");
+
+    byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+    ContractTestUtils.writeDataset(getFileSystem(), dest, data, data.length,
+            16, true);
+  }
+
+  @Test
+  public void testUnbuffer() throws IOException {
+    // Open file, read half the data, and then call unbuffer
+    try (FSDataInputStream inputStream = getFileSystem().open(dest)) {
+      assertTrue("unexpected stream type "
+              + inputStream.getWrappedStream().getClass().getSimpleName(),
+              inputStream.getWrappedStream() instanceof AbfsInputStream);
+      readAndAssertBytesRead(inputStream, 8);
+      assertFalse("AbfsInputStream buffer should not be null",
+              isBufferNull(inputStream));
+      inputStream.unbuffer();
+
+      // Check the the underlying buffer is null
+      assertTrue("AbfsInputStream buffer should be null",
+              isBufferNull(inputStream));
+    }
+  }
+
+  private boolean isBufferNull(FSDataInputStream inputStream) {
+    return ((AbfsInputStream) inputStream.getWrappedStream()).getBuffer() == null;
+  }
+
+  /**
+   * Read the specified number of bytes from the given
+   * {@link FSDataInputStream} and assert that
+   * {@link FSDataInputStream#read(byte[])} read the specified number of bytes.
+   */
+  private static void readAndAssertBytesRead(FSDataInputStream inputStream,
+                                             int bytesToRead) throws IOException {
+    assertEquals("AbfsInputStream#read did not read the correct number of "
+            + "bytes", bytesToRead, inputStream.read(new byte[bytesToRead]));
+  }
+}

+ 5 - 0
hadoop-tools/hadoop-azure/src/test/resources/abfs.xml

@@ -61,4 +61,9 @@
         <name>fs.contract.supports-getfilestatus</name>
         <value>true</value>
     </property>
+
+    <property>
+        <name>fs.contract.supports-unbuffer</name>
+        <value>true</value>
+    </property>
 </configuration>