|
@@ -19,31 +19,40 @@
|
|
|
package org.apache.hadoop.fs.azurebfs.services;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-
|
|
|
-import org.junit.Assert;
|
|
|
-import org.junit.Test;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.Optional;
|
|
|
+import java.util.Random;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
|
|
|
import org.assertj.core.api.Assertions;
|
|
|
+import org.junit.Assert;
|
|
|
+import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
|
|
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
|
|
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
|
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
|
|
|
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
|
|
|
import org.apache.hadoop.fs.azurebfs.utils.TestCachedSASToken;
|
|
|
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
|
|
+import org.apache.hadoop.fs.impl.OpenFileParameters;
|
|
|
|
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
|
+import static org.mockito.ArgumentMatchers.anyBoolean;
|
|
|
+import static org.mockito.ArgumentMatchers.anyString;
|
|
|
import static org.mockito.Mockito.doReturn;
|
|
|
import static org.mockito.Mockito.doThrow;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.spy;
|
|
|
import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -192,6 +201,106 @@ public class TestAbfsInputStream extends
|
|
|
ReadBufferManager.getBufferManager().setThresholdAgeMilliseconds(REDUCED_READ_BUFFER_AGE_THRESHOLD);
|
|
|
}
|
|
|
|
|
|
+ private void writeBufferToNewFile(Path testFile, byte[] buffer) throws IOException {
|
|
|
+ AzureBlobFileSystem fs = getFileSystem();
|
|
|
+ fs.create(testFile);
|
|
|
+ FSDataOutputStream out = fs.append(testFile);
|
|
|
+ out.write(buffer);
|
|
|
+ out.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyOpenWithProvidedStatus(Path path, FileStatus fileStatus,
|
|
|
+ byte[] buf, AbfsRestOperationType source)
|
|
|
+ throws IOException, ExecutionException, InterruptedException {
|
|
|
+ byte[] readBuf = new byte[buf.length];
|
|
|
+ AzureBlobFileSystem fs = getFileSystem();
|
|
|
+ FutureDataInputStreamBuilder builder = fs.openFile(path);
|
|
|
+ builder.withFileStatus(fileStatus);
|
|
|
+ FSDataInputStream in = builder.build().get();
|
|
|
+ assertEquals(String.format(
|
|
|
+ "Open with fileStatus [from %s result]: Incorrect number of bytes read",
|
|
|
+ source), buf.length, in.read(readBuf));
|
|
|
+ assertArrayEquals(String
|
|
|
+ .format("Open with fileStatus [from %s result]: Incorrect read data",
|
|
|
+ source), readBuf, buf);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkGetPathStatusCalls(Path testFile, FileStatus fileStatus,
|
|
|
+ AzureBlobFileSystemStore abfsStore, AbfsClient mockClient,
|
|
|
+ AbfsRestOperationType source, TracingContext tracingContext)
|
|
|
+ throws IOException {
|
|
|
+
|
|
|
+ // verify GetPathStatus not invoked when FileStatus is provided
|
|
|
+ abfsStore.openFileForRead(testFile, Optional
|
|
|
+ .ofNullable(new OpenFileParameters().withStatus(fileStatus)), null, tracingContext);
|
|
|
+ verify(mockClient, times(0).description((String.format(
|
|
|
+ "FileStatus [from %s result] provided, GetFileStatus should not be invoked",
|
|
|
+ source)))).getPathStatus(anyString(), anyBoolean(), any(TracingContext.class));
|
|
|
+
|
|
|
+ // verify GetPathStatus invoked when FileStatus not provided
|
|
|
+ abfsStore.openFileForRead(testFile,
|
|
|
+ Optional.empty(), null,
|
|
|
+ tracingContext);
|
|
|
+ verify(mockClient, times(1).description(
|
|
|
+ "GetPathStatus should be invoked when FileStatus not provided"))
|
|
|
+ .getPathStatus(anyString(), anyBoolean(), any(TracingContext.class));
|
|
|
+
|
|
|
+ Mockito.reset(mockClient); //clears invocation count for next test case
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testOpenFileWithOptions() throws Exception {
|
|
|
+ AzureBlobFileSystem fs = getFileSystem();
|
|
|
+ String testFolder = "/testFolder";
|
|
|
+ Path smallTestFile = new Path(testFolder + "/testFile0");
|
|
|
+ Path largeTestFile = new Path(testFolder + "/testFile1");
|
|
|
+ fs.mkdirs(new Path(testFolder));
|
|
|
+ int readBufferSize = getConfiguration().getReadBufferSize();
|
|
|
+ byte[] smallBuffer = new byte[5];
|
|
|
+ byte[] largeBuffer = new byte[readBufferSize + 5];
|
|
|
+ new Random().nextBytes(smallBuffer);
|
|
|
+ new Random().nextBytes(largeBuffer);
|
|
|
+ writeBufferToNewFile(smallTestFile, smallBuffer);
|
|
|
+ writeBufferToNewFile(largeTestFile, largeBuffer);
|
|
|
+
|
|
|
+ FileStatus[] getFileStatusResults = {fs.getFileStatus(smallTestFile),
|
|
|
+ fs.getFileStatus(largeTestFile)};
|
|
|
+ FileStatus[] listStatusResults = fs.listStatus(new Path(testFolder));
|
|
|
+
|
|
|
+ // open with fileStatus from GetPathStatus
|
|
|
+ verifyOpenWithProvidedStatus(smallTestFile, getFileStatusResults[0],
|
|
|
+ smallBuffer, AbfsRestOperationType.GetPathStatus);
|
|
|
+ verifyOpenWithProvidedStatus(largeTestFile, getFileStatusResults[1],
|
|
|
+ largeBuffer, AbfsRestOperationType.GetPathStatus);
|
|
|
+
|
|
|
+ // open with fileStatus from ListStatus
|
|
|
+ verifyOpenWithProvidedStatus(smallTestFile, listStatusResults[0], smallBuffer,
|
|
|
+ AbfsRestOperationType.ListPaths);
|
|
|
+ verifyOpenWithProvidedStatus(largeTestFile, listStatusResults[1], largeBuffer,
|
|
|
+ AbfsRestOperationType.ListPaths);
|
|
|
+
|
|
|
+ // verify number of GetPathStatus invocations
|
|
|
+ AzureBlobFileSystemStore abfsStore = getAbfsStore(fs);
|
|
|
+ AbfsClient mockClient = spy(getAbfsClient(abfsStore));
|
|
|
+ setAbfsClient(abfsStore, mockClient);
|
|
|
+ TracingContext tracingContext = getTestTracingContext(fs, false);
|
|
|
+ checkGetPathStatusCalls(smallTestFile, getFileStatusResults[0],
|
|
|
+ abfsStore, mockClient, AbfsRestOperationType.GetPathStatus, tracingContext);
|
|
|
+ checkGetPathStatusCalls(largeTestFile, getFileStatusResults[1],
|
|
|
+ abfsStore, mockClient, AbfsRestOperationType.GetPathStatus, tracingContext);
|
|
|
+ checkGetPathStatusCalls(smallTestFile, listStatusResults[0],
|
|
|
+ abfsStore, mockClient, AbfsRestOperationType.ListPaths, tracingContext);
|
|
|
+ checkGetPathStatusCalls(largeTestFile, listStatusResults[1],
|
|
|
+ abfsStore, mockClient, AbfsRestOperationType.ListPaths, tracingContext);
|
|
|
+
|
|
|
+ // Verify with incorrect filestatus
|
|
|
+ getFileStatusResults[0].setPath(new Path("wrongPath"));
|
|
|
+ intercept(ExecutionException.class,
|
|
|
+ () -> verifyOpenWithProvidedStatus(smallTestFile,
|
|
|
+ getFileStatusResults[0], smallBuffer,
|
|
|
+ AbfsRestOperationType.GetPathStatus));
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* This test expects AbfsInputStream to throw the exception that readAhead
|
|
|
* thread received on read. The readAhead thread must be initiated from the
|