|
@@ -33,11 +33,14 @@ import java.util.Date;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
import java.util.TimeZone;
|
|
import java.util.TimeZone;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
|
|
+import java.util.concurrent.CompletableFuture;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.regex.Matcher;
|
|
import java.util.regex.Matcher;
|
|
import java.util.regex.Pattern;
|
|
import java.util.regex.Pattern;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
|
|
+import java.util.Collections;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Optional;
|
|
import java.util.Stack;
|
|
import java.util.Stack;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
|
|
|
|
@@ -61,6 +64,7 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.fs.PositionedReadable;
|
|
import org.apache.hadoop.fs.Seekable;
|
|
import org.apache.hadoop.fs.Seekable;
|
|
import org.apache.hadoop.fs.StreamCapabilities;
|
|
import org.apache.hadoop.fs.StreamCapabilities;
|
|
import org.apache.hadoop.fs.Syncable;
|
|
import org.apache.hadoop.fs.Syncable;
|
|
@@ -70,6 +74,8 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
|
|
import org.apache.hadoop.fs.azure.security.Constants;
|
|
import org.apache.hadoop.fs.azure.security.Constants;
|
|
import org.apache.hadoop.fs.azure.security.RemoteWasbDelegationTokenManager;
|
|
import org.apache.hadoop.fs.azure.security.RemoteWasbDelegationTokenManager;
|
|
import org.apache.hadoop.fs.azure.security.WasbDelegationTokenManager;
|
|
import org.apache.hadoop.fs.azure.security.WasbDelegationTokenManager;
|
|
|
|
+import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
|
|
|
|
+import org.apache.hadoop.fs.impl.OpenFileParameters;
|
|
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
|
|
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
@@ -79,6 +85,7 @@ import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
|
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
|
|
|
|
+import org.apache.hadoop.util.LambdaUtils;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
|
@@ -915,6 +922,43 @@ public class NativeAzureFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public int read(long position, byte[] buffer, int offset, int length)
|
|
|
|
+ throws IOException {
|
|
|
|
+ // SpotBugs reports bug type IS2_INCONSISTENT_SYNC here.
|
|
|
|
+ // This report is not valid here.
|
|
|
|
+ // 'this.in' is instance of BlockBlobInputStream and read(long, byte[], int, int)
|
|
|
|
+ // calls it's Super class method when 'fs.azure.block.blob.buffered.pread.disable'
|
|
|
|
+ // is configured false. Super class FSInputStream's implementation is having
|
|
|
|
+ // proper synchronization.
|
|
|
|
+ // When 'fs.azure.block.blob.buffered.pread.disable' is true, we want a lock free
|
|
|
|
+ // implementation of blob read. Here we don't use any of the InputStream's
|
|
|
|
+ // shared resource (buffer) and also don't change any cursor position etc.
|
|
|
|
+ // So its safe to go with unsynchronized way of read.
|
|
|
|
+ if (in instanceof PositionedReadable) {
|
|
|
|
+ try {
|
|
|
|
+ int result = ((PositionedReadable) this.in).read(position, buffer,
|
|
|
|
+ offset, length);
|
|
|
|
+ if (null != statistics && result > 0) {
|
|
|
|
+ statistics.incrementBytesRead(result);
|
|
|
|
+ }
|
|
|
|
+ return result;
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(e);
|
|
|
|
+ if (innerException instanceof StorageException) {
|
|
|
|
+ LOG.error("Encountered Storage Exception for read on Blob : {}"
|
|
|
|
+ + " Exception details: {} Error Code : {}",
|
|
|
|
+ key, e, ((StorageException) innerException).getErrorCode());
|
|
|
|
+ if (NativeAzureFileSystemHelper.isFileNotFoundException((StorageException) innerException)) {
|
|
|
|
+ throw new FileNotFoundException(String.format("%s is not found", key));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return super.read(position, buffer, offset, length);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public synchronized void close() throws IOException {
|
|
public synchronized void close() throws IOException {
|
|
if (!closed) {
|
|
if (!closed) {
|
|
@@ -3043,6 +3087,12 @@ public class NativeAzureFileSystem extends FileSystem {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public FSDataInputStream open(Path f, int bufferSize) throws FileNotFoundException, IOException {
|
|
public FSDataInputStream open(Path f, int bufferSize) throws FileNotFoundException, IOException {
|
|
|
|
+ return open(f, bufferSize, Optional.empty());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private FSDataInputStream open(Path f, int bufferSize,
|
|
|
|
+ Optional<Configuration> options)
|
|
|
|
+ throws FileNotFoundException, IOException {
|
|
|
|
|
|
LOG.debug("Opening file: {}", f.toString());
|
|
LOG.debug("Opening file: {}", f.toString());
|
|
|
|
|
|
@@ -3077,7 +3127,7 @@ public class NativeAzureFileSystem extends FileSystem {
|
|
|
|
|
|
InputStream inputStream;
|
|
InputStream inputStream;
|
|
try {
|
|
try {
|
|
- inputStream = store.retrieve(key);
|
|
|
|
|
|
+ inputStream = store.retrieve(key, 0, options);
|
|
} catch(Exception ex) {
|
|
} catch(Exception ex) {
|
|
Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(ex);
|
|
Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(ex);
|
|
|
|
|
|
@@ -3094,6 +3144,18 @@ public class NativeAzureFileSystem extends FileSystem {
|
|
new NativeAzureFsInputStream(inputStream, key, meta.getLen()), bufferSize));
|
|
new NativeAzureFsInputStream(inputStream, key, meta.getLen()), bufferSize));
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ protected CompletableFuture<FSDataInputStream> openFileWithOptions(Path path,
|
|
|
|
+ OpenFileParameters parameters) throws IOException {
|
|
|
|
+ AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
|
|
|
|
+ parameters.getMandatoryKeys(),
|
|
|
|
+ Collections.emptySet(),
|
|
|
|
+ "for " + path);
|
|
|
|
+ return LambdaUtils.eval(
|
|
|
|
+ new CompletableFuture<>(), () ->
|
|
|
|
+ open(path, parameters.getBufferSize(), Optional.of(parameters.getOptions())));
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public boolean rename(Path src, Path dst) throws FileNotFoundException, IOException {
|
|
public boolean rename(Path src, Path dst) throws FileNotFoundException, IOException {
|
|
|
|
|