Skip to content
Merged
Show file tree
Hide file tree
Changes from 10 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,10 @@ public class AbfsConfiguration{
DefaultValue = DEFAULT_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS_IN_SECONDS)
private long sasTokenRenewPeriodForStreamsInSeconds;

@BooleanConfigurationValidatorAnnotation(ConfigurationKey =
FS_AZURE_ENABLE_FLUSH, DefaultValue = DEFAULT_ENABLE_ABFS_LIST_ITERATOR)
private boolean enableAbfsListIterator;

public AbfsConfiguration(final Configuration rawConfig, String accountName)
throws IllegalAccessException, InvalidConfigurationValueException, IOException {
this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders(
Expand Down Expand Up @@ -896,6 +900,10 @@ public int getMaxWriteRequestsToQueue() {
return this.maxWriteRequestsToQueue;
}

public boolean enableAbfsListIterator() {
return this.enableAbfsListIterator;
}

@VisibleForTesting
void setReadBufferSize(int bufferSize) {
this.readBufferSize = bufferSize;
Expand Down Expand Up @@ -961,4 +969,9 @@ public void setOptimizeFooterRead(boolean optimizeFooterRead) {
this.optimizeFooterRead = optimizeFooterRead;
}

@VisibleForTesting
public void setEnableAbfsListIterator(boolean enableAbfsListIterator) {
this.enableAbfsListIterator = enableAbfsListIterator;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,8 @@
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
Expand Down Expand Up @@ -78,6 +80,7 @@
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.functional.RemoteIterators;
import org.apache.hadoop.util.Progressable;

import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*;
Expand Down Expand Up @@ -982,6 +985,19 @@ public boolean exists(Path f) throws IOException {
return super.exists(f);
}

@Override
public RemoteIterator<FileStatus> listStatusIterator(Path path)
throws IOException {
LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path);
if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) {
AbfsListStatusRemoteIterator abfsLsItr =
new AbfsListStatusRemoteIterator(getFileStatus(path), abfsStore);
return RemoteIterators.typeCastingRemoteIterator(abfsLsItr);
} else {
return super.listStatusIterator(path);
}
}

private FileStatus tryGetFileStatus(final Path f) {
try {
return getFileStatus(f);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,7 @@
import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials;
import org.apache.hadoop.fs.azurebfs.services.AbfsPerfTracker;
import org.apache.hadoop.fs.azurebfs.services.AbfsPerfInfo;
import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
import org.apache.hadoop.fs.azurebfs.utils.Base64;
import org.apache.hadoop.fs.azurebfs.utils.CRC64;
import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils;
Expand Down Expand Up @@ -131,7 +132,7 @@
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class AzureBlobFileSystemStore implements Closeable {
public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class);

private AbfsClient client;
Expand Down Expand Up @@ -838,6 +839,7 @@ public FileStatus getFileStatus(final Path path) throws IOException {
* @param path The list path.
* @return the entries in the path.
* */
@Override
public FileStatus[] listStatus(final Path path) throws IOException {
return listStatus(path, null);
}
Expand All @@ -854,7 +856,17 @@ public FileStatus[] listStatus(final Path path) throws IOException {
* @return the entries in the path start from "startFrom" in lexical order.
* */
@InterfaceStability.Unstable
@Override
public FileStatus[] listStatus(final Path path, final String startFrom) throws IOException {
List<FileStatus> fileStatuses = new ArrayList<>();
listStatus(path, startFrom, fileStatuses, true, null);
return fileStatuses.toArray(new FileStatus[fileStatuses.size()]);
}

@Override
public String listStatus(final Path path, final String startFrom,
List<FileStatus> fileStatuses, final boolean fetchAll,
String continuation) throws IOException {
final Instant startAggregate = abfsPerfTracker.getLatencyInstant();
long countAggregate = 0;
boolean shouldContinue = true;
Expand All @@ -865,16 +877,16 @@ public FileStatus[] listStatus(final Path path, final String startFrom) throws I
startFrom);

final String relativePath = getRelativePath(path);
String continuation = null;

// generate continuation token if a valid startFrom is provided.
if (startFrom != null && !startFrom.isEmpty()) {
continuation = getIsNamespaceEnabled()
? generateContinuationTokenForXns(startFrom)
: generateContinuationTokenForNonXns(relativePath, startFrom);
if (continuation == null || continuation.isEmpty()) {
// generate continuation token if a valid startFrom is provided.
if (startFrom != null && !startFrom.isEmpty()) {
continuation = getIsNamespaceEnabled()
? generateContinuationTokenForXns(startFrom)
: generateContinuationTokenForNonXns(relativePath, startFrom);
}
}

ArrayList<FileStatus> fileStatuses = new ArrayList<>();
do {
try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) {
AbfsRestOperation op = client.listPath(relativePath, false,
Expand Down Expand Up @@ -928,15 +940,16 @@ public FileStatus[] listStatus(final Path path, final String startFrom) throws I

perfInfo.registerSuccess(true);
countAggregate++;
shouldContinue = continuation != null && !continuation.isEmpty();
shouldContinue =
fetchAll && continuation != null && !continuation.isEmpty();

if (!shouldContinue) {
perfInfo.registerAggregates(startAggregate, countAggregate);
}
}
} while (shouldContinue);

return fileStatuses.toArray(new FileStatus[fileStatuses.size()]);
return continuation;
}

// generate continuation token for xns account
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,8 @@ public final class ConfigurationKeys {
public static final String FS_AZURE_SKIP_SUPER_USER_REPLACEMENT = "fs.azure.identity.transformer.skip.superuser.replacement";
public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER = "fs.azure.account.keyprovider";
public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
/** Setting this true will make the driver use it's own RemoteIterator implementation */
public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator";

/** End point of ABFS account: {@value}. */
public static final String AZURE_ABFS_ENDPOINT = "fs.azure.abfs.endpoint";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,5 +101,7 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_DELETE_CONSIDERED_IDEMPOTENT = true;
public static final int DEFAULT_CLOCK_SKEW_WITH_SERVER_IN_MS = 5 * 60 * 1000; // 5 mins

public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;

private FileSystemConfigurations() {}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/**
* 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.azurebfs.services;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.CompletableFuture;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.RemoteIterator;

public class AbfsListStatusRemoteIterator implements RemoteIterator<FileStatus> {

private static final Logger LOG = LoggerFactory
.getLogger(AbfsListStatusRemoteIterator.class);

private static final boolean FETCH_ALL_FALSE = false;
private static final int MAX_QUEUE_SIZE = 10;

private final FileStatus fileStatus;
private final ListingSupport listingSupport;
private final ArrayBlockingQueue<Iterator<FileStatus>> iteratorsQueue;
private final Object asyncOpLock = new Object();

private volatile boolean isAsyncInProgress = false;
private boolean firstBatch = true;
private String continuation;
private Iterator<FileStatus> currIterator;
private IOException ioException;

public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
final ListingSupport listingSupport) {
this.fileStatus = fileStatus;
this.listingSupport = listingSupport;
iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
currIterator = Collections.emptyIterator();
fetchBatchesAsync();
}

@Override
public boolean hasNext() throws IOException {
if (currIterator.hasNext()) {
return true;
}
updateCurrentIterator();
return currIterator.hasNext();
}

@Override
public FileStatus next() throws IOException {
if (!this.hasNext()) {
throw new NoSuchElementException();
}
return currIterator.next();
}

private void updateCurrentIterator() throws IOException {
fetchBatchesAsync();
synchronized (this) {
if (iteratorsQueue.isEmpty()) {
if (ioException != null) {
throw ioException;
}
if (isListingComplete()) {
return;
}
}
}
try {
currIterator = iteratorsQueue.take();
if (!currIterator.hasNext() && !isListingComplete()) {
updateCurrentIterator();
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.error("Thread got interrupted: {}", e);
}
}

private synchronized boolean isListingComplete() {
return !firstBatch && (continuation == null || continuation.isEmpty());
}

private void fetchBatchesAsync() {
if (isAsyncInProgress) {
return;
}
synchronized (asyncOpLock) {
if (isAsyncInProgress) {
return;
}
isAsyncInProgress = true;
}
CompletableFuture.runAsync(() -> asyncOp());
}

private void asyncOp() {
try {
while (!isListingComplete() && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
addNextBatchIteratorToQueue();
}
} catch (IOException e) {
ioException = e;
iteratorsQueue.offer(Collections.emptyIterator());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.error("Thread got interrupted: {}", e);
} finally {
synchronized (asyncOpLock) {
isAsyncInProgress = false;
}
}
}

private synchronized void addNextBatchIteratorToQueue()
throws IOException, InterruptedException {
List<FileStatus> fileStatuses = new ArrayList<>();
continuation = listingSupport
.listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
continuation);
iteratorsQueue.put(fileStatuses.iterator());
if (firstBatch) {
firstBatch = false;
}
}

}
Loading