Skip to content

HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers #5117

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Show file tree
Hide file tree
Changes from all 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 @@ -421,6 +421,36 @@ public final class StreamStatisticNames {
public static final String STREAM_READ_PREFETCH_OPERATIONS
= "stream_read_prefetch_operations";

/**
* Count of prefetch blocks used.
*/
public static final String STREAM_READ_PREFETCH_BLOCKS_USED
= "stream_read_prefetch_blocks_used";
/**
* Count of prefetch bytes used.
*/
public static final String STREAM_READ_PREFETCH_BYTES_USED
= "stream_read_prefetch_bytes_used";

/**
* Count of prefetch blocks discarded unused: {@value}.
*/
public static final String STREAM_READ_PREFETCH_BLOCKS_DISCARDED
= "stream_read_prefetch_blocks_discarded";

/**
* Count of prefetch bytes discarded from unused blocks: {@value}.
* May or may not include bytes from blocks which were partially accessed.
*/
public static final String STREAM_READ_PREFETCH_BYTES_DISCARDED
= "stream_read_prefetch_bytes_discarded";

/**
* Count of prefetch blocks evicted, used or unused: {@value}.
*/
public static final String STREAM_READ_PREFETCH_BLOCKS_EVICTED
= "stream_read_prefetch_blocks_evicted";

/**
* Total number of block in disk cache.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import java.util.Hashtable;
import java.util.List;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.Map;
import java.util.Optional;
Expand All @@ -46,6 +45,7 @@
import javax.annotation.Nullable;

import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
import org.apache.hadoop.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -291,7 +291,7 @@ protected CompletableFuture<FSDataInputStream> openFileWithOptions(
LOG.debug("AzureBlobFileSystem.openFileWithOptions path: {}", path);
AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
parameters.getMandatoryKeys(),
Collections.emptySet(),
ConfigurationKeys.KNOWN_OPENFILE_KEYS,
"for " + path);
return LambdaUtils.eval(
new CompletableFuture<>(), () ->
Expand Down Expand Up @@ -1628,6 +1628,7 @@ public boolean hasPathCapability(final Path path, final String capability)
case CommonPathCapabilities.FS_PERMISSIONS:
case CommonPathCapabilities.FS_APPEND:
case CommonPathCapabilities.ETAGS_AVAILABLE:
case ConfigurationKeys.FS_AZURE_CAPABILITY_PREFETCH_SAFE: // safe from buffer sharing:
return true;

case CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,17 @@

package org.apache.hadoop.fs.azurebfs.constants;

import java.util.Collections;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;

import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;

/**
* Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file.
*/
Expand Down Expand Up @@ -259,5 +266,27 @@ public static String accountProperty(String property, String account) {
* @see FileSystem#openFile(org.apache.hadoop.fs.Path)
*/
public static final String FS_AZURE_BUFFERED_PREAD_DISABLE = "fs.azure.buffered.pread.disable";

/**
* Has the ReadBufferManager fix of HADOOP-18521 been applied?
* This can be queried on {@code hasCapability()} and
* on the filesystem {@code hasPathCapability()} probes.
*/
public static final String FS_AZURE_CAPABILITY_PREFETCH_SAFE = "fs.azure.capability.prefetch.safe";

/**
* Known keys for openFile(), including the standard ones.
*/
public static final Set<String> KNOWN_OPENFILE_KEYS;

static {
Set<String> collect = Stream.of(
FS_AZURE_CAPABILITY_PREFETCH_SAFE,
FS_AZURE_BUFFERED_PREAD_DISABLE)
.collect(Collectors.toSet());
collect.addAll(FS_OPTION_OPENFILE_STANDARD_OPTIONS);
KNOWN_OPENFILE_KEYS = Collections.unmodifiableSet(collect);
}

private ConfigurationKeys() {}
}
Loading