HADOOP-19131. Assist reflection IO with WrappedOperations class (#6686)

1. The class WrappedIO has been extended with more filesystem operations

- openFile()
- PathCapabilities
- StreamCapabilities
- ByteBufferPositionedReadable

All these static methods raise UncheckedIOExceptions rather than
checked ones.

2. The adjacent class org.apache.hadoop.io.wrappedio.WrappedStatistics
provides similar access to IOStatistics/IOStatisticsContext classes
and operations.

Allows callers to:
* Get a serializable IOStatisticsSnapshot from an IOStatisticsSource or
  IOStatistics instance
* Save an IOStatisticsSnapshot to file
* Convert an IOStatisticsSnapshot to JSON
* Given an object which may be an IOStatisticsSource, return an object
  whose toString() value is a dynamically generated, human readable summary.
  This is for logging.
* Separate getters to the different sections of IOStatistics.
* Mean values are returned as a Map.Pair<Long, Long> of (samples, sum)
  from which means may be calculated.

There are examples of the dynamic bindings to these classes in:

org.apache.hadoop.io.wrappedio.impl.DynamicWrappedIO
org.apache.hadoop.io.wrappedio.impl.DynamicWrappedStatistics

These use DynMethods and other classes in the package
org.apache.hadoop.util.dynamic which are based on the
Apache Parquet equivalents.
This makes re-implementing these in that library and others
which their own fork of the classes (example: Apache Iceberg)

3. The openFile() option "fs.option.openfile.read.policy" has
added specific file format policies for the core filetypes

* avro
* columnar
* csv
* hbase
* json
* orc
* parquet

S3A chooses the appropriate sequential/random policy as a 

A policy `parquet, columnar, vector, random, adaptive` will use the parquet policy for
any filesystem aware of it, falling back to the first entry in the list which
the specific version of the filesystem recognizes

4. New Path capability fs.capability.virtual.block.locations

Indicates that locations are generated client side
and don't refer to real hosts.

Contributed by Steve Loughran
This commit is contained in:
Steve Loughran 2024-08-14 14:43:00 +01:00 committed by GitHub
parent fa83c9a805
commit 55a576906d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
40 changed files with 4895 additions and 48 deletions

View File

@ -454,4 +454,10 @@
<Class name="org.apache.hadoop.ipc.internal.ShadedProtobufHelper" />
<Bug pattern="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION" />
</Match>
<!-- class cast after an assignableFrom check. -->
<Match>
<Class name="org.apache.hadoop.util.dynamic.DynMethods" />
<Bug pattern="BC_UNCONFIRMED_CAST" />
</Match>
</FindBugsFilter>

View File

@ -187,4 +187,20 @@ private CommonPathCapabilities() {
*/
public static final String BULK_DELETE = "fs.capability.bulk.delete";
/**
* Capability string to probe for block locations returned in {@code LocatedFileStatus}
* instances from calls such as {@code getBlockLocations()} and {@code listStatus()}l
* to be 'virtual' rather than actual values resolved against a Distributed Filesystem including
* HDFS: {@value}.
* <p>
* Key implications from this path capability being true:
* <ol>
* <li>Work can be scheduled anywhere</li>
* <li>Creation of the location list is a low cost-client side operation</li>
* </ol>
* Implication #2 means there is no performance penalty from use of FileSystem operations which
* return lists or iterators of {@code LocatedFileStatus}.
*/
public static final String VIRTUAL_BLOCK_LOCATIONS = "fs.capability.virtual.block.locations";
}

View File

@ -262,6 +262,14 @@ public int read(long position, ByteBuffer buf) throws IOException {
"by " + in.getClass().getCanonicalName());
}
/**
* Delegate to the underlying stream.
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* @throws IOException on a failure from the nested stream.
* @throws UnsupportedOperationException if the inner stream does not
* support this operation.
*/
@Override
public void readFully(long position, ByteBuffer buf) throws IOException {
if (in instanceof ByteBufferPositionedReadable) {

View File

@ -573,6 +573,12 @@ private OpenFileOptions() {
public static final String FS_OPTION_OPENFILE_BUFFER_SIZE =
FS_OPTION_OPENFILE + "buffer.size";
/**
* OpenFile footer cache flag: {@value}.
*/
public static final String FS_OPTION_OPENFILE_FOOTER_CACHE =
FS_OPTION_OPENFILE + "footer.cache";
/**
* OpenFile option for read policies: {@value}.
*/
@ -586,6 +592,7 @@ private OpenFileOptions() {
public static final Set<String> FS_OPTION_OPENFILE_STANDARD_OPTIONS =
Collections.unmodifiableSet(Stream.of(
FS_OPTION_OPENFILE_BUFFER_SIZE,
FS_OPTION_OPENFILE_FOOTER_CACHE,
FS_OPTION_OPENFILE_READ_POLICY,
FS_OPTION_OPENFILE_LENGTH,
FS_OPTION_OPENFILE_SPLIT_START,
@ -599,11 +606,61 @@ private OpenFileOptions() {
"adaptive";
/**
* Read policy {@value} -whateve the implementation does by default.
* We are an avro file: {@value}.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_AVRO = "avro";
/**
* This is a columnar file format.
* Do whatever is needed to optimize for it: {@value}.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_COLUMNAR =
"columnar";
/**
* This is a CSV file of plain or UTF-8 text
* to be read sequentially.
* Do whatever is needed to optimize for it: {@value}.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_CSV =
"csv";
/**
* Read policy {@value} -whatever the implementation does by default.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_DEFAULT =
"default";
/**
* This is a table file for Apache HBase.
* Do whatever is needed to optimize for it: {@value}.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_HBASE =
"hbase";
/**
* This is a JSON file of UTF-8 text, including a
* JSON line file where each line is a JSON entity.
* Do whatever is needed to optimize for it: {@value}.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_JSON =
"json";
/**
* This is an ORC file.
* Do whatever is needed to optimize for it: {@value}.
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_ORC =
"orc";
/**
* This is a parquet file with a v1/v3 footer: {@value}.
* Do whatever is needed to optimize for it, such as footer
* prefetch and cache,
*/
public static final String FS_OPTION_OPENFILE_READ_POLICY_PARQUET =
"parquet";
/**
* Read policy for random IO: {@value}.
*/
@ -634,7 +691,13 @@ private OpenFileOptions() {
public static final Set<String> FS_OPTION_OPENFILE_READ_POLICIES =
Collections.unmodifiableSet(Stream.of(
FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE,
FS_OPTION_OPENFILE_READ_POLICY_AVRO,
FS_OPTION_OPENFILE_READ_POLICY_COLUMNAR,
FS_OPTION_OPENFILE_READ_POLICY_CSV,
FS_OPTION_OPENFILE_READ_POLICY_DEFAULT,
FS_OPTION_OPENFILE_READ_POLICY_JSON,
FS_OPTION_OPENFILE_READ_POLICY_ORC,
FS_OPTION_OPENFILE_READ_POLICY_PARQUET,
FS_OPTION_OPENFILE_READ_POLICY_RANDOM,
FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL,
FS_OPTION_OPENFILE_READ_POLICY_VECTOR,

View File

@ -1320,6 +1320,8 @@ public boolean hasPathCapability(final Path path, final String capability)
case CommonPathCapabilities.FS_PATHHANDLES:
case CommonPathCapabilities.FS_PERMISSIONS:
case CommonPathCapabilities.FS_TRUNCATE:
// block locations are generated locally
case CommonPathCapabilities.VIRTUAL_BLOCK_LOCATIONS:
return true;
case CommonPathCapabilities.FS_SYMLINKS:
return FileSystem.areSymlinksEnabled();

View File

@ -18,17 +18,30 @@
package org.apache.hadoop.io.wrappedio;
import java.io.IOException;
import java.io.InputStream;
import java.io.UncheckedIOException;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BulkDelete;
import org.apache.hadoop.fs.ByteBufferPositionedReadable;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathCapabilities;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.util.functional.FutureIO;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions;
/**
@ -82,7 +95,8 @@ public static int bulkDelete_pageSize(FileSystem fs, Path path) {
* @param fs filesystem
* @param base path to delete under.
* @param paths list of paths which must be absolute and under the base path.
* @return a list of all the paths which couldn't be deleted for a reason other than "not found" and any associated error message.
* @return a list of all the paths which couldn't be deleted for a reason other
* than "not found" and any associated error message.
* @throws UnsupportedOperationException bulk delete under that path is not supported.
* @throws UncheckedIOException if an IOE was raised.
* @throws IllegalArgumentException if a path argument is invalid.
@ -97,4 +111,137 @@ public static List<Map.Entry<Path, String>> bulkDelete_delete(FileSystem fs,
}
});
}
/**
* Does a path have a given capability?
* Calls {@link PathCapabilities#hasPathCapability(Path, String)},
* mapping IOExceptions to false.
* @param fs filesystem
* @param path path to query the capability of.
* @param capability non-null, non-empty string to query the path for support.
* @return true if the capability is supported under that part of the FS.
* resolving paths or relaying the call.
* @throws IllegalArgumentException invalid arguments
*/
public static boolean pathCapabilities_hasPathCapability(Object fs,
Path path,
String capability) {
try {
return ((PathCapabilities) fs).hasPathCapability(path, capability);
} catch (IOException e) {
return false;
}
}
/**
* Does an object implement {@link StreamCapabilities} and, if so,
* what is the result of the probe for the capability?
* Calls {@link StreamCapabilities#hasCapability(String)},
* @param object object to probe
* @param capability capability string
* @return true iff the object implements StreamCapabilities and the capability is
* declared available.
*/
public static boolean streamCapabilities_hasCapability(Object object, String capability) {
if (!(object instanceof StreamCapabilities)) {
return false;
}
return ((StreamCapabilities) object).hasCapability(capability);
}
/**
* OpenFile assistant, easy reflection-based access to
* {@link FileSystem#openFile(Path)} and blocks
* awaiting the operation completion.
* @param fs filesystem
* @param path path
* @param policy read policy
* @param status optional file status
* @param length optional file length
* @param options nullable map of other options
* @return stream of the opened file
* @throws UncheckedIOException if an IOE was raised.
*/
@InterfaceStability.Stable
public static FSDataInputStream fileSystem_openFile(
final FileSystem fs,
final Path path,
final String policy,
@Nullable final FileStatus status,
@Nullable final Long length,
@Nullable final Map<String, String> options) {
final FutureDataInputStreamBuilder builder = uncheckIOExceptions(() ->
fs.openFile(path));
if (policy != null) {
builder.opt(FS_OPTION_OPENFILE_READ_POLICY, policy);
}
if (status != null) {
builder.withFileStatus(status);
}
if (length != null) {
builder.opt(FS_OPTION_OPENFILE_LENGTH, Long.toString(length));
}
if (options != null) {
// add all the options map entries
options.forEach(builder::opt);
}
// wait for the opening.
return uncheckIOExceptions(() ->
FutureIO.awaitFuture(builder.build()));
}
/**
* Return path of the enclosing root for a given path.
* The enclosing root path is a common ancestor that should be used for temp and staging dirs
* as well as within encryption zones and other restricted directories.
* @param fs filesystem
* @param path file path to find the enclosing root path for
* @return a path to the enclosing root
* @throws IOException early checks like failure to resolve path cause IO failures
*/
public static Path fileSystem_getEnclosingRoot(FileSystem fs, Path path) throws IOException {
return fs.getEnclosingRoot(path);
}
/**
* Delegate to {@link ByteBufferPositionedReadable#read(long, ByteBuffer)}.
* @param in input stream
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* Note: that is the default behaviour of {@link FSDataInputStream#readFully(long, ByteBuffer)}.
*/
public static void byteBufferPositionedReadable_readFully(
InputStream in,
long position,
ByteBuffer buf) {
if (!(in instanceof ByteBufferPositionedReadable)) {
throw new UnsupportedOperationException("Not a ByteBufferPositionedReadable: " + in);
}
uncheckIOExceptions(() -> {
((ByteBufferPositionedReadable) in).readFully(position, buf);
return null;
});
}
/**
* Probe to see if the input stream is an instance of ByteBufferPositionedReadable.
* If the stream is an FSDataInputStream, the wrapped stream is checked.
* @param in input stream
* @return true if the stream implements the interface (including a wrapped stream)
* and that it declares the stream capability.
*/
public static boolean byteBufferPositionedReadable_readFullyAvailable(
InputStream in) {
if (!(in instanceof ByteBufferPositionedReadable)) {
return false;
}
if (in instanceof FSDataInputStream) {
// ask the wrapped stream.
return byteBufferPositionedReadable_readFullyAvailable(
((FSDataInputStream) in).getWrappedStream());
}
// now rely on the input stream implementing path capabilities, which
// all the Hadoop FS implementations do.
return streamCapabilities_hasCapability(in, StreamCapabilities.PREADBYTEBUFFER);
}
}

View File

@ -0,0 +1,357 @@
/*
* 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.io.wrappedio;
import java.io.Serializable;
import java.io.UncheckedIOException;
import java.util.HashMap;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsContext;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.util.functional.FunctionRaisingIOE;
import org.apache.hadoop.util.functional.Tuples;
import static org.apache.hadoop.fs.statistics.IOStatisticsContext.getCurrentIOStatisticsContext;
import static org.apache.hadoop.fs.statistics.IOStatisticsContext.setThreadIOStatisticsContext;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
import static org.apache.hadoop.util.Preconditions.checkArgument;
import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions;
/**
* Reflection-friendly access to IOStatistics APIs.
* All {@code Serializable} arguments/return values are actually
* {@code IOStatisticsSource} instances; passing in the wrong value
* will raise IllegalArgumentExceptions.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class WrappedStatistics {
private WrappedStatistics() {
}
/**
* Probe for an object being an instance of {@code IOStatisticsSource}.
* @param object object to probe
* @return true if the object is the right type.
*/
public static boolean isIOStatisticsSource(Object object) {
return object instanceof IOStatisticsSource;
}
/**
* Probe for an object being an instance of {@code IOStatistics}.
* @param object object to probe
* @return true if the object is the right type.
*/
public static boolean isIOStatistics(Object object) {
return object instanceof IOStatistics;
}
/**
* Probe for an object being an instance of {@code IOStatisticsSnapshot}.
* @param object object to probe
* @return true if the object is the right type.
*/
public static boolean isIOStatisticsSnapshot(Serializable object) {
return object instanceof IOStatisticsSnapshot;
}
/**
* Aggregate an existing {@link IOStatisticsSnapshot} with
* the supplied statistics.
* @param snapshot snapshot to update
* @param statistics IOStatistics to add
* @return true if the snapshot was updated.
* @throws IllegalArgumentException if the {@code statistics} argument is not
* null but not an instance of IOStatistics, or if {@code snapshot} is invalid.
*/
public static boolean iostatisticsSnapshot_aggregate(
Serializable snapshot, @Nullable Object statistics) {
requireIOStatisticsSnapshot(snapshot);
if (statistics == null) {
return false;
}
checkArgument(statistics instanceof IOStatistics,
"Not an IOStatistics instance: %s", statistics);
final IOStatistics sourceStats = (IOStatistics) statistics;
return applyToIOStatisticsSnapshot(snapshot, s ->
s.aggregate(sourceStats));
}
/**
* Create a new {@link IOStatisticsSnapshot} instance.
* @return an empty IOStatisticsSnapshot.
*/
public static Serializable iostatisticsSnapshot_create() {
return iostatisticsSnapshot_create(null);
}
/**
* Create a new {@link IOStatisticsSnapshot} instance.
* @param source optional source statistics
* @return an IOStatisticsSnapshot.
* @throws ClassCastException if the {@code source} is not null and not an IOStatistics instance
*/
public static Serializable iostatisticsSnapshot_create(@Nullable Object source) {
return new IOStatisticsSnapshot((IOStatistics) source);
}
/**
* Load IOStatisticsSnapshot from a Hadoop filesystem.
* @param fs filesystem
* @param path path
* @return the loaded snapshot
* @throws UncheckedIOException Any IO exception.
*/
public static Serializable iostatisticsSnapshot_load(
FileSystem fs,
Path path) {
return uncheckIOExceptions(() ->
IOStatisticsSnapshot.serializer().load(fs, path));
}
/**
* Extract the IOStatistics from an object in a serializable form.
* @param source source object, may be null/not a statistics source/instance
* @return {@link IOStatisticsSnapshot} or null if the object is null/doesn't have statistics
*/
public static Serializable iostatisticsSnapshot_retrieve(@Nullable Object source) {
IOStatistics stats = retrieveIOStatistics(source);
if (stats == null) {
return null;
}
return iostatisticsSnapshot_create(stats);
}
/**
* Save IOStatisticsSnapshot to a Hadoop filesystem as a JSON file.
* @param snapshot statistics
* @param fs filesystem
* @param path path
* @param overwrite should any existing file be overwritten?
* @throws UncheckedIOException Any IO exception.
*/
public static void iostatisticsSnapshot_save(
@Nullable Serializable snapshot,
FileSystem fs,
Path path,
boolean overwrite) {
applyToIOStatisticsSnapshot(snapshot, s -> {
IOStatisticsSnapshot.serializer().save(fs, path, s, overwrite);
return null;
});
}
/**
* Save IOStatisticsSnapshot to a JSON string.
* @param snapshot statistics; may be null or of an incompatible type
* @return JSON string value
* @throws UncheckedIOException Any IO/jackson exception.
* @throws IllegalArgumentException if the supplied class is not a snapshot
*/
public static String iostatisticsSnapshot_toJsonString(@Nullable Serializable snapshot) {
return applyToIOStatisticsSnapshot(snapshot,
IOStatisticsSnapshot.serializer()::toJson);
}
/**
* Load IOStatisticsSnapshot from a JSON string.
* @param json JSON string value.
* @return deserialized snapshot.
* @throws UncheckedIOException Any IO/jackson exception.
*/
public static Serializable iostatisticsSnapshot_fromJsonString(
final String json) {
return uncheckIOExceptions(() ->
IOStatisticsSnapshot.serializer().fromJson(json));
}
/**
* Get the counters of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of counters.
*/
public static Map<String, Long> iostatistics_counters(
Serializable source) {
return applyToIOStatisticsSnapshot(source, IOStatisticsSnapshot::counters);
}
/**
* Get the gauges of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of gauges.
*/
public static Map<String, Long> iostatistics_gauges(
Serializable source) {
return applyToIOStatisticsSnapshot(source, IOStatisticsSnapshot::gauges);
}
/**
* Get the minimums of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of minimums.
*/
public static Map<String, Long> iostatistics_minimums(
Serializable source) {
return applyToIOStatisticsSnapshot(source, IOStatisticsSnapshot::minimums);
}
/**
* Get the maximums of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of maximums.
*/
public static Map<String, Long> iostatistics_maximums(
Serializable source) {
return applyToIOStatisticsSnapshot(source, IOStatisticsSnapshot::maximums);
}
/**
* Get the means of an IOStatisticsSnapshot.
* Each value in the map is the (sample, sum) tuple of the values;
* the mean is then calculated by dividing sum/sample wherever sample count is non-zero.
* @param source source of statistics.
* @return a map of mean key to (sample, sum) tuples.
*/
public static Map<String, Map.Entry<Long, Long>> iostatistics_means(
Serializable source) {
return applyToIOStatisticsSnapshot(source, stats -> {
Map<String, Map.Entry<Long, Long>> map = new HashMap<>();
stats.meanStatistics().forEach((k, v) ->
map.put(k, Tuples.pair(v.getSamples(), v.getSum())));
return map;
});
}
/**
* Get the context's {@link IOStatisticsContext} which
* implements {@link IOStatisticsSource}.
* This is either a thread-local value or a global empty context.
* @return instance of {@link IOStatisticsContext}.
*/
public static Object iostatisticsContext_getCurrent() {
return getCurrentIOStatisticsContext();
}
/**
* Set the IOStatisticsContext for the current thread.
* @param statisticsContext IOStatistics context instance for the
* current thread. If null, the context is reset.
*/
public static void iostatisticsContext_setThreadIOStatisticsContext(
@Nullable Object statisticsContext) {
setThreadIOStatisticsContext((IOStatisticsContext) statisticsContext);
}
/**
* Static probe to check if the thread-level IO statistics enabled.
* @return true if the thread-level IO statistics are enabled.
*/
public static boolean iostatisticsContext_enabled() {
return IOStatisticsContext.enabled();
}
/**
* Reset the context's IOStatistics.
* {@link IOStatisticsContext#reset()}
*/
public static void iostatisticsContext_reset() {
getCurrentIOStatisticsContext().reset();
}
/**
* Take a snapshot of the context IOStatistics.
* {@link IOStatisticsContext#snapshot()}
* @return an instance of {@link IOStatisticsSnapshot}.
*/
public static Serializable iostatisticsContext_snapshot() {
return getCurrentIOStatisticsContext().snapshot();
}
/**
* Aggregate into the IOStatistics context the statistics passed in via
* IOStatistics/source parameter.
* <p>
* Returns false if the source is null or does not contain any statistics.
* @param source implementation of {@link IOStatisticsSource} or {@link IOStatistics}
* @return true if the the source object was aggregated.
*/
public static boolean iostatisticsContext_aggregate(Object source) {
IOStatistics stats = retrieveIOStatistics(source);
if (stats != null) {
getCurrentIOStatisticsContext().getAggregator().aggregate(stats);
return true;
} else {
return false;
}
}
/**
* Convert IOStatistics to a string form, with all the metrics sorted
* and empty value stripped.
* @param statistics A statistics instance; may be null
* @return string value or the empty string if null
*/
public static String iostatistics_toPrettyString(@Nullable Object statistics) {
return statistics == null
? ""
: ioStatisticsToPrettyString((IOStatistics) statistics);
}
/**
* Apply a function to an object which may be an IOStatisticsSnapshot.
* @param <T> return type
* @param source statistics snapshot
* @param fun function to invoke if {@code source} is valid.
* @return the applied value
* @throws UncheckedIOException Any IO exception.
* @throws IllegalArgumentException if the supplied class is not a snapshot
*/
public static <T> T applyToIOStatisticsSnapshot(
Serializable source,
FunctionRaisingIOE<IOStatisticsSnapshot, T> fun) {
return fun.unchecked(requireIOStatisticsSnapshot(source));
}
/**
* Require the parameter to be an instance of {@link IOStatisticsSnapshot}.
* @param snapshot object to validate
* @return cast value
* @throws IllegalArgumentException if the supplied class is not a snapshot
*/
private static IOStatisticsSnapshot requireIOStatisticsSnapshot(final Serializable snapshot) {
checkArgument(snapshot instanceof IOStatisticsSnapshot,
"Not an IOStatisticsSnapshot %s", snapshot);
return (IOStatisticsSnapshot) snapshot;
}
}

View File

@ -0,0 +1,500 @@
/*
* 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.io.wrappedio.impl;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.dynamic.DynMethods;
import static org.apache.hadoop.util.dynamic.BindingUtils.available;
import static org.apache.hadoop.util.dynamic.BindingUtils.checkAvailable;
import static org.apache.hadoop.util.dynamic.BindingUtils.extractIOEs;
import static org.apache.hadoop.util.dynamic.BindingUtils.loadClass;
import static org.apache.hadoop.util.dynamic.BindingUtils.loadStaticMethod;
/**
* The wrapped IO methods in {@code org.apache.hadoop.io.wrappedio.WrappedIO},
* dynamically loaded.
*/
public final class DynamicWrappedIO {
private static final Logger LOG = LoggerFactory.getLogger(DynamicWrappedIO.class);
/**
* Classname of the wrapped IO class: {@value}.
*/
private static final String WRAPPED_IO_CLASSNAME =
"org.apache.hadoop.io.wrappedio.WrappedIO";
/**
* Method name for openFile: {@value}.
*/
private static final String FILESYSTEM_OPEN_FILE = "fileSystem_openFile";
/**
* Method name for bulk delete: {@value}.
*/
private static final String BULKDELETE_DELETE = "bulkDelete_delete";
/**
* Method name for bulk delete: {@value}.
*/
private static final String BULKDELETE_PAGESIZE = "bulkDelete_pageSize";
/**
* Method name for {@code byteBufferPositionedReadable}: {@value}.
*/
private static final String BYTE_BUFFER_POSITIONED_READABLE_READ_FULLY_AVAILABLE =
"byteBufferPositionedReadable_readFullyAvailable";
/**
* Method name for {@code byteBufferPositionedReadable}: {@value}.
*/
private static final String BYTE_BUFFER_POSITIONED_READABLE_READ_FULLY =
"byteBufferPositionedReadable_readFully";
/**
* Method name for {@code PathCapabilities.hasPathCapability()}.
* {@value}
*/
private static final String PATH_CAPABILITIES_HAS_PATH_CAPABILITY =
"pathCapabilities_hasPathCapability";
/**
* Method name for {@code StreamCapabilities.hasCapability()}.
* {@value}
*/
private static final String STREAM_CAPABILITIES_HAS_CAPABILITY =
"streamCapabilities_hasCapability";
/**
* A singleton instance of the wrapper.
*/
private static final DynamicWrappedIO INSTANCE = new DynamicWrappedIO();
/**
* Read policy for parquet files: {@value}.
*/
public static final String PARQUET_READ_POLICIES = "parquet, columnar, vector, random";
/**
* Was wrapped IO loaded?
* In the hadoop codebase, this is true.
* But in other libraries it may not always be true...this
* field is used to assist copy-and-paste adoption.
*/
private final boolean loaded;
/**
* Method binding.
* {@code WrappedIO.bulkDelete_delete(FileSystem, Path, Collection)}.
*/
private final DynMethods.UnboundMethod bulkDeleteDeleteMethod;
/**
* Method binding.
* {@code WrappedIO.bulkDelete_pageSize(FileSystem, Path)}.
*/
private final DynMethods.UnboundMethod bulkDeletePageSizeMethod;
/**
* Dynamic openFile() method.
* {@code WrappedIO.fileSystem_openFile(FileSystem, Path, String, FileStatus, Long, Map)}.
*/
private final DynMethods.UnboundMethod fileSystemOpenFileMethod;
private final DynMethods.UnboundMethod pathCapabilitiesHasPathCapabilityMethod;
private final DynMethods.UnboundMethod streamCapabilitiesHasCapabilityMethod;
private final DynMethods.UnboundMethod byteBufferPositionedReadableReadFullyAvailableMethod;
private final DynMethods.UnboundMethod byteBufferPositionedReadableReadFullyMethod;
public DynamicWrappedIO() {
this(WRAPPED_IO_CLASSNAME);
}
public DynamicWrappedIO(String classname) {
// Wrapped IO class.
Class<?> wrappedClass = loadClass(classname);
loaded = wrappedClass != null;
// bulk delete APIs
bulkDeleteDeleteMethod = loadStaticMethod(
wrappedClass,
List.class,
BULKDELETE_DELETE,
FileSystem.class,
Path.class,
Collection.class);
bulkDeletePageSizeMethod = loadStaticMethod(
wrappedClass,
Integer.class,
BULKDELETE_PAGESIZE,
FileSystem.class,
Path.class);
// load the openFile method
fileSystemOpenFileMethod = loadStaticMethod(
wrappedClass,
FSDataInputStream.class,
FILESYSTEM_OPEN_FILE,
FileSystem.class,
Path.class,
String.class,
FileStatus.class,
Long.class,
Map.class);
// path and stream capabilities
pathCapabilitiesHasPathCapabilityMethod = loadStaticMethod(wrappedClass,
boolean.class,
PATH_CAPABILITIES_HAS_PATH_CAPABILITY,
Object.class,
Path.class,
String.class);
streamCapabilitiesHasCapabilityMethod = loadStaticMethod(wrappedClass,
boolean.class,
STREAM_CAPABILITIES_HAS_CAPABILITY,
Object.class,
String.class);
// ByteBufferPositionedReadable
byteBufferPositionedReadableReadFullyAvailableMethod = loadStaticMethod(wrappedClass,
Void.class,
BYTE_BUFFER_POSITIONED_READABLE_READ_FULLY_AVAILABLE,
InputStream.class);
byteBufferPositionedReadableReadFullyMethod = loadStaticMethod(wrappedClass,
Void.class,
BYTE_BUFFER_POSITIONED_READABLE_READ_FULLY,
InputStream.class,
long.class,
ByteBuffer.class);
}
/**
* Is the wrapped IO class loaded?
* @return true if the wrappedIO class was found and loaded.
*/
public boolean loaded() {
return loaded;
}
/**
* For testing: verify that all methods were found.
* @throws UnsupportedOperationException if the method was not found.
*/
void requireAllMethodsAvailable() throws UnsupportedOperationException {
final DynMethods.UnboundMethod[] methods = {
bulkDeleteDeleteMethod,
bulkDeletePageSizeMethod,
fileSystemOpenFileMethod,
pathCapabilitiesHasPathCapabilityMethod,
streamCapabilitiesHasCapabilityMethod,
byteBufferPositionedReadableReadFullyAvailableMethod,
byteBufferPositionedReadableReadFullyMethod,
};
for (DynMethods.UnboundMethod method : methods) {
LOG.info("Checking method {}", method);
if (!available(method)) {
throw new UnsupportedOperationException("Unbound " + method);
}
}
}
/**
* Are the bulk delete methods available?
* @return true if the methods were found.
*/
public boolean bulkDelete_available() {
return available(bulkDeleteDeleteMethod);
}
/**
* Get the maximum number of objects/files to delete in a single request.
* @param fileSystem filesystem
* @param path path to delete under.
* @return a number greater than or equal to zero.
* @throws UnsupportedOperationException bulk delete under that path is not supported.
* @throws IllegalArgumentException path not valid.
* @throws IOException problems resolving paths
* @throws RuntimeException invocation failure.
*/
public int bulkDelete_pageSize(final FileSystem fileSystem, final Path path)
throws IOException {
checkAvailable(bulkDeletePageSizeMethod);
return extractIOEs(() ->
bulkDeletePageSizeMethod.invoke(null, fileSystem, path));
}
/**
* Delete a list of files/objects.
* <ul>
* <li>Files must be under the path provided in {@code base}.</li>
* <li>The size of the list must be equal to or less than the page size.</li>
* <li>Directories are not supported; the outcome of attempting to delete
* directories is undefined (ignored; undetected, listed as failures...).</li>
* <li>The operation is not atomic.</li>
* <li>The operation is treated as idempotent: network failures may
* trigger resubmission of the request -any new objects created under a
* path in the list may then be deleted.</li>
* <li>There is no guarantee that any parent directories exist after this call.
* </li>
* </ul>
* @param fs filesystem
* @param base path to delete under.
* @param paths list of paths which must be absolute and under the base path.
* @return a list of all the paths which couldn't be deleted for a reason other than
* "not found" and any associated error message.
* @throws UnsupportedOperationException bulk delete under that path is not supported.
* @throws IllegalArgumentException if a path argument is invalid.
* @throws IOException IO problems including networking, authentication and more.
*/
public List<Map.Entry<Path, String>> bulkDelete_delete(FileSystem fs,
Path base,
Collection<Path> paths) throws IOException {
checkAvailable(bulkDeleteDeleteMethod);
return extractIOEs(() ->
bulkDeleteDeleteMethod.invoke(null, fs, base, paths));
}
/**
* Is the {@link #fileSystem_openFile(FileSystem, Path, String, FileStatus, Long, Map)}
* method available.
* @return true if the optimized open file method can be invoked.
*/
public boolean fileSystem_openFile_available() {
return available(fileSystemOpenFileMethod);
}
/**
* OpenFile assistant, easy reflection-based access to
* {@code FileSystem#openFile(Path)} and blocks
* awaiting the operation completion.
* @param fs filesystem
* @param path path
* @param policy read policy
* @param status optional file status
* @param length optional file length
* @param options nullable map of other options
* @return stream of the opened file
* @throws IOException if the operation was attempted and failed.
*/
public FSDataInputStream fileSystem_openFile(
final FileSystem fs,
final Path path,
final String policy,
@Nullable final FileStatus status,
@Nullable final Long length,
@Nullable final Map<String, String> options)
throws IOException {
checkAvailable(fileSystemOpenFileMethod);
return extractIOEs(() ->
fileSystemOpenFileMethod.invoke(null,
fs, path, policy, status, length, options));
}
/**
* Does a path have a given capability?
* Calls {@code PathCapabilities#hasPathCapability(Path, String)},
* mapping IOExceptions to false.
* @param fs filesystem
* @param path path to query the capability of.
* @param capability non-null, non-empty string to query the path for support.
* @return true if the capability is supported
* under that part of the FS
* false if the method is not loaded or the path lacks the capability.
* @throws IllegalArgumentException invalid arguments
*/
public boolean pathCapabilities_hasPathCapability(Object fs,
Path path,
String capability) {
if (!available(pathCapabilitiesHasPathCapabilityMethod)) {
return false;
}
return pathCapabilitiesHasPathCapabilityMethod.invoke(null, fs, path, capability);
}
/**
* Does an object implement {@code StreamCapabilities} and, if so,
* what is the result of the probe for the capability?
* Calls {@code StreamCapabilities#hasCapability(String)},
* @param object object to probe
* @param capability capability string
* @return true iff the object implements StreamCapabilities and the capability is
* declared available.
*/
public boolean streamCapabilities_hasCapability(Object object, String capability) {
if (!available(streamCapabilitiesHasCapabilityMethod)) {
return false;
}
return streamCapabilitiesHasCapabilityMethod.invoke(null, object, capability);
}
/**
* Are the ByteBufferPositionedReadable methods loaded?
* This does not check that a specific stream implements the API;
* use {@link #byteBufferPositionedReadable_readFullyAvailable(InputStream)}.
* @return true if the hadoop libraries have the method.
*/
public boolean byteBufferPositionedReadable_available() {
return available(byteBufferPositionedReadableReadFullyAvailableMethod);
}
/**
* Probe to see if the input stream is an instance of ByteBufferPositionedReadable.
* If the stream is an FSDataInputStream, the wrapped stream is checked.
* @param in input stream
* @return true if the API is available, the stream implements the interface
* (including the innermost wrapped stream) and that it declares the stream capability.
* @throws IOException if the operation was attempted and failed.
*/
public boolean byteBufferPositionedReadable_readFullyAvailable(
InputStream in) throws IOException {
if (available(byteBufferPositionedReadableReadFullyAvailableMethod)) {
return extractIOEs(() ->
byteBufferPositionedReadableReadFullyAvailableMethod.invoke(null, in));
} else {
return false;
}
}
/**
* Delegate to {@code ByteBufferPositionedReadable#read(long, ByteBuffer)}.
* @param in input stream
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* @throws UnsupportedOperationException if the input doesn't implement
* the interface or, if when invoked, it is raised.
* Note: that is the default behaviour of {@code FSDataInputStream#readFully(long, ByteBuffer)}.
* @throws IOException if the operation was attempted and failed.
*/
public void byteBufferPositionedReadable_readFully(
InputStream in,
long position,
ByteBuffer buf) throws IOException {
checkAvailable(byteBufferPositionedReadableReadFullyMethod);
extractIOEs(() ->
byteBufferPositionedReadableReadFullyMethod.invoke(null, in, position, buf));
}
/**
* Get the singleton instance.
* @return the instance
*/
public static DynamicWrappedIO instance() {
return INSTANCE;
}
/**
* Is the wrapped IO class loaded?
* @return true if the instance is loaded.
*/
public static boolean isAvailable() {
return instance().loaded();
}
/**
* Open a file.
* <p>
* If the WrappedIO class is found, use it.
* <p>
* If not, falls back to the classic {@code fs.open(Path)} call.
* @param fs filesystem
* @param status file status
* @param readPolicies read policy to use
* @return the input stream
* @throws IOException any IO failure.
*/
public static FSDataInputStream openFile(
FileSystem fs,
FileStatus status,
String readPolicies) throws IOException {
return openFileOnInstance(instance(), fs, status, readPolicies);
}
/**
* Open a file.
* <p>
* If the WrappedIO class is found, uses
* {@link #fileSystem_openFile(FileSystem, Path, String, FileStatus, Long, Map)} with
* {@link #PARQUET_READ_POLICIES} as the list of read policies and passing down
* the file status.
* <p>
* If not, falls back to the classic {@code fs.open(Path)} call.
* @param instance dynamic wrapped IO instance.
* @param fs filesystem
* @param status file status
* @param readPolicies read policy to use
* @return the input stream
* @throws IOException any IO failure.
*/
@VisibleForTesting
static FSDataInputStream openFileOnInstance(
DynamicWrappedIO instance,
FileSystem fs,
FileStatus status,
String readPolicies) throws IOException {
FSDataInputStream stream;
if (instance.fileSystem_openFile_available()) {
// use openfile for a higher performance read
// and the ability to set a read policy.
// This optimizes for cloud storage by saving on IO
// in open and choosing the range for GET requests.
// For other stores, it ultimately invokes the classic open(Path)
// call so is no more expensive than before.
LOG.debug("Opening file {} through fileSystem_openFile", status);
stream = instance.fileSystem_openFile(fs,
status.getPath(),
readPolicies,
status,
null,
null);
} else {
LOG.debug("Opening file {} through open()", status);
stream = fs.open(status.getPath());
}
return stream;
}
}

View File

@ -0,0 +1,678 @@
/*
* 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.io.wrappedio.impl;
import java.io.Serializable;
import java.io.UncheckedIOException;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.util.dynamic.DynMethods;
import static org.apache.hadoop.util.dynamic.BindingUtils.available;
import static org.apache.hadoop.util.dynamic.BindingUtils.checkAvailable;
import static org.apache.hadoop.util.dynamic.BindingUtils.loadClass;
import static org.apache.hadoop.util.dynamic.BindingUtils.loadStaticMethod;
/**
* The wrapped IOStatistics methods in {@code WrappedStatistics},
* dynamically loaded.
* This is suitable for copy-and-paste into other libraries which have some
* version of the Parquet DynMethods classes already present.
*/
public final class DynamicWrappedStatistics {
/**
* Classname of the wrapped statistics class: {@value}.
*/
public static final String WRAPPED_STATISTICS_CLASSNAME =
"org.apache.hadoop.io.wrappedio.WrappedStatistics";
/**
* Method name: {@value}.
*/
public static final String IS_IOSTATISTICS_SOURCE = "isIOStatisticsSource";
/**
* Method name: {@value}.
*/
public static final String IS_IOSTATISTICS = "isIOStatistics";
/**
* Method name: {@value}.
*/
public static final String IS_IOSTATISTICS_SNAPSHOT = "isIOStatisticsSnapshot";
/**
* IOStatisticsContext method: {@value}.
*/
public static final String IOSTATISTICS_CONTEXT_AGGREGATE = "iostatisticsContext_aggregate";
/**
* IOStatisticsContext method: {@value}.
*/
public static final String IOSTATISTICS_CONTEXT_ENABLED = "iostatisticsContext_enabled";
/**
* IOStatisticsContext method: {@value}.
*/
public static final String IOSTATISTICS_CONTEXT_GET_CURRENT = "iostatisticsContext_getCurrent";
/**
* IOStatisticsContext method: {@value}.
*/
public static final String IOSTATISTICS_CONTEXT_SET_THREAD_CONTEXT =
"iostatisticsContext_setThreadIOStatisticsContext";
/**
* IOStatisticsContext method: {@value}.
*/
public static final String IOSTATISTICS_CONTEXT_RESET = "iostatisticsContext_reset";
/**
* IOStatisticsContext method: {@value}.
*/
public static final String IOSTATISTICS_CONTEXT_SNAPSHOT = "iostatisticsContext_snapshot";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_AGGREGATE = "iostatisticsSnapshot_aggregate";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_CREATE = "iostatisticsSnapshot_create";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_FROM_JSON_STRING =
"iostatisticsSnapshot_fromJsonString";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_LOAD = "iostatisticsSnapshot_load";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_RETRIEVE = "iostatisticsSnapshot_retrieve";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_SAVE = "iostatisticsSnapshot_save";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_SNAPSHOT_TO_JSON_STRING =
"iostatisticsSnapshot_toJsonString";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_TO_PRETTY_STRING =
"iostatistics_toPrettyString";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_COUNTERS = "iostatistics_counters";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_GAUGES = "iostatistics_gauges";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_MINIMUMS = "iostatistics_minimums";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_MAXIMUMS = "iostatistics_maximums";
/**
* Method name: {@value}.
*/
public static final String IOSTATISTICS_MEANS = "iostatistics_means";
/**
* Was wrapped IO loaded?
* In the hadoop codebase, this is true.
* But in other libraries it may not always be true...this
* field is used to assist copy-and-paste adoption.
*/
private final boolean loaded;
/*
IOStatisticsContext methods.
*/
private final DynMethods.UnboundMethod iostatisticsContextAggregateMethod;
private final DynMethods.UnboundMethod iostatisticsContextEnabledMethod;
private final DynMethods.UnboundMethod iostatisticsContextGetCurrentMethod;
private final DynMethods.UnboundMethod iostatisticsContextResetMethod;
private final DynMethods.UnboundMethod iostatisticsContextSetThreadContextMethod;
private final DynMethods.UnboundMethod iostatisticsContextSnapshotMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotAggregateMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotCreateMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotCreateWithSourceMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotLoadMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotFromJsonStringMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotRetrieveMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotSaveMethod;
private final DynMethods.UnboundMethod iostatisticsToPrettyStringMethod;
private final DynMethods.UnboundMethod iostatisticsSnapshotToJsonStringMethod;
private final DynMethods.UnboundMethod iostatisticsCountersMethod;
private final DynMethods.UnboundMethod iostatisticsGaugesMethod;
private final DynMethods.UnboundMethod iostatisticsMinimumsMethod;
private final DynMethods.UnboundMethod iostatisticsMaximumsMethod;
private final DynMethods.UnboundMethod iostatisticsMeansMethod;
private final DynMethods.UnboundMethod isIOStatisticsSourceMethod;
private final DynMethods.UnboundMethod isIOStatisticsMethod;
private final DynMethods.UnboundMethod isIOStatisticsSnapshotMethod;
public DynamicWrappedStatistics() {
this(WRAPPED_STATISTICS_CLASSNAME);
}
public DynamicWrappedStatistics(String classname) {
// wrap the real class.
Class<?> wrappedClass = loadClass(classname);
loaded = wrappedClass != null;
// instanceof checks
isIOStatisticsSourceMethod = loadStaticMethod(wrappedClass,
Boolean.class, IS_IOSTATISTICS_SOURCE, Object.class);
isIOStatisticsMethod = loadStaticMethod(wrappedClass,
Boolean.class, IS_IOSTATISTICS, Object.class);
isIOStatisticsSnapshotMethod = loadStaticMethod(wrappedClass,
Boolean.class, IS_IOSTATISTICS_SNAPSHOT, Serializable.class);
// IOStatisticsContext operations
iostatisticsContextAggregateMethod = loadStaticMethod(wrappedClass,
Boolean.class, IOSTATISTICS_CONTEXT_AGGREGATE, Object.class);
iostatisticsContextEnabledMethod = loadStaticMethod(wrappedClass,
Boolean.class, IOSTATISTICS_CONTEXT_ENABLED);
iostatisticsContextGetCurrentMethod = loadStaticMethod(wrappedClass,
Object.class, IOSTATISTICS_CONTEXT_GET_CURRENT);
iostatisticsContextResetMethod = loadStaticMethod(wrappedClass,
Void.class, IOSTATISTICS_CONTEXT_RESET);
iostatisticsContextSetThreadContextMethod = loadStaticMethod(wrappedClass,
Void.class, IOSTATISTICS_CONTEXT_SET_THREAD_CONTEXT, Object.class);
iostatisticsContextSnapshotMethod = loadStaticMethod(wrappedClass,
Serializable.class, IOSTATISTICS_CONTEXT_SNAPSHOT);
// IOStatistics Snapshot operations
iostatisticsSnapshotAggregateMethod =
loadStaticMethod(wrappedClass,
Boolean.class,
IOSTATISTICS_SNAPSHOT_AGGREGATE,
Serializable.class,
Object.class);
iostatisticsSnapshotCreateMethod =
loadStaticMethod(wrappedClass,
Serializable.class,
IOSTATISTICS_SNAPSHOT_CREATE);
iostatisticsSnapshotCreateWithSourceMethod =
loadStaticMethod(wrappedClass,
Serializable.class,
IOSTATISTICS_SNAPSHOT_CREATE,
Object.class);
iostatisticsSnapshotFromJsonStringMethod =
loadStaticMethod(wrappedClass,
Serializable.class,
IOSTATISTICS_SNAPSHOT_FROM_JSON_STRING,
String.class);
iostatisticsSnapshotToJsonStringMethod =
loadStaticMethod(wrappedClass,
String.class,
IOSTATISTICS_SNAPSHOT_TO_JSON_STRING,
Serializable.class);
iostatisticsSnapshotRetrieveMethod =
loadStaticMethod(wrappedClass,
Serializable.class,
IOSTATISTICS_SNAPSHOT_RETRIEVE,
Object.class);
iostatisticsSnapshotLoadMethod =
loadStaticMethod(wrappedClass,
Serializable.class,
IOSTATISTICS_SNAPSHOT_LOAD,
FileSystem.class,
Path.class);
iostatisticsSnapshotSaveMethod =
loadStaticMethod(wrappedClass,
Void.class,
IOSTATISTICS_SNAPSHOT_SAVE,
Serializable.class,
FileSystem.class,
Path.class,
boolean.class); // note: not Boolean.class
// getting contents of snapshots
iostatisticsCountersMethod =
loadStaticMethod(wrappedClass,
Map.class,
IOSTATISTICS_COUNTERS,
Serializable.class);
iostatisticsGaugesMethod =
loadStaticMethod(wrappedClass,
Map.class,
IOSTATISTICS_GAUGES,
Serializable.class);
iostatisticsMinimumsMethod =
loadStaticMethod(wrappedClass,
Map.class,
IOSTATISTICS_MINIMUMS,
Serializable.class);
iostatisticsMaximumsMethod =
loadStaticMethod(wrappedClass,
Map.class,
IOSTATISTICS_MAXIMUMS,
Serializable.class);
iostatisticsMeansMethod =
loadStaticMethod(wrappedClass,
Map.class,
IOSTATISTICS_MEANS,
Serializable.class);
// stringification
iostatisticsToPrettyStringMethod =
loadStaticMethod(wrappedClass,
String.class,
IOSTATISTICS_TO_PRETTY_STRING,
Object.class);
}
/**
* Is the wrapped statistics class loaded?
* @return true if the wrappedIO class was found and loaded.
*/
public boolean loaded() {
return loaded;
}
/**
* Are the core IOStatistics methods and classes available.
* @return true if the relevant methods are loaded.
*/
public boolean ioStatisticsAvailable() {
return available(iostatisticsSnapshotCreateMethod);
}
/**
* Are the IOStatisticsContext methods and classes available?
* @return true if the relevant methods are loaded.
*/
public boolean ioStatisticsContextAvailable() {
return available(iostatisticsContextEnabledMethod);
}
/**
* Require a IOStatistics to be available.
* @throws UnsupportedOperationException if the method was not found.
*/
private void checkIoStatisticsAvailable() {
checkAvailable(iostatisticsSnapshotCreateMethod);
}
/**
* Require IOStatisticsContext methods to be available.
* @throws UnsupportedOperationException if the classes/methods were not found
*/
private void checkIoStatisticsContextAvailable() {
checkAvailable(iostatisticsContextEnabledMethod);
}
/**
* Probe for an object being an instance of {@code IOStatisticsSource}.
* @param object object to probe
* @return true if the object is the right type, false if the classes
* were not found or the object is null/of a different type
*/
public boolean isIOStatisticsSource(Object object) {
return ioStatisticsAvailable()
&& (boolean) isIOStatisticsSourceMethod.invoke(null, object);
}
/**
* Probe for an object being an instance of {@code IOStatisticsSource}.
* @param object object to probe
* @return true if the object is the right type, false if the classes
* were not found or the object is null/of a different type
*/
public boolean isIOStatistics(Object object) {
return ioStatisticsAvailable()
&& (boolean) isIOStatisticsMethod.invoke(null, object);
}
/**
* Probe for an object being an instance of {@code IOStatisticsSnapshot}.
* @param object object to probe
* @return true if the object is the right type, false if the classes
* were not found or the object is null/of a different type
*/
public boolean isIOStatisticsSnapshot(Serializable object) {
return ioStatisticsAvailable()
&& (boolean) isIOStatisticsSnapshotMethod.invoke(null, object);
}
/**
* Probe to check if the thread-level IO statistics enabled.
* If the relevant classes and methods were not found, returns false
* @return true if the IOStatisticsContext API was found
* and is enabled.
*/
public boolean iostatisticsContext_enabled() {
return ioStatisticsAvailable()
&& (boolean) iostatisticsContextEnabledMethod.invoke(null);
}
/**
* Get the context's {@code IOStatisticsContext} which
* implements {@code IOStatisticsSource}.
* This is either a thread-local value or a global empty context.
* @return instance of {@code IOStatisticsContext}.
* @throws UnsupportedOperationException if the IOStatisticsContext API was not found
*/
public Object iostatisticsContext_getCurrent()
throws UnsupportedOperationException {
checkIoStatisticsContextAvailable();
return iostatisticsContextGetCurrentMethod.invoke(null);
}
/**
* Set the IOStatisticsContext for the current thread.
* @param statisticsContext IOStatistics context instance for the
* current thread. If null, the context is reset.
* @throws UnsupportedOperationException if the IOStatisticsContext API was not found
*/
public void iostatisticsContext_setThreadIOStatisticsContext(
@Nullable Object statisticsContext) throws UnsupportedOperationException {
checkIoStatisticsContextAvailable();
iostatisticsContextSetThreadContextMethod.invoke(null, statisticsContext);
}
/**
* Reset the context's IOStatistics.
* {@code IOStatisticsContext#reset()}
* @throws UnsupportedOperationException if the IOStatisticsContext API was not found
*/
public void iostatisticsContext_reset()
throws UnsupportedOperationException {
checkIoStatisticsContextAvailable();
iostatisticsContextResetMethod.invoke(null);
}
/**
* Take a snapshot of the context IOStatistics.
* {@code IOStatisticsContext#snapshot()}
* @return an instance of {@code IOStatisticsSnapshot}.
* @throws UnsupportedOperationException if the IOStatisticsContext API was not found
*/
public Serializable iostatisticsContext_snapshot()
throws UnsupportedOperationException {
checkIoStatisticsContextAvailable();
return iostatisticsContextSnapshotMethod.invoke(null);
}
/**
* Aggregate into the IOStatistics context the statistics passed in via
* IOStatistics/source parameter.
* <p>
* Returns false if the source is null or does not contain any statistics.
* @param source implementation of {@link IOStatisticsSource} or {@link IOStatistics}
* @return true if the the source object was aggregated.
*/
public boolean iostatisticsContext_aggregate(Object source) {
checkIoStatisticsContextAvailable();
return iostatisticsContextAggregateMethod.invoke(null, source);
}
/**
* Aggregate an existing {@code IOStatisticsSnapshot} with
* the supplied statistics.
* @param snapshot snapshot to update
* @param statistics IOStatistics to add
* @return true if the snapshot was updated.
* @throws IllegalArgumentException if the {@code statistics} argument is not
* null but not an instance of IOStatistics, or if {@code snapshot} is invalid.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public boolean iostatisticsSnapshot_aggregate(
Serializable snapshot, @Nullable Object statistics)
throws UnsupportedOperationException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotAggregateMethod.invoke(null, snapshot, statistics);
}
/**
* Create a new {@code IOStatisticsSnapshot} instance.
* @return an empty IOStatisticsSnapshot.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public Serializable iostatisticsSnapshot_create()
throws UnsupportedOperationException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotCreateMethod.invoke(null);
}
/**
* Create a new {@code IOStatisticsSnapshot} instance.
* @param source optional source statistics
* @return an IOStatisticsSnapshot.
* @throws ClassCastException if the {@code source} is not valid.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public Serializable iostatisticsSnapshot_create(
@Nullable Object source)
throws UnsupportedOperationException, ClassCastException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotCreateWithSourceMethod.invoke(null, source);
}
/**
* Save IOStatisticsSnapshot to a JSON string.
* @param snapshot statistics; may be null or of an incompatible type
* @return JSON string value or null if source is not an IOStatisticsSnapshot
* @throws UncheckedIOException Any IO/jackson exception.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public String iostatisticsSnapshot_toJsonString(@Nullable Serializable snapshot)
throws UncheckedIOException, UnsupportedOperationException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotToJsonStringMethod.invoke(null, snapshot);
}
/**
* Load IOStatisticsSnapshot from a JSON string.
* @param json JSON string value.
* @return deserialized snapshot.
* @throws UncheckedIOException Any IO/jackson exception.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public Serializable iostatisticsSnapshot_fromJsonString(
final String json) throws UncheckedIOException, UnsupportedOperationException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotFromJsonStringMethod.invoke(null, json);
}
/**
* Load IOStatisticsSnapshot from a Hadoop filesystem.
* @param fs filesystem
* @param path path
* @return the loaded snapshot
* @throws UncheckedIOException Any IO exception.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public Serializable iostatisticsSnapshot_load(
FileSystem fs,
Path path) throws UncheckedIOException, UnsupportedOperationException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotLoadMethod.invoke(null, fs, path);
}
/**
* Extract the IOStatistics from an object in a serializable form.
* @param source source object, may be null/not a statistics source/instance
* @return {@code IOStatisticsSnapshot} or null if the object is null/doesn't have statistics
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public Serializable iostatisticsSnapshot_retrieve(@Nullable Object source)
throws UnsupportedOperationException {
checkIoStatisticsAvailable();
return iostatisticsSnapshotRetrieveMethod.invoke(null, source);
}
/**
* Save IOStatisticsSnapshot to a Hadoop filesystem as a JSON file.
* @param snapshot statistics
* @param fs filesystem
* @param path path
* @param overwrite should any existing file be overwritten?
* @throws UncheckedIOException Any IO exception.
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public void iostatisticsSnapshot_save(
@Nullable Serializable snapshot,
FileSystem fs,
Path path,
boolean overwrite) throws UncheckedIOException, UnsupportedOperationException {
checkIoStatisticsAvailable();
iostatisticsSnapshotSaveMethod.invoke(null, snapshot, fs, path, overwrite);
}
/**
* Get the counters of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of counters.
*/
public Map<String, Long> iostatistics_counters(
Serializable source) {
return iostatisticsCountersMethod.invoke(null, source);
}
/**
* Get the gauges of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of gauges.
*/
public Map<String, Long> iostatistics_gauges(
Serializable source) {
return iostatisticsGaugesMethod.invoke(null, source);
}
/**
* Get the minimums of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of minimums.
*/
public Map<String, Long> iostatistics_minimums(
Serializable source) {
return iostatisticsMinimumsMethod.invoke(null, source);
}
/**
* Get the maximums of an IOStatisticsSnapshot.
* @param source source of statistics.
* @return the map of maximums.
*/
public Map<String, Long> iostatistics_maximums(
Serializable source) {
return iostatisticsMaximumsMethod.invoke(null, source);
}
/**
* Get the means of an IOStatisticsSnapshot.
* Each value in the map is the (sample, sum) tuple of the values;
* the mean is then calculated by dividing sum/sample wherever sample is non-zero.
* @param source source of statistics.
* @return a map of mean key to (sample, sum) tuples.
*/
public Map<String, Map.Entry<Long, Long>> iostatistics_means(
Serializable source) {
return iostatisticsMeansMethod.invoke(null, source);
}
/**
* Convert IOStatistics to a string form, with all the metrics sorted
* and empty value stripped.
* @param statistics A statistics instance.
* @return string value or the empty string if null
* @throws UnsupportedOperationException if the IOStatistics classes were not found
*/
public String iostatistics_toPrettyString(Object statistics) {
checkIoStatisticsAvailable();
return iostatisticsToPrettyStringMethod.invoke(null, statistics);
}
@Override
public String toString() {
return "DynamicWrappedStatistics{" +
"ioStatisticsAvailable =" + ioStatisticsAvailable() +
", ioStatisticsContextAvailable =" + ioStatisticsContextAvailable() +
'}';
}
}

View File

@ -0,0 +1,29 @@
/*
* 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.
*/
/**
* Implementation/testing support for wrapped IO.
*/
@InterfaceAudience.LimitedPrivate("testing")
@InterfaceStability.Unstable
package org.apache.hadoop.io.wrappedio.impl;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,35 @@
/*
* 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.
*/
/**
* Support for dynamic access to filesystem operations which are not available
* in older hadoop releases.
* <p>
* Classes in this package tagged as {@code @InterfaceAudience#Public} export
* methods to be loaded by reflection by other applications/libraries.
* Tests against these SHOULD use reflection themselves so as to guarantee
* stability of reflection-based access.
* <p>
* Classes tagged as private/limited private are for support and testing.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
package org.apache.hadoop.io.wrappedio;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,214 @@
/*
* 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.util.dynamic;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import static org.apache.hadoop.util.Preconditions.checkState;
/**
* Utility methods to assist binding to Hadoop APIs through reflection.
* Source: {@code org.apache.parquet.hadoop.util.wrapped.io.BindingUtils}.
*/
@InterfaceAudience.LimitedPrivate("testing")
@InterfaceStability.Unstable
public final class BindingUtils {
private static final Logger LOG = LoggerFactory.getLogger(BindingUtils.class);
private BindingUtils() {}
/**
* Load a class by name.
* @param className classname
* @return the class or null if it could not be loaded.
*/
public static Class<?> loadClass(String className) {
try {
return Class.forName(className);
} catch (ClassNotFoundException e) {
LOG.debug("No class {}", className, e);
return null;
}
}
/**
* Load a class by name.
* @param className classname
* @return the class.
* @throws RuntimeException if the class was not found.
*/
public static Class<?> loadClassSafely(String className) {
try {
return Class.forName(className);
} catch (ClassNotFoundException e) {
throw new RuntimeException(e);
}
}
/**
* Load a class by name.
* @param cl classloader to use.
* @param className classname
* @return the class or null if it could not be loaded.
*/
public static Class<?> loadClass(ClassLoader cl, String className) {
try {
return cl.loadClass(className);
} catch (ClassNotFoundException e) {
LOG.debug("No class {}", className, e);
return null;
}
}
/**
* Get an invocation from the source class, which will be unavailable() if
* the class is null or the method isn't found.
*
* @param <T> return type
* @param source source. If null, the method is a no-op.
* @param returnType return type class (unused)
* @param name method name
* @param parameterTypes parameters
*
* @return the method or "unavailable"
*/
public static <T> DynMethods.UnboundMethod loadInvocation(
Class<?> source, Class<? extends T> returnType, String name, Class<?>... parameterTypes) {
if (source != null) {
final DynMethods.UnboundMethod m = new DynMethods.Builder(name)
.impl(source, name, parameterTypes)
.orNoop()
.build();
if (m.isNoop()) {
// this is a sign of a mismatch between this class's expected
// signatures and actual ones.
// log at debug.
LOG.debug("Failed to load method {} from {}", name, source);
} else {
LOG.debug("Found method {} from {}", name, source);
}
return m;
} else {
return noop(name);
}
}
/**
* Load a static method from the source class, which will be a noop() if
* the class is null or the method isn't found.
* If the class and method are not found, then an {@code IllegalStateException}
* is raised on the basis that this means that the binding class is broken,
* rather than missing/out of date.
*
* @param <T> return type
* @param source source. If null, the method is a no-op.
* @param returnType return type class (unused)
* @param name method name
* @param parameterTypes parameters
*
* @return the method or a no-op.
* @throws IllegalStateException if the method is not static.
*/
public static <T> DynMethods.UnboundMethod loadStaticMethod(
Class<?> source, Class<? extends T> returnType, String name, Class<?>... parameterTypes) {
final DynMethods.UnboundMethod method =
loadInvocation(source, returnType, name, parameterTypes);
if (!available(method)) {
LOG.debug("Method not found: {}", name);
}
checkState(method.isStatic(), "Method is not static %s", method);
return method;
}
/**
* Create a no-op method.
*
* @param name method name
*
* @return a no-op method.
*/
public static DynMethods.UnboundMethod noop(final String name) {
return new DynMethods.Builder(name).orNoop().build();
}
/**
* Given a sequence of methods, verify that they are all available.
*
* @param methods methods
*
* @return true if they are all implemented
*/
public static boolean implemented(DynMethods.UnboundMethod... methods) {
for (DynMethods.UnboundMethod method : methods) {
if (method.isNoop()) {
return false;
}
}
return true;
}
/**
* Require a method to be available.
* @param method method to probe
* @throws UnsupportedOperationException if the method was not found.
*/
public static void checkAvailable(DynMethods.UnboundMethod method)
throws UnsupportedOperationException {
if (!available(method)) {
throw new UnsupportedOperationException("Unbound " + method);
}
}
/**
* Is a method available?
* @param method method to probe
* @return true iff the method is found and loaded.
*/
public static boolean available(DynMethods.UnboundMethod method) {
return !method.isNoop();
}
/**
* Invoke the supplier, catching any {@code UncheckedIOException} raised,
* extracting the inner IOException and rethrowing it.
* @param call call to invoke
* @return result
* @param <T> type of result
* @throws IOException if the call raised an IOException wrapped by an UncheckedIOException.
*/
public static <T> T extractIOEs(Supplier<T> call) throws IOException {
try {
return call.get();
} catch (UncheckedIOException e) {
throw e.getCause();
}
}
}

View File

@ -0,0 +1,273 @@
/*
* 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.util.dynamic;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import static org.apache.hadoop.util.dynamic.DynMethods.throwIfInstance;
import static org.apache.hadoop.util.Preconditions.checkArgument;
/**
* Dynamic constructors.
* Taken from {@code org.apache.parquet.util.DynConstructors}.
*/
@InterfaceAudience.LimitedPrivate("testing")
@InterfaceStability.Unstable
public class DynConstructors {
public static final class Ctor<C> extends DynMethods.UnboundMethod {
private final Constructor<C> ctor;
private final Class<? extends C> constructed;
private Ctor(Constructor<C> constructor, Class<? extends C> constructed) {
super(null, "newInstance");
this.ctor = constructor;
this.constructed = constructed;
}
public Class<? extends C> getConstructedClass() {
return constructed;
}
public C newInstanceChecked(Object... args) throws Exception {
try {
return ctor.newInstance(args);
} catch (InstantiationException | IllegalAccessException e) {
throw e;
} catch (InvocationTargetException e) {
throwIfInstance(e.getCause(), Exception.class);
throwIfInstance(e.getCause(), RuntimeException.class);
throw new RuntimeException(e.getCause());
}
}
public C newInstance(Object... args) {
try {
return newInstanceChecked(args);
} catch (Exception e) {
throwIfInstance(e, RuntimeException.class);
throw new RuntimeException(e);
}
}
@Override
@SuppressWarnings("unchecked")
public <R> R invoke(Object target, Object... args) {
checkArgument(target == null, "Invalid call to constructor: target must be null");
return (R) newInstance(args);
}
@Override
@SuppressWarnings("unchecked")
public <R> R invokeChecked(Object target, Object... args) throws Exception {
checkArgument(target == null, "Invalid call to constructor: target must be null");
return (R) newInstanceChecked(args);
}
@Override
public DynMethods.BoundMethod bind(Object receiver) {
throw new IllegalStateException("Cannot bind constructors");
}
@Override
public boolean isStatic() {
return true;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(constructor=" + ctor + ", class=" + constructed + ")";
}
}
public static class Builder {
private final Class<?> baseClass;
private ClassLoader loader = Thread.currentThread().getContextClassLoader();
private Ctor ctor = null;
private Map<String, Throwable> problems = new HashMap<String, Throwable>();
public Builder(Class<?> baseClass) {
this.baseClass = baseClass;
}
public Builder() {
this.baseClass = null;
}
/**
* Set the {@link ClassLoader} used to lookup classes by name.
* <p>
* If not set, the current thread's ClassLoader is used.
*
* @param value a ClassLoader
* @return this Builder for method chaining
*/
public Builder loader(ClassLoader value) {
this.loader = value;
return this;
}
public Builder impl(String className, Class<?>... types) {
// don't do any work if an implementation has been found
if (ctor != null) {
return this;
}
try {
Class<?> targetClass = Class.forName(className, true, loader);
impl(targetClass, types);
} catch (NoClassDefFoundError | ClassNotFoundException e) {
// cannot load this implementation
problems.put(className, e);
}
return this;
}
public <T> Builder impl(Class<T> targetClass, Class<?>... types) {
// don't do any work if an implementation has been found
if (ctor != null) {
return this;
}
try {
ctor = new Ctor<T>(targetClass.getConstructor(types), targetClass);
} catch (NoSuchMethodException e) {
// not the right implementation
problems.put(methodName(targetClass, types), e);
}
return this;
}
public Builder hiddenImpl(Class<?>... types) {
hiddenImpl(baseClass, types);
return this;
}
@SuppressWarnings("unchecked")
public Builder hiddenImpl(String className, Class<?>... types) {
// don't do any work if an implementation has been found
if (ctor != null) {
return this;
}
try {
Class targetClass = Class.forName(className, true, loader);
hiddenImpl(targetClass, types);
} catch (NoClassDefFoundError | ClassNotFoundException e) {
// cannot load this implementation
problems.put(className, e);
}
return this;
}
public <T> Builder hiddenImpl(Class<T> targetClass, Class<?>... types) {
// don't do any work if an implementation has been found
if (ctor != null) {
return this;
}
try {
Constructor<T> hidden = targetClass.getDeclaredConstructor(types);
AccessController.doPrivileged(new MakeAccessible(hidden));
ctor = new Ctor<T>(hidden, targetClass);
} catch (NoSuchMethodException | SecurityException e) {
// unusable or not the right implementation
problems.put(methodName(targetClass, types), e);
}
return this;
}
@SuppressWarnings("unchecked")
public <C> Ctor<C> buildChecked() throws NoSuchMethodException {
if (ctor != null) {
return ctor;
}
throw new NoSuchMethodException(
"Cannot find constructor for " + baseClass + "\n" + formatProblems(problems));
}
@SuppressWarnings("unchecked")
public <C> Ctor<C> build() {
if (ctor != null) {
return ctor;
}
throw new RuntimeException("Cannot find constructor for " + baseClass
+ "\n" + formatProblems(problems));
}
}
private static final class MakeAccessible implements PrivilegedAction<Void> {
private Constructor<?> hidden;
private MakeAccessible(Constructor<?> hidden) {
this.hidden = hidden;
}
@Override
public Void run() {
hidden.setAccessible(true);
return null;
}
}
private static String formatProblems(Map<String, Throwable> problems) {
StringBuilder sb = new StringBuilder();
boolean first = true;
for (Map.Entry<String, Throwable> problem : problems.entrySet()) {
if (first) {
first = false;
} else {
sb.append("\n");
}
sb.append("\tMissing ")
.append(problem.getKey())
.append(" [")
.append(problem.getValue().getClass().getName())
.append(": ")
.append(problem.getValue().getMessage())
.append("]");
}
return sb.toString();
}
private static String methodName(Class<?> targetClass, Class<?>... types) {
StringBuilder sb = new StringBuilder();
sb.append(targetClass.getName()).append("(");
boolean first = true;
for (Class<?> type : types) {
if (first) {
first = false;
} else {
sb.append(",");
}
sb.append(type.getName());
}
sb.append(")");
return sb.toString();
}
}

View File

@ -0,0 +1,544 @@
/*
* 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.util.dynamic;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.Arrays;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.util.Preconditions;
import static org.apache.hadoop.util.Preconditions.checkState;
/**
* Dynamic method invocation.
* Taken from {@code org.apache.parquet.util.DynMethods}.
*/
@InterfaceAudience.LimitedPrivate("testing")
@InterfaceStability.Unstable
public final class DynMethods {
private static final Logger LOG = LoggerFactory.getLogger(DynMethods.class);
private DynMethods() {
}
/**
* Convenience wrapper class around {@link Method}.
* <p>
* Allows callers to invoke the wrapped method with all Exceptions wrapped by
* RuntimeException, or with a single Exception catch block.
*/
public static class UnboundMethod {
private final Method method;
private final String name;
private final int argLength;
UnboundMethod(Method method, String name) {
this.method = method;
this.name = name;
this.argLength =
(method == null || method.isVarArgs()) ? -1 : method.getParameterTypes().length;
}
@SuppressWarnings("unchecked")
public <R> R invokeChecked(Object target, Object... args) throws Exception {
try {
if (argLength < 0) {
return (R) method.invoke(target, args);
} else {
if (argLength != args.length) {
LOG.error("expected {} arguments but got {}", argLength, args.length);
}
return (R) method.invoke(target, Arrays.copyOfRange(args, 0, argLength));
}
} catch (InvocationTargetException e) {
throwIfInstance(e.getCause(), Exception.class);
throwIfInstance(e.getCause(), RuntimeException.class);
throw new RuntimeException(e.getCause());
}
}
public <R> R invoke(Object target, Object... args) {
try {
return this.<R>invokeChecked(target, args);
} catch (Exception e) {
throwIfInstance(e, RuntimeException.class);
throw new RuntimeException(e);
}
}
/**
* Invoke a static method.
* @param args arguments.
* @return result.
* @param <R> type of result.
*/
public <R> R invokeStatic(Object... args) {
checkState(isStatic(), "Method is not static %s", toString());
return invoke(null, args);
}
/**
* Returns this method as a BoundMethod for the given receiver.
* @param receiver an Object to receive the method invocation
* @return a {@link BoundMethod} for this method and the receiver
* @throws IllegalStateException if the method is static
* @throws IllegalArgumentException if the receiver's class is incompatible
*/
public BoundMethod bind(Object receiver) {
checkState(!isStatic(), "Cannot bind static method %s",
method.toGenericString());
Preconditions.checkArgument(method.getDeclaringClass().isAssignableFrom(receiver.getClass()),
"Cannot bind %s to instance of %s", method.toGenericString(), receiver.getClass());
return new BoundMethod(this, receiver);
}
/**
* @return whether the method is a static method
*/
public boolean isStatic() {
return Modifier.isStatic(method.getModifiers());
}
/**
* @return whether the method is a noop
*/
public boolean isNoop() {
return this == NOOP;
}
/**
* Returns this method as a StaticMethod.
* @return a {@link StaticMethod} for this method
* @throws IllegalStateException if the method is not static
*/
public StaticMethod asStatic() {
checkState(isStatic(), "Method is not static");
return new StaticMethod(this);
}
public String toString() {
return "DynMethods.UnboundMethod(name=" + name + " method=" + method.toGenericString() + ")";
}
/**
* Singleton {@link UnboundMethod}, performs no operation and returns null.
*/
private static final UnboundMethod NOOP = new UnboundMethod(null, "NOOP") {
@Override
public <R> R invokeChecked(Object target, Object... args) throws Exception {
return null;
}
@Override
public BoundMethod bind(Object receiver) {
return new BoundMethod(this, receiver);
}
@Override
public StaticMethod asStatic() {
return new StaticMethod(this);
}
@Override
public boolean isStatic() {
return true;
}
@Override
public String toString() {
return "DynMethods.UnboundMethod(NOOP)";
}
};
}
public static final class BoundMethod {
private final UnboundMethod method;
private final Object receiver;
private BoundMethod(UnboundMethod method, Object receiver) {
this.method = method;
this.receiver = receiver;
}
public <R> R invokeChecked(Object... args) throws Exception {
return method.invokeChecked(receiver, args);
}
public <R> R invoke(Object... args) {
return method.invoke(receiver, args);
}
}
public static final class StaticMethod {
private final UnboundMethod method;
private StaticMethod(UnboundMethod method) {
this.method = method;
}
public <R> R invokeChecked(Object... args) throws Exception {
return method.invokeChecked(null, args);
}
public <R> R invoke(Object... args) {
return method.invoke(null, args);
}
}
/**
* If the given throwable is an instance of E, throw it as an E.
* @param t an exception instance
* @param excClass an exception class t may be an instance of
* @param <E> the type of exception that will be thrown if throwable is an instance
* @throws E if t is an instance of E
*/
@SuppressWarnings("unchecked")
public static <E extends Exception> void throwIfInstance(Throwable t, Class<E> excClass)
throws E {
if (excClass.isAssignableFrom(t.getClass())) {
// the throwable is already an exception, so throw it
throw (E)t;
}
}
public static final class Builder {
private final String name;
private ClassLoader loader = Thread.currentThread().getContextClassLoader();
private UnboundMethod method = null;
public Builder(String methodName) {
this.name = methodName;
}
/**
* Set the {@link ClassLoader} used to lookup classes by name.
* <p>
* If not set, the current thread's ClassLoader is used.
* @param classLoader a ClassLoader
* @return this Builder for method chaining
*/
public Builder loader(ClassLoader classLoader) {
this.loader = classLoader;
return this;
}
/**
* If no implementation has been found, adds a NOOP method.
* <p>
* Note: calls to impl will not match after this method is called!
* @return this Builder for method chaining
*/
public Builder orNoop() {
if (method == null) {
this.method = UnboundMethod.NOOP;
}
return this;
}
/**
* Checks for an implementation, first finding the given class by name.
* @param className name of a class
* @param methodName name of a method (different from constructor)
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder impl(String className, String methodName, Class<?>... argClasses) {
// don't do any work if an implementation has been found
if (method != null) {
return this;
}
try {
Class<?> targetClass = Class.forName(className, true, loader);
impl(targetClass, methodName, argClasses);
} catch (ClassNotFoundException e) {
// class not found on supplied classloader.
LOG.debug("failed to load class {}", className, e);
}
return this;
}
/**
* Checks for an implementation, first finding the given class by name.
* <p>
* The name passed to the constructor is the method name used.
* @param className name of a class
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder impl(String className, Class<?>... argClasses) {
impl(className, name, argClasses);
return this;
}
/**
* Checks for a method implementation.
* @param targetClass the class to check for an implementation
* @param methodName name of a method (different from constructor)
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder impl(Class<?> targetClass, String methodName, Class<?>... argClasses) {
// don't do any work if an implementation has been found
if (method != null) {
return this;
}
try {
this.method = new UnboundMethod(targetClass.getMethod(methodName, argClasses), name);
} catch (NoSuchMethodException e) {
// not the right implementation
LOG.debug("failed to load method {} from class {}", methodName, targetClass, e);
}
return this;
}
/**
* Checks for a method implementation.
* <p>
* The name passed to the constructor is the method name used.
* @param targetClass the class to check for an implementation
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder impl(Class<?> targetClass, Class<?>... argClasses) {
impl(targetClass, name, argClasses);
return this;
}
public Builder ctorImpl(Class<?> targetClass, Class<?>... argClasses) {
// don't do any work if an implementation has been found
if (method != null) {
return this;
}
try {
this.method = new DynConstructors.Builder().impl(targetClass, argClasses).buildChecked();
} catch (NoSuchMethodException e) {
// not the right implementation
LOG.debug("failed to load constructor arity {} from class {}", argClasses.length,
targetClass, e);
}
return this;
}
public Builder ctorImpl(String className, Class<?>... argClasses) {
// don't do any work if an implementation has been found
if (method != null) {
return this;
}
try {
this.method = new DynConstructors.Builder().impl(className, argClasses).buildChecked();
} catch (NoSuchMethodException e) {
// not the right implementation
LOG.debug("failed to load constructor arity {} from class {}", argClasses.length, className,
e);
}
return this;
}
/**
* Checks for an implementation, first finding the given class by name.
* @param className name of a class
* @param methodName name of a method (different from constructor)
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder hiddenImpl(String className, String methodName, Class<?>... argClasses) {
// don't do any work if an implementation has been found
if (method != null) {
return this;
}
try {
Class<?> targetClass = Class.forName(className, true, loader);
hiddenImpl(targetClass, methodName, argClasses);
} catch (ClassNotFoundException e) {
// class not found on supplied classloader.
LOG.debug("failed to load class {}", className, e);
}
return this;
}
/**
* Checks for an implementation, first finding the given class by name.
* <p>
* The name passed to the constructor is the method name used.
* @param className name of a class
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder hiddenImpl(String className, Class<?>... argClasses) {
hiddenImpl(className, name, argClasses);
return this;
}
/**
* Checks for a method implementation.
* @param targetClass the class to check for an implementation
* @param methodName name of a method (different from constructor)
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder hiddenImpl(Class<?> targetClass, String methodName, Class<?>... argClasses) {
// don't do any work if an implementation has been found
if (method != null) {
return this;
}
try {
Method hidden = targetClass.getDeclaredMethod(methodName, argClasses);
AccessController.doPrivileged(new MakeAccessible(hidden));
this.method = new UnboundMethod(hidden, name);
} catch (SecurityException | NoSuchMethodException e) {
// unusable or not the right implementation
LOG.debug("failed to load method {} from class {}", methodName, targetClass, e);
}
return this;
}
/**
* Checks for a method implementation.
* <p>
* The name passed to the constructor is the method name used.
* @param targetClass the class to check for an implementation
* @param argClasses argument classes for the method
* @return this Builder for method chaining
*/
public Builder hiddenImpl(Class<?> targetClass, Class<?>... argClasses) {
hiddenImpl(targetClass, name, argClasses);
return this;
}
/**
* Returns the first valid implementation as a UnboundMethod or throws a
* NoSuchMethodException if there is none.
* @return a {@link UnboundMethod} with a valid implementation
* @throws NoSuchMethodException if no implementation was found
*/
public UnboundMethod buildChecked() throws NoSuchMethodException {
if (method != null) {
return method;
} else {
throw new NoSuchMethodException("Cannot find method: " + name);
}
}
/**
* Returns the first valid implementation as a UnboundMethod or throws a
* RuntimeError if there is none.
* @return a {@link UnboundMethod} with a valid implementation
* @throws RuntimeException if no implementation was found
*/
public UnboundMethod build() {
if (method != null) {
return method;
} else {
throw new RuntimeException("Cannot find method: " + name);
}
}
/**
* Returns the first valid implementation as a BoundMethod or throws a
* NoSuchMethodException if there is none.
* @param receiver an Object to receive the method invocation
* @return a {@link BoundMethod} with a valid implementation and receiver
* @throws IllegalStateException if the method is static
* @throws IllegalArgumentException if the receiver's class is incompatible
* @throws NoSuchMethodException if no implementation was found
*/
public BoundMethod buildChecked(Object receiver) throws NoSuchMethodException {
return buildChecked().bind(receiver);
}
/**
* Returns the first valid implementation as a BoundMethod or throws a
* RuntimeError if there is none.
* @param receiver an Object to receive the method invocation
* @return a {@link BoundMethod} with a valid implementation and receiver
* @throws IllegalStateException if the method is static
* @throws IllegalArgumentException if the receiver's class is incompatible
* @throws RuntimeException if no implementation was found
*/
public BoundMethod build(Object receiver) {
return build().bind(receiver);
}
/**
* Returns the first valid implementation as a StaticMethod or throws a
* NoSuchMethodException if there is none.
* @return a {@link StaticMethod} with a valid implementation
* @throws IllegalStateException if the method is not static
* @throws NoSuchMethodException if no implementation was found
*/
public StaticMethod buildStaticChecked() throws NoSuchMethodException {
return buildChecked().asStatic();
}
/**
* Returns the first valid implementation as a StaticMethod or throws a
* RuntimeException if there is none.
* @return a {@link StaticMethod} with a valid implementation
* @throws IllegalStateException if the method is not static
* @throws RuntimeException if no implementation was found
*/
public StaticMethod buildStatic() {
return build().asStatic();
}
}
private static final class MakeAccessible implements PrivilegedAction<Void> {
private Method hidden;
MakeAccessible(Method hidden) {
this.hidden = hidden;
}
@Override
public Void run() {
hidden.setAccessible(true);
return null;
}
}
}

View File

@ -0,0 +1,31 @@
/*
* 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.
*/
/**
* Dynamic class loading and instantiation.
* Taken from {@code org.apache.parquet};
* there is also a fork of this in Apache Iceberg,
* so code using these classes should be relatively
* easily portable between the projects.
*/
@InterfaceAudience.LimitedPrivate("testing")
@InterfaceStability.Unstable
package org.apache.hadoop.util.dynamic;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.util.functional;
import java.io.IOException;
import java.io.UncheckedIOException;
/**
* Function of arity 2 which may raise an IOException.
@ -37,4 +38,19 @@ public interface BiFunctionRaisingIOE<T, U, R> {
* @throws IOException Any IO failure
*/
R apply(T t, U u) throws IOException;
/**
* Apply unchecked.
* @param t argument
* @param u argument 2
* @return the evaluated function
* @throws UncheckedIOException IOE raised.
*/
default R unchecked(T t, U u) {
try {
return apply(t, u);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
}

View File

@ -19,9 +19,14 @@
package org.apache.hadoop.util.functional;
import java.io.IOException;
import java.io.UncheckedIOException;
/**
* This is a callable which only raises an IOException.
* Its method {@link #unchecked()} invokes the {@link #apply()}
* method and wraps all IOEs in UncheckedIOException;
* call this if you need to pass this through java streaming
* APIs
* @param <R> return type
*/
@FunctionalInterface
@ -33,4 +38,18 @@ public interface CallableRaisingIOE<R> {
* @throws IOException Any IO failure
*/
R apply() throws IOException;
/**
* Apply unchecked.
* @return the evaluated call
* @throws UncheckedIOException IOE raised.
*/
default R unchecked() {
try {
return apply();
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.util.functional;
import java.io.IOException;
import java.io.UncheckedIOException;
/**
* Function of arity 1 which may raise an IOException.
@ -35,4 +36,18 @@ public interface FunctionRaisingIOE<T, R> {
* @throws IOException Any IO failure
*/
R apply(T t) throws IOException;
/**
* Apply unchecked.
* @param t argument
* @return the evaluated function
* @throws UncheckedIOException IOE raised.
*/
default R unchecked(T t) {
try {
return apply(t);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
}

View File

@ -20,6 +20,7 @@
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.function.Function;
import java.util.function.Supplier;
import org.apache.hadoop.classification.InterfaceAudience;
@ -42,11 +43,7 @@ private FunctionalIO() {
* @throws UncheckedIOException if an IOE was raised.
*/
public static <T> T uncheckIOExceptions(CallableRaisingIOE<T> call) {
try {
return call.apply();
} catch (IOException e) {
throw new UncheckedIOException(e);
}
return call.unchecked();
}
/**
@ -56,7 +53,7 @@ public static <T> T uncheckIOExceptions(CallableRaisingIOE<T> call) {
* @return a supplier which invokes the call.
*/
public static <T> Supplier<T> toUncheckedIOExceptionSupplier(CallableRaisingIOE<T> call) {
return () -> uncheckIOExceptions(call);
return call::unchecked;
}
/**
@ -75,4 +72,18 @@ public static <T> T extractIOExceptions(Supplier<T> call) throws IOException {
}
}
/**
* Convert a {@link FunctionRaisingIOE} as a {@link Supplier}.
* @param fun function to wrap
* @param <T> type of input
* @param <R> type of return value.
* @return a new function which invokes the inner function and wraps
* exceptions.
*/
public static <T, R> Function<T, R> toUncheckedFunction(FunctionRaisingIOE<T, R> fun) {
return fun::unchecked;
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.util.functional;
import java.util.Map;
import java.util.Objects;
import org.apache.hadoop.classification.InterfaceStability;
@ -83,5 +84,21 @@ public String toString() {
return "(" + key + ", " + value + ')';
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Tuple<?, ?> tuple = (Tuple<?, ?>) o;
return Objects.equals(key, tuple.key) && Objects.equals(value, tuple.value);
}
@Override
public int hashCode() {
return Objects.hash(key, value);
}
}
}

View File

@ -77,7 +77,7 @@ new `optLong()`, `optDouble()`, `mustLong()` and `mustDouble()` builder methods.
## Invariants
The `FutureDataInputStreamBuilder` interface does not require parameters or
or the state of `FileSystem` until [`build()`](#build) is
or the state of `FileSystem` until `build()` is
invoked and/or during the asynchronous open operation itself.
Some aspects of the state of the filesystem, MAY be checked in the initial
@ -377,20 +377,30 @@ performance -and vice versa.
subsystems.
1. If a policy is not recognized, the filesystem client MUST ignore it.
| Policy | Meaning |
|--------------|----------------------------------------------------------|
| `adaptive` | Any adaptive policy implemented by the store. |
| `default` | The default policy for this store. Generally "adaptive". |
| `random` | Optimize for random access. |
| `sequential` | Optimize for sequential access. |
| `vector` | The Vectored IO API is intended to be used. |
| `whole-file` | The whole file will be read. |
| Policy | Meaning |
|--------------|------------------------------------------------------------------------|
| `adaptive` | Any adaptive policy implemented by the store. |
| `avro` | This is an avro format which will be read sequentially |
| `csv` | This is CSV data which will be read sequentially |
| `default` | The default policy for this store. Generally "adaptive". |
| `columnar` | This is any columnar format other than ORC/parquet. |
| `hbase` | This is an HBase Table |
| `json` | This is a UTF-8 JSON/JSON lines format which will be read sequentially |
| `orc` | This is an ORC file. Optimize for it. |
| `parquet` | This is a Parquet file. Optimize for it. |
| `random` | Optimize for random access. |
| `sequential` | Optimize for sequential access. |
| `vector` | The Vectored IO API is intended to be used. |
| `whole-file` | The whole file will be read. |
Choosing the wrong read policy for an input source may be inefficient.
Choosing the wrong read policy for an input source may be inefficient but never fatal.
A list of read policies MAY be supplied; the first one recognized/supported by
the filesystem SHALL be the one used. This allows for custom policies to be
supported, for example an `hbase-hfile` policy optimized for HBase HFiles.
the filesystem SHALL be the one used. This allows for configurations which are compatible
across versions. A policy `parquet, columnar, vector, random, adaptive` will use the parquet policy for
any filesystem aware of it, falling back to `columnar`, `vector`, `random` and finally `adaptive`.
The S3A connector will recognize the `random` since Hadoop 3.3.5 (i.e. since the `openFile()` API
was added), and `vector` from Hadoop 3.4.0.
The S3A and ABFS input streams both implement
the [IOStatisticsSource](iostatistics.html) API, and can be queried for their IO
@ -425,7 +435,7 @@ sequential to random seek policies may be exensive.
When applications explicitly set the `fs.option.openfile.read.policy` option, if
they know their read plan, they SHOULD declare which policy is most appropriate.
#### <a name="read.policy.default"></a> Read Policy ``
#### <a name="read.policy.default"></a> Read Policy `default`
The default policy for the filesystem instance.
Implementation/installation-specific.
@ -473,7 +483,45 @@ Strategies can include:
Applications which know that the entire file is to be read from an opened stream SHOULD declare this
read policy.
### <a name="openfile.length"></a> Option: `fs.option.openfile.length`
#### <a name="read.policy.columnar"></a> Read Policy `columnar`
Declare that the data is some (unspecific) columnar format and that read sequencies
should be expected to be random IO of whole column stripes/rowgroups, possibly fetching associated
column statistics first, to determine whether a scan of a stripe/rowgroup can
be skipped entirely.
#### <a name="read.policy.fileformat.parquet"></a> File Format Read Policies `parquet`, and `orc`
These are read policies which declare that the file is of a specific columnar format
and that the input stream MAY be optimized for reading from these.
In particular
* File footers may be fetched and cached.
* Vector IO and random IO SHOULD be expected.
These read policies are a Hadoop 3.4.x addition, so applications and
libraries targeting multiple versions, SHOULD list their fallback
policies if these are not recognized, e.g. request a policy such as `parquet, vector, random`.
#### <a name="read.policy.fileformat.sequential"></a> File format Read Policies `avro`, `json` and `csv`
These are read policies which declare that the file is of a specific sequential format
and that the input stream MAY be optimized for reading from these.
These read policies are a Hadoop 3.4.x addition, so applications and
libraries targeting multiple versions, SHOULD list their fallback
policies if these are not recognized, e.g. request a policy such as `avro, sequential`.
#### <a name="read.policy.fileformat.hbase"></a> File Format Read Policy `hbase`
The file is an HBase table.
Use whatever policy is appropriate for these files, where `random` is
what should be used unless there are specific optimizations related to HBase.
### <a name="openfile.length"></a> Option: `fs.option.openfile.length`: `Long`
Declare the length of a file.
@ -499,7 +547,7 @@ If this option is used by the FileSystem implementation
* If a file status is supplied along with a value in `fs.opt.openfile.length`;
the file status values take precedence.
### <a name="split.start"></a> Options: `fs.option.openfile.split.start` and `fs.option.openfile.split.end`
### <a name="split.start"></a> Options: `fs.option.openfile.split.start` and `fs.option.openfile.split.end`: `Long`
Declare the start and end of the split when a file has been split for processing
in pieces.
@ -528,6 +576,21 @@ Therefore clients MUST be allowed to `seek()`/`read()` past the length
set in `fs.option.openfile.split.end` if the file is actually longer
than that value.
### <a name="footer.cache"></a> Option: `fs.option.openfile.footer.cache`: `Boolean`
Should a footer be cached?
* This is a hint for clients which cache footers.
* If a format with known footers are is declared in the read policy, the
default footer cache policy of that file type SHALL be used.
This option allows for that default policy to be overridden.
This is recommended if an application wishes to explicitly declare that Parquet/ORC files
are being read -but does not want or need the filesystem stream to cache any footer
because the application itself does such caching.
Duplicating footer caching is inefficient and if there is memory/memory cache conflict,
potentially counter-efficient.
## <a name="s3a"></a> S3A-specific options
The S3A Connector supports custom options for readahead and seek policy.

View File

@ -25,7 +25,6 @@
import java.util.Map;
import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -35,6 +34,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.wrappedio.WrappedIO;
import org.apache.hadoop.io.wrappedio.impl.DynamicWrappedIO;
import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
@ -43,6 +43,9 @@
/**
* Contract tests for bulk delete operation.
* Many of these tests use {@link WrappedIO} wrappers through reflection,
* to validate the codepath we expect libraries designed to work with
* multiple versions to use.
*/
public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractTestBase {
@ -66,11 +69,18 @@ public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractT
*/
protected FileSystem fs;
@Before
public void setUp() throws Exception {
/**
* Reflection support.
*/
private DynamicWrappedIO dynamicWrappedIO;
@Override
public void setup() throws Exception {
super.setup();
fs = getFileSystem();
basePath = path(getClass().getName());
pageSize = WrappedIO.bulkDelete_pageSize(getFileSystem(), basePath);
dynamicWrappedIO = new DynamicWrappedIO();
pageSize = dynamicWrappedIO.bulkDelete_pageSize(fs, basePath);
fs.mkdirs(basePath);
}
@ -103,15 +113,15 @@ public void testPathsSizeEqualsPageSizePrecondition() throws Exception {
@Test
public void testPathsSizeGreaterThanPageSizePrecondition() throws Exception {
List<Path> listOfPaths = createListOfPaths(pageSize + 1, basePath);
intercept(IllegalArgumentException.class,
() -> bulkDelete_delete(getFileSystem(), basePath, listOfPaths));
intercept(IllegalArgumentException.class, () ->
dynamicWrappedIO.bulkDelete_delete(getFileSystem(), basePath, listOfPaths));
}
@Test
public void testPathsSizeLessThanPageSizePrecondition() throws Exception {
List<Path> listOfPaths = createListOfPaths(pageSize - 1, basePath);
// Bulk delete call should pass with no exception.
bulkDelete_delete(getFileSystem(), basePath, listOfPaths);
dynamicWrappedIO.bulkDelete_delete(getFileSystem(), basePath, listOfPaths);
}
@Test
@ -285,7 +295,9 @@ public void testDeleteSamePathsMoreThanOnce() throws Exception {
*/
protected void pageSizePreconditionForTest(int size) {
if (size > pageSize) {
skip("Test requires paths size less than or equal to page size: " + pageSize);
skip("Test requires paths size less than or equal to page size: "
+ pageSize
+ "; actual size is " + size);
}
}

View File

@ -30,6 +30,7 @@
import org.apache.hadoop.fs.PathCapabilities;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.functional.RemoteIterators;
@ -651,6 +652,22 @@ public static void createFile(FileSystem fs,
Path path,
boolean overwrite,
byte[] data) throws IOException {
file(fs, path, overwrite, data);
}
/**
* Create a file, returning IOStatistics.
* @param fs filesystem
* @param path path to write
* @param overwrite overwrite flag
* @param data source dataset. Can be null
* @return any IOStatistics from the stream
* @throws IOException on any problem
*/
public static IOStatistics file(FileSystem fs,
Path path,
boolean overwrite,
byte[] data) throws IOException {
FSDataOutputStream stream = fs.create(path, overwrite);
try {
if (data != null && data.length > 0) {
@ -660,6 +677,7 @@ public static void createFile(FileSystem fs,
} finally {
IOUtils.closeStream(stream);
}
return stream.getIOStatistics();
}
/**

View File

@ -0,0 +1,484 @@
/*
* 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.io.wrappedio.impl;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.InputStream;
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.contract.localfs.LocalFSContract;
import org.apache.hadoop.io.wrappedio.WrappedIO;
import org.apache.hadoop.util.Lists;
import static java.nio.ByteBuffer.allocate;
import static org.apache.hadoop.fs.CommonPathCapabilities.BULK_DELETE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.fs.StreamCapabilities.IOSTATISTICS_CONTEXT;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.file;
import static org.apache.hadoop.util.dynamic.BindingUtils.loadClass;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.util.functional.Tuples.pair;
/**
* Test WrappedIO operations.
* <p>
* This is a contract test; the base class is bonded to the local fs;
* it is possible for other stores to implement themselves.
* All classes/constants are referenced here because they are part of the reflected
* API. If anything changes, application code breaks.
*/
public class TestWrappedIO extends AbstractFSContractTestBase {
private static final Logger LOG = LoggerFactory.getLogger(TestWrappedIO.class);
/**
* Dynamic wrapped IO.
*/
private DynamicWrappedIO io;
/**
* Dynamically Wrapped IO statistics.
*/
private DynamicWrappedStatistics statistics;
@Before
public void setup() throws Exception {
super.setup();
io = new DynamicWrappedIO();
statistics = new DynamicWrappedStatistics();
statistics.iostatisticsContext_reset();
}
@Override
public void teardown() throws Exception {
super.teardown();
logIOStatisticsContext();
}
@Override
protected AbstractFSContract createContract(final Configuration conf) {
return new LocalFSContract(conf);
}
/**
* Verify the {@link #clazz(String)} method raises an assertion
* if the class isn't found.
*/
@Test
public void testClassResolution() throws Throwable {
intercept(AssertionError.class, () -> clazz("no.such.class"));
}
@Test
public void testAllMethodsFound() throws Throwable {
io.requireAllMethodsAvailable();
}
/**
* Test the openFile operation.
* Lots of calls are made to read the same file to save on setup/teardown
* overhead and to allow for some statistics collection.
*/
@Test
public void testOpenFileOperations() throws Throwable {
Path path = path("testOpenFileOperations");
final int len = 100;
final byte[] data = dataset(len, 'a', 26);
final FileSystem fs = getFileSystem();
// create the file and any statistics from it.
final Serializable iostats = statistics.iostatisticsSnapshot_create(
file(fs, path, true, data));
final FileStatus st = fs.getFileStatus(path);
final boolean ioStatisticsContextCapability;
describe("reading file " + path);
try (FSDataInputStream in = DynamicWrappedIO.openFile(fs,
fs.getFileStatus(path),
DynamicWrappedIO.PARQUET_READ_POLICIES)) {
Assertions.assertThat(in.read())
.describedAs("first byte")
.isEqualTo('a');
ioStatisticsContextCapability = supportsIOStatisticsContext(in);
if (ioStatisticsContextCapability) {
LOG.info("Stream has IOStatisticsContext support: {}", in);
} else {
LOG.info("Stream has no IOStatisticsContext support: {}", in);
}
Assertions.assertThat(ioStatisticsContextCapability)
.describedAs("Retrieved stream capability %s from %s",
IOSTATISTICS_CONTEXT, in)
.isEqualTo(WrappedIO.streamCapabilities_hasCapability(in, IOSTATISTICS_CONTEXT));
Assertions.assertThat(ioStatisticsContextCapability)
.describedAs("Actual stream capability %s from %s",
IOSTATISTICS_CONTEXT, in)
.isEqualTo(in.hasCapability(IOSTATISTICS_CONTEXT));
retrieveAndAggregate(iostats, in);
}
// open with a status
try (FSDataInputStream s = openFile(path, null, st, null, null)) {
s.seek(1);
s.read();
// and do a small amount of statistics collection
retrieveAndAggregate(iostats, s);
}
// open with a length and random IO passed in the map
try (FSDataInputStream s = openFile(path, null, null,
(long) len,
map(pair(FS_OPTION_OPENFILE_READ_POLICY, "random")))) {
s.seek(len - 10);
s.read();
retrieveAndAggregate(iostats, s);
}
// now open a file with a length option greater than the file length
// this string is used in exception logging to report where in the
// sequence an IOE was raised.
String validationPoint = "openfile call";
// open with a length and random IO passed in via the map
try (FSDataInputStream s = openFile(path, null, null,
null,
map(pair(FS_OPTION_OPENFILE_LENGTH, len * 2),
pair(FS_OPTION_OPENFILE_READ_POLICY, "random")))) {
// fails if the file length was determined and fixed in open,
// and the stream doesn't permit seek() beyond the file length.
validationPoint = "seek()";
s.seek(len + 10);
validationPoint = "readFully()";
// readFully must fail.
s.readFully(len + 10, new byte[10], 0, 10);
Assertions.fail("Expected an EOFException but readFully from %s", s);
} catch (EOFException expected) {
// expected
LOG.info("EOF successfully raised, validation point: {}", validationPoint);
LOG.debug("stack", expected);
}
// if we get this far, do a bulk delete
Assertions.assertThat(io.pathCapabilities_hasPathCapability(fs, path, BULK_DELETE))
.describedAs("Path capability %s", BULK_DELETE)
.isTrue();
// first assert page size was picked up
Assertions.assertThat(io.bulkDelete_pageSize(fs, path))
.describedAs("bulkDelete_pageSize for %s", path)
.isGreaterThanOrEqualTo(1);
// then do the delete.
// pass in the parent path for the bulk delete to avoid HADOOP-19196
Assertions
.assertThat(io.bulkDelete_delete(fs, path.getParent(), Lists.newArrayList(path)))
.describedAs("outcome of bulk delete")
.isEmpty();
}
@Test
public void testOpenFileNotFound() throws Throwable {
Path path = path("testOpenFileNotFound");
intercept(FileNotFoundException.class, () ->
io.fileSystem_openFile(getFileSystem(), path, null, null, null, null));
}
/**
* Test ByteBufferPositionedReadable.
* This is implemented by HDFS but not much else; this test skips if the stream
* doesn't support it.
*/
@Test
public void testByteBufferPositionedReadable() throws Throwable {
Path path = path("testByteBufferPositionedReadable");
final int len = 100;
final byte[] data = dataset(len, 'a', 26);
final FileSystem fs = getFileSystem();
file(fs, path, true, data);
describe("reading file " + path);
try (FSDataInputStream in = openFile(path, "random", null, (long) len, null)) {
// skip rest of test if API is not found.
if (io.byteBufferPositionedReadable_readFullyAvailable(in)) {
LOG.info("ByteBufferPositionedReadable is available in {}", in);
ByteBuffer buffer = allocate(len);
io.byteBufferPositionedReadable_readFully(in, 0, buffer);
Assertions.assertThat(buffer.array())
.describedAs("Full buffer read of %s", in)
.isEqualTo(data);
// read from offset (verifies the offset is passed in)
final int offset = 10;
final int range = len - offset;
buffer = allocate(range);
io.byteBufferPositionedReadable_readFully(in, offset, buffer);
byte[] byteArray = new byte[range];
in.readFully(offset, byteArray);
Assertions.assertThat(buffer.array())
.describedAs("Offset buffer read of %s", in)
.isEqualTo(byteArray);
// now try to read past the EOF
// first verify the stream rejects this call directly
intercept(EOFException.class, () ->
in.readFully(len + 1, allocate(len)));
// then do the same through the wrapped API
intercept(EOFException.class, () ->
io.byteBufferPositionedReadable_readFully(in, len + 1, allocate(len)));
} else {
LOG.info("ByteBufferPositionedReadable is not available in {}", in);
// expect failures here
intercept(UnsupportedOperationException.class, () ->
io.byteBufferPositionedReadable_readFully(in, 0, allocate(len)));
}
}
}
@Test
public void testFilesystemIOStatistics() throws Throwable {
final FileSystem fs = getFileSystem();
final Serializable iostats = statistics.iostatisticsSnapshot_retrieve(fs);
if (iostats != null) {
final String status = statistics.iostatisticsSnapshot_toJsonString(iostats);
final Serializable roundTripped = statistics.iostatisticsSnapshot_fromJsonString(
status);
final Path path = methodPath();
statistics.iostatisticsSnapshot_save(roundTripped, fs, path, true);
final Serializable loaded = statistics.iostatisticsSnapshot_load(fs, path);
Assertions.assertThat(loaded)
.describedAs("loaded statistics from %s", path)
.isNotNull()
.satisfies(statistics::isIOStatisticsSnapshot);
LOG.info("loaded statistics {}",
statistics.iostatistics_toPrettyString(loaded));
}
}
/**
* Retrieve any IOStatistics from a class, and aggregate it to the
* existing IOStatistics.
* @param iostats statistics to update
* @param object statistics source
*/
private void retrieveAndAggregate(final Serializable iostats, final Object object) {
statistics.iostatisticsSnapshot_aggregate(iostats,
statistics.iostatisticsSnapshot_retrieve(object));
}
/**
* Log IOStatisticsContext if enabled.
*/
private void logIOStatisticsContext() {
// context IOStats
if (statistics.iostatisticsContext_enabled()) {
final Serializable iostats = statistics.iostatisticsContext_snapshot();
LOG.info("Context: {}",
toPrettyString(iostats));
} else {
LOG.info("IOStatisticsContext disabled");
}
}
private String toPrettyString(final Object iostats) {
return statistics.iostatistics_toPrettyString(iostats);
}
/**
* Does the object update the thread-local IOStatisticsContext?
* @param o object to cast to StreamCapabilities and probe for the capability.
* @return true if the methods were found, the interface implemented and the probe successful.
*/
private boolean supportsIOStatisticsContext(final Object o) {
return io.streamCapabilities_hasCapability(o, IOSTATISTICS_CONTEXT);
}
/**
* Open a file through dynamic invocation of {@link FileSystem#openFile(Path)}.
* @param path path
* @param policy read policy
* @param status optional file status
* @param length file length or null
* @param options nullable map of other options
* @return stream of the opened file
*/
private FSDataInputStream openFile(
final Path path,
final String policy,
final FileStatus status,
final Long length,
final Map<String, String> options) throws Throwable {
final FSDataInputStream stream = io.fileSystem_openFile(
getFileSystem(), path, policy, status, length, options);
Assertions.assertThat(stream)
.describedAs("null stream from openFile(%s)", path)
.isNotNull();
return stream;
}
/**
* Build a map from the tuples, which all have the value of
* their toString() method used.
* @param tuples object list (must be even)
* @return a map.
*/
private Map<String, String> map(Map.Entry<String, Object>... tuples) {
Map<String, String> map = new HashMap<>();
for (Map.Entry<String, Object> tuple : tuples) {
map.put(tuple.getKey(), tuple.getValue().toString());
}
return map;
}
/**
* Load a class by name; includes an assertion that the class was loaded.
* @param className classname
* @return the class.
*/
private static Class<?> clazz(final String className) {
final Class<?> clazz = loadClass(className);
Assertions.assertThat(clazz)
.describedAs("Class %s not found", className)
.isNotNull();
return clazz;
}
/**
* Simulate a no binding and verify that everything downgrades as expected.
*/
@Test
public void testNoWrappedClass() throws Throwable {
final DynamicWrappedIO broken = new DynamicWrappedIO(this.getClass().getName());
Assertions.assertThat(broken)
.describedAs("broken dynamic io %s", broken)
.matches(d -> !d.bulkDelete_available())
.matches(d -> !d.byteBufferPositionedReadable_available())
.matches(d -> !d.fileSystem_openFile_available());
final Path path = methodPath();
final FileSystem fs = getFileSystem();
// bulk deletes fail
intercept(UnsupportedOperationException.class, () ->
broken.bulkDelete_pageSize(fs, path));
intercept(UnsupportedOperationException.class, () ->
broken.bulkDelete_delete(fs, path, Lists.newArrayList()));
// openfile
intercept(UnsupportedOperationException.class, () ->
broken.fileSystem_openFile(fs, path, "", null, null, null));
// hasPathCapability downgrades
Assertions.assertThat(broken.pathCapabilities_hasPathCapability(fs, path, "anything"))
.describedAs("hasPathCapability(anything) via %s", broken)
.isFalse();
// byte buffer positioned readable
ContractTestUtils.touch(fs, path);
try (InputStream in = fs.open(path)) {
Assertions.assertThat(broken.byteBufferPositionedReadable_readFullyAvailable(in))
.describedAs("byteBufferPositionedReadable_readFullyAvailable on %s", in)
.isFalse();
intercept(UnsupportedOperationException.class, () ->
broken.byteBufferPositionedReadable_readFully(in, 0, allocate(1)));
}
}
/**
* Simulate a missing binding and verify that static methods fallback as required.
*/
@Test
public void testMissingClassFallbacks() throws Throwable {
Path path = path("testMissingClassFallbacks");
final FileSystem fs = getFileSystem();
file(fs, path, true, dataset(100, 'a', 26));
final DynamicWrappedIO broken = new DynamicWrappedIO(this.getClass().getName());
try (FSDataInputStream in = DynamicWrappedIO.openFileOnInstance(broken,
fs, fs.getFileStatus(path), DynamicWrappedIO.PARQUET_READ_POLICIES)) {
Assertions.assertThat(in.read())
.describedAs("first byte")
.isEqualTo('a');
}
}
/**
* Verify that if an attempt is made to bond to a class where the methods
* exist but are not static, that this fails during the object construction rather
* than on invocation.
*/
@Test
public void testNonStaticMethods() throws Throwable {
intercept(IllegalStateException.class, () ->
new DynamicWrappedIO(NonStaticBulkDeleteMethods.class.getName()));
}
/**
* This class declares the bulk delete methods, but as non-static; the expectation
* is that class loading will raise an {@link IllegalStateException}.
*/
private static final class NonStaticBulkDeleteMethods {
public int bulkDelete_pageSize(FileSystem ignoredFs, Path ignoredPath) {
return 0;
}
public List<Map.Entry<Path, String>> bulkDelete_delete(
FileSystem ignoredFs,
Path ignoredBase,
Collection<Path> ignoredPaths) {
return null;
}
}
}

View File

@ -0,0 +1,496 @@
/*
* 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.io.wrappedio.impl;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.io.UncheckedIOException;
import java.util.Map;
import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.statistics.IOStatisticsContext;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.apache.hadoop.util.functional.Tuples;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Unit tests for IOStatistics wrapping.
* <p>
* This mixes direct use of the API to generate statistics data for
* the reflection accessors to retrieve and manipulate.
*/
public class TestWrappedStatistics extends AbstractHadoopTestBase {
private static final Logger LOG = LoggerFactory.getLogger(TestWrappedIO.class);
/**
* Stub Serializable.
*/
private static final Serializable SERIALIZABLE = new Serializable() {};
/**
* Dynamically Wrapped IO statistics.
*/
private final DynamicWrappedStatistics statistics = new DynamicWrappedStatistics();
/**
* Local FS.
*/
private LocalFileSystem local;
/**
* Path to temporary file.
*/
private Path jsonPath;
@Before
public void setUp() throws Exception {
String testDataDir = new FileSystemTestHelper().getTestRootDir();
File tempDir = new File(testDataDir);
local = FileSystem.getLocal(new Configuration());
// Temporary file.
File jsonFile = new File(tempDir, "snapshot.json");
jsonPath = new Path(jsonFile.toURI());
}
/**
* The class must load, with all method groups available.
*/
@Test
public void testLoaded() throws Throwable {
Assertions.assertThat(statistics.ioStatisticsAvailable())
.describedAs("IOStatistics class must be available")
.isTrue();
Assertions.assertThat(statistics.ioStatisticsContextAvailable())
.describedAs("IOStatisticsContext must be available")
.isTrue();
}
@Test
public void testCreateEmptySnapshot() throws Throwable {
Assertions.assertThat(statistics.iostatisticsSnapshot_create())
.describedAs("iostatisticsSnapshot_create()")
.isInstanceOf(IOStatisticsSnapshot.class)
.satisfies(statistics::isIOStatisticsSnapshot)
.satisfies(statistics::isIOStatistics);
}
@Test
public void testCreateNullSource() throws Throwable {
Assertions.assertThat(statistics.iostatisticsSnapshot_create(null))
.describedAs("iostatisticsSnapshot_create(null)")
.isInstanceOf(IOStatisticsSnapshot.class);
}
@Test
public void testCreateOther() throws Throwable {
Assertions.assertThat(statistics.iostatisticsSnapshot_create(null))
.describedAs("iostatisticsSnapshot_create(null)")
.isInstanceOf(IOStatisticsSnapshot.class);
}
@Test
public void testCreateNonIOStatsSource() throws Throwable {
intercept(ClassCastException.class, () ->
statistics.iostatisticsSnapshot_create("hello"));
}
@Test
public void testRetrieveNullSource() throws Throwable {
Assertions.assertThat(statistics.iostatisticsSnapshot_retrieve(null))
.describedAs("iostatisticsSnapshot_retrieve(null)")
.isNull();
}
@Test
public void testRetrieveNonIOStatsSource() throws Throwable {
Assertions.assertThat(statistics.iostatisticsSnapshot_retrieve(this))
.describedAs("iostatisticsSnapshot_retrieve(this)")
.isNull();
}
/**
* Assert handling of json serialization for null value.
*/
@Test
public void testNullInstanceToJson() throws Throwable {
intercept(IllegalArgumentException.class, () -> toJsonString(null));
}
/**
* Assert handling of json serialization for wrong value.
*/
@Test
public void testWrongSerializableTypeToJson() throws Throwable {
intercept(IllegalArgumentException.class, () -> toJsonString(SERIALIZABLE));
}
/**
* Try to aggregate into the wrong type.
*/
@Test
public void testAggregateWrongSerializable() throws Throwable {
intercept(IllegalArgumentException.class, () ->
statistics.iostatisticsSnapshot_aggregate(SERIALIZABLE,
statistics.iostatisticsContext_getCurrent()));
}
/**
* Try to save the wrong type.
*/
@Test
public void testSaveWrongSerializable() throws Throwable {
intercept(IllegalArgumentException.class, () ->
statistics.iostatisticsSnapshot_save(SERIALIZABLE, local, jsonPath, true));
}
/**
* Test all the IOStatisticsContext operations, including
* JSON round trip of the statistics.
*/
@Test
public void testIOStatisticsContextMethods() {
Assertions.assertThat(statistics.ioStatisticsContextAvailable())
.describedAs("ioStatisticsContextAvailable() of %s", statistics)
.isTrue();
Assertions.assertThat(statistics.iostatisticsContext_enabled())
.describedAs("iostatisticsContext_enabled() of %s", statistics)
.isTrue();
// get the current context, validate it
final Object current = statistics.iostatisticsContext_getCurrent();
Assertions.assertThat(current)
.describedAs("IOStatisticsContext")
.isInstanceOf(IOStatisticsContext.class)
.satisfies(statistics::isIOStatisticsSource);
// take a snapshot
final Serializable snapshot = statistics.iostatisticsContext_snapshot();
Assertions.assertThat(snapshot)
.satisfies(statistics::isIOStatisticsSnapshot);
// use the retrieve API to create a snapshot from the IOStatisticsSource interface
final Serializable retrieved = statistics.iostatisticsSnapshot_retrieve(current);
assertJsonEqual(retrieved, snapshot);
// to/from JSON
final String json = toJsonString(snapshot);
LOG.info("Serialized to json {}", json);
final Serializable snap2 = statistics.iostatisticsSnapshot_fromJsonString(json);
assertJsonEqual(snap2, snapshot);
// get the values
statistics.iostatistics_counters(snapshot);
statistics.iostatistics_gauges(snapshot);
statistics.iostatistics_minimums(snapshot);
statistics.iostatistics_maximums(snapshot);
statistics.iostatistics_means(snapshot);
// set to null
statistics.iostatisticsContext_setThreadIOStatisticsContext(null);
Assertions.assertThat(statistics.iostatisticsContext_getCurrent())
.describedAs("current IOStatisticsContext after resetting")
.isNotSameAs(current);
// then set to the "current" value
statistics.iostatisticsContext_setThreadIOStatisticsContext(current);
Assertions.assertThat(statistics.iostatisticsContext_getCurrent())
.describedAs("current IOStatisticsContext after resetting")
.isSameAs(current);
// and reset
statistics.iostatisticsContext_reset();
// now aggregate the retrieved stats into it.
Assertions.assertThat(statistics.iostatisticsContext_aggregate(retrieved))
.describedAs("iostatisticsContext_aggregate of %s", retrieved)
.isTrue();
}
/**
* Perform some real IOStatisticsContext operations.
*/
@Test
public void testIOStatisticsContextInteraction() {
statistics.iostatisticsContext_reset();
// create a snapshot with a counter
final IOStatisticsSnapshot snapshot =
(IOStatisticsSnapshot) statistics.iostatisticsSnapshot_create();
snapshot.setCounter("c1", 10);
// aggregate twice
statistics.iostatisticsContext_aggregate(snapshot);
statistics.iostatisticsContext_aggregate(snapshot);
// take a snapshot
final IOStatisticsSnapshot snap2 =
(IOStatisticsSnapshot) statistics.iostatisticsContext_snapshot();
// assert the valuue
assertThatStatisticCounter(snap2, "c1")
.isEqualTo(20);
}
/**
* Expect that two IOStatisticsInstances serialized to exactly the same JSON.
* @param actual actual value.
* @param expected expected value
*/
private void assertJsonEqual(Serializable actual, Serializable expected) {
Assertions.assertThat(toJsonString(actual))
.describedAs("JSON format string of %s", actual)
.isEqualTo(toJsonString(expected));
}
/**
* Convert a snapshot to a JSON string.
* @param snapshot IOStatisticsSnapshot
* @return a JSON serialization.
*/
private String toJsonString(final Serializable snapshot) {
return statistics.iostatisticsSnapshot_toJsonString(snapshot);
}
/**
* Create an empty snapshot, save it then load back.
*/
@Test
public void testLocalSaveOfEmptySnapshot() throws Throwable {
final Serializable snapshot = statistics.iostatisticsSnapshot_create();
statistics.iostatisticsSnapshot_save(snapshot, local, jsonPath, true);
final Serializable loaded = statistics.iostatisticsSnapshot_load(local, jsonPath);
LOG.info("loaded statistics {}",
statistics.iostatistics_toPrettyString(loaded));
// now try to save over the same path with overwrite false
intercept(UncheckedIOException.class, () ->
statistics.iostatisticsSnapshot_save(snapshot, local, jsonPath, false));
// after delete the load fails
local.delete(jsonPath, false);
intercept(UncheckedIOException.class, () ->
statistics.iostatisticsSnapshot_load(local, jsonPath));
}
/**
* Build up a complex statistic and assert extraction on it.
*/
@Test
public void testStatisticExtraction() throws Throwable {
final IOStatisticsStore store = IOStatisticsBinding.iostatisticsStore()
.withCounters("c1", "c2")
.withGauges("g1")
.withDurationTracking("d1", "d2")
.build();
store.incrementCounter("c1");
store.setGauge("g1", 10);
trackDurationOfInvocation(store, "d1", () ->
sleep(20));
store.trackDuration("d1").close();
intercept(IOException.class, () ->
trackDurationOfInvocation(store, "d2", () -> {
sleep(10);
throw new IOException("generated");
}));
final Serializable snapshot = statistics.iostatisticsSnapshot_create(store);
// complex round trip
statistics.iostatisticsSnapshot_save(snapshot, local, jsonPath, true);
final Serializable loaded = statistics.iostatisticsSnapshot_load(local, jsonPath);
LOG.info("loaded statistics {}",
statistics.iostatistics_toPrettyString(loaded));
assertJsonEqual(loaded, snapshot);
// get the values
Assertions.assertThat(statistics.iostatistics_counters(loaded))
.containsOnlyKeys("c1", "c2",
"d1", "d1.failures",
"d2", "d2.failures")
.containsEntry("c1", 1L)
.containsEntry("d1", 2L)
.containsEntry("d2", 1L);
Assertions.assertThat(statistics.iostatistics_gauges(loaded))
.containsOnlyKeys("g1")
.containsEntry("g1", 10L);
final Map<String, Long> minimums = statistics.iostatistics_minimums(snapshot);
Assertions.assertThat(minimums)
.containsEntry("d1.min", 0L);
final long d2FailuresMin = minimums.get("d2.failures.min");
Assertions.assertThat(d2FailuresMin)
.describedAs("min d2.failures")
.isGreaterThan(0);
final Map<String, Long> maximums = statistics.iostatistics_maximums(snapshot);
Assertions.assertThat(maximums)
.containsEntry("d2.failures.max", d2FailuresMin);
final long d1Max = maximums.get("d1.max");
final Map<String, Map.Entry<Long, Long>> means =
statistics.iostatistics_means(snapshot);
Assertions.assertThat(means)
.containsEntry("d1.mean", Tuples.pair(2L, d1Max))
.containsEntry("d2.failures.mean", Tuples.pair(1L, d2FailuresMin));
}
/**
* Sleep for some milliseconds; interruptions are swallowed.
* @param millis time in milliseconds
*/
private static void sleep(final int millis) {
try {
Thread.sleep(millis);
} catch (InterruptedException ignored) {
}
}
/**
* Bind to an empty class to simulate a runtime where none of the methods were found
* through reflection, and verify the expected failure semantics.
*/
@Test
public void testMissingIOStatisticsMethods() throws Throwable {
final DynamicWrappedStatistics missing =
new DynamicWrappedStatistics(StubClass.class.getName());
// probes which just return false
Assertions.assertThat(missing.ioStatisticsAvailable())
.describedAs("ioStatisticsAvailable() of %s", missing)
.isFalse();
// probes of type of argument which return false if the
// methods are missing
Assertions.assertThat(missing.isIOStatistics(SERIALIZABLE))
.describedAs("isIOStatistics() of %s", missing)
.isFalse();
Assertions.assertThat(missing.isIOStatisticsSource(SERIALIZABLE))
.describedAs("isIOStatisticsSource() of %s", missing)
.isFalse();
Assertions.assertThat(missing.isIOStatisticsSnapshot(SERIALIZABLE))
.describedAs("isIOStatisticsSnapshot() of %s", missing)
.isFalse();
// operations which raise exceptions
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_create());
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_create(this));
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_aggregate(SERIALIZABLE, this));
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_fromJsonString("{}"));
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_toJsonString(SERIALIZABLE));
final Path path = new Path("/");
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_load(local, path));
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_save(SERIALIZABLE, local, path, true));
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsSnapshot_retrieve(this));
intercept(UnsupportedOperationException.class, () ->
missing.iostatistics_toPrettyString(this));
}
/**
* Empty class to bind against and ensure all methods fail to bind.
*/
private static final class StubClass { }
/**
* Bind to {@link StubClass} to simulate a runtime where none of the methods were found
* through reflection, and verify the expected failure semantics.
*/
@Test
public void testMissingContextMethods() throws Throwable {
final DynamicWrappedStatistics missing =
new DynamicWrappedStatistics(StubClass.class.getName());
// probes which just return false
Assertions.assertThat(missing.ioStatisticsContextAvailable())
.describedAs("ioStatisticsContextAvailable() of %s", missing)
.isFalse();
Assertions.assertThat(missing.iostatisticsContext_enabled())
.describedAs("iostatisticsContext_enabled() of %s", missing)
.isFalse();
// operations which raise exceptions
intercept(UnsupportedOperationException.class, missing::iostatisticsContext_reset);
intercept(UnsupportedOperationException.class, missing::iostatisticsContext_getCurrent);
intercept(UnsupportedOperationException.class, missing::iostatisticsContext_snapshot);
intercept(UnsupportedOperationException.class, () ->
missing.iostatisticsContext_setThreadIOStatisticsContext(null));
}
/**
* Validate class checks in {@code iostatisticsSnapshot_aggregate()}.
*/
@Test
public void testStatisticCasting() throws Throwable {
Serializable iostats = statistics.iostatisticsSnapshot_create(null);
final String wrongType = "wrong type";
intercept(IllegalArgumentException.class, () ->
statistics.iostatisticsSnapshot_aggregate(iostats, wrongType));
}
}

View File

@ -0,0 +1,85 @@
/*
* 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.util.dynamic;
/**
* This is a class for testing {@link DynMethods} and {@code DynConstructors}.
* <p>
* Derived from {@code org.apache.parquet.util} test suites.
*/
public class Concatenator {
public static class SomeCheckedException extends Exception {
}
private String sep = "";
public Concatenator() {
}
public Concatenator(String sep) {
this.sep = sep;
}
private Concatenator(char sep) {
this.sep = String.valueOf(sep);
}
public Concatenator(Exception e) throws Exception {
throw e;
}
public static Concatenator newConcatenator(String sep) {
return new Concatenator(sep);
}
private void setSeparator(String value) {
this.sep = value;
}
public String concat(String left, String right) {
return left + sep + right;
}
public String concat(String left, String middle, String right) {
return left + sep + middle + sep + right;
}
public String concat(Exception e) throws Exception {
throw e;
}
public String concat(String... strings) {
if (strings.length >= 1) {
StringBuilder sb = new StringBuilder();
sb.append(strings[0]);
for (int i = 1; i < strings.length; i += 1) {
sb.append(sep);
sb.append(strings[i]);
}
return sb.toString();
}
return null;
}
public static String cat(String... strings) {
return new Concatenator().concat(strings);
}
}

View File

@ -0,0 +1,170 @@
/*
* 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.util.dynamic;
import java.util.concurrent.Callable;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Derived from {@code org.apache.parquet.util} test suites.
*/
public class TestDynConstructors extends AbstractHadoopTestBase {
@Test
public void testNoImplCall() throws Exception {
final DynConstructors.Builder builder = new DynConstructors.Builder();
intercept(NoSuchMethodException.class,
(Callable<DynMethods.UnboundMethod>) builder::buildChecked);
intercept(RuntimeException.class, () ->
builder.build());
}
@Test
public void testMissingClass() throws Exception {
final DynConstructors.Builder builder = new DynConstructors.Builder()
.impl("not.a.RealClass");
intercept(NoSuchMethodException.class,
(Callable<DynMethods.UnboundMethod>) builder::buildChecked);
intercept(RuntimeException.class, (Callable<DynMethods.UnboundMethod>) builder::build);
}
@Test
public void testMissingConstructor() throws Exception {
final DynConstructors.Builder builder = new DynConstructors.Builder()
.impl(Concatenator.class, String.class, String.class);
intercept(NoSuchMethodException.class,
(Callable<DynMethods.UnboundMethod>) builder::buildChecked);
intercept(RuntimeException.class,
(Callable<DynMethods.UnboundMethod>) builder::build);
}
@Test
public void testFirstImplReturned() throws Exception {
final DynConstructors.Ctor<Concatenator> sepCtor = new DynConstructors.Builder()
.impl("not.a.RealClass", String.class)
.impl(Concatenator.class, String.class)
.impl(Concatenator.class)
.buildChecked();
Concatenator dashCat = sepCtor.newInstanceChecked("-");
Assert.assertEquals("Should construct with the 1-arg version",
"a-b", dashCat.concat("a", "b"));
intercept(IllegalArgumentException.class, () ->
sepCtor.newInstanceChecked("/", "-"));
intercept(IllegalArgumentException.class, () ->
sepCtor.newInstance("/", "-"));
DynConstructors.Ctor<Concatenator> defaultCtor = new DynConstructors.Builder()
.impl("not.a.RealClass", String.class)
.impl(Concatenator.class)
.impl(Concatenator.class, String.class)
.buildChecked();
Concatenator cat = defaultCtor.newInstanceChecked();
Assert.assertEquals("Should construct with the no-arg version",
"ab", cat.concat("a", "b"));
}
@Test
public void testExceptionThrown() throws Exception {
final Concatenator.SomeCheckedException exc = new Concatenator.SomeCheckedException();
final DynConstructors.Ctor<Concatenator> sepCtor = new DynConstructors.Builder()
.impl("not.a.RealClass", String.class)
.impl(Concatenator.class, Exception.class)
.buildChecked();
intercept(Concatenator.SomeCheckedException.class, () ->
sepCtor.newInstanceChecked(exc));
intercept(RuntimeException.class, () -> sepCtor.newInstance(exc));
}
@Test
public void testStringClassname() throws Exception {
final DynConstructors.Ctor<Concatenator> sepCtor = new DynConstructors.Builder()
.impl(Concatenator.class.getName(), String.class)
.buildChecked();
Assert.assertNotNull("Should find 1-arg constructor", sepCtor.newInstance("-"));
}
@Test
public void testHiddenMethod() throws Exception {
intercept(NoSuchMethodException.class, () ->
new DynMethods.Builder("setSeparator")
.impl(Concatenator.class, char.class)
.buildChecked());
final DynConstructors.Ctor<Concatenator> sepCtor = new DynConstructors.Builder()
.hiddenImpl(Concatenator.class.getName(), char.class)
.buildChecked();
Assert.assertNotNull("Should find hidden ctor with hiddenImpl", sepCtor);
Concatenator slashCat = sepCtor.newInstanceChecked('/');
Assert.assertEquals("Should use separator /",
"a/b", slashCat.concat("a", "b"));
}
@Test
public void testBind() throws Exception {
final DynConstructors.Ctor<Concatenator> ctor = new DynConstructors.Builder()
.impl(Concatenator.class.getName())
.buildChecked();
Assert.assertTrue("Should always be static", ctor.isStatic());
intercept(IllegalStateException.class, () ->
ctor.bind(null));
}
@Test
public void testInvoke() throws Exception {
final DynMethods.UnboundMethod ctor = new DynConstructors.Builder()
.impl(Concatenator.class.getName())
.buildChecked();
intercept(IllegalArgumentException.class, () ->
ctor.invokeChecked("a"));
intercept(IllegalArgumentException.class, () ->
ctor.invoke("a"));
Assert.assertNotNull("Should allow invokeChecked(null, ...)",
ctor.invokeChecked(null));
Assert.assertNotNull("Should allow invoke(null, ...)",
ctor.invoke(null));
}
}

View File

@ -0,0 +1,320 @@
/*
* 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.util.dynamic;
import java.util.concurrent.Callable;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Copied from {@code org.apache.parquet.util} test suites.
*/
public class TestDynMethods extends AbstractHadoopTestBase {
@Test
public void testNoImplCall() throws Exception {
final DynMethods.Builder builder = new DynMethods.Builder("concat");
intercept(NoSuchMethodException.class,
(Callable<DynMethods.UnboundMethod>) builder::buildChecked);
intercept(RuntimeException.class,
(Callable<DynMethods.UnboundMethod>) builder::build);
}
@Test
public void testMissingClass() throws Exception {
final DynMethods.Builder builder = new DynMethods.Builder("concat")
.impl("not.a.RealClass", String.class, String.class);
intercept(NoSuchMethodException.class,
(Callable<DynMethods.UnboundMethod>) builder::buildChecked);
intercept(RuntimeException.class, () ->
builder.build());
}
@Test
public void testMissingMethod() throws Exception {
final DynMethods.Builder builder = new DynMethods.Builder("concat")
.impl(Concatenator.class, "cat2strings", String.class, String.class);
intercept(NoSuchMethodException.class,
(Callable<DynMethods.UnboundMethod>) builder::buildChecked);
intercept(RuntimeException.class, () ->
builder.build());
}
@Test
public void testFirstImplReturned() throws Exception {
Concatenator obj = new Concatenator("-");
DynMethods.UnboundMethod cat2 = new DynMethods.Builder("concat")
.impl("not.a.RealClass", String.class, String.class)
.impl(Concatenator.class, String.class, String.class)
.impl(Concatenator.class, String.class, String.class, String.class)
.buildChecked();
Assert.assertEquals("Should call the 2-arg version successfully",
"a-b", cat2.invoke(obj, "a", "b"));
Assert.assertEquals("Should ignore extra arguments",
"a-b", cat2.invoke(obj, "a", "b", "c"));
DynMethods.UnboundMethod cat3 = new DynMethods.Builder("concat")
.impl("not.a.RealClass", String.class, String.class)
.impl(Concatenator.class, String.class, String.class, String.class)
.impl(Concatenator.class, String.class, String.class)
.build();
Assert.assertEquals("Should call the 3-arg version successfully",
"a-b-c", cat3.invoke(obj, "a", "b", "c"));
Assert.assertEquals("Should call the 3-arg version null padding",
"a-b-null", cat3.invoke(obj, "a", "b"));
}
@Test
public void testVarArgs() throws Exception {
DynMethods.UnboundMethod cat = new DynMethods.Builder("concat")
.impl(Concatenator.class, String[].class)
.buildChecked();
Assert.assertEquals("Should use the varargs version", "abcde",
cat.invokeChecked(
new Concatenator(),
(Object) new String[]{"a", "b", "c", "d", "e"}));
Assert.assertEquals("Should use the varargs version", "abcde",
cat.bind(new Concatenator())
.invokeChecked((Object) new String[]{"a", "b", "c", "d", "e"}));
}
@Test
public void testIncorrectArguments() throws Exception {
final Concatenator obj = new Concatenator("-");
final DynMethods.UnboundMethod cat = new DynMethods.Builder("concat")
.impl("not.a.RealClass", String.class, String.class)
.impl(Concatenator.class, String.class, String.class)
.buildChecked();
intercept(IllegalArgumentException.class, () ->
cat.invoke(obj, 3, 4));
intercept(IllegalArgumentException.class, () ->
cat.invokeChecked(obj, 3, 4));
}
@Test
public void testExceptionThrown() throws Exception {
final Concatenator.SomeCheckedException exc = new Concatenator.SomeCheckedException();
final Concatenator obj = new Concatenator("-");
final DynMethods.UnboundMethod cat = new DynMethods.Builder("concat")
.impl("not.a.RealClass", String.class, String.class)
.impl(Concatenator.class, Exception.class)
.buildChecked();
intercept(Concatenator.SomeCheckedException.class, () ->
cat.invokeChecked(obj, exc));
intercept(RuntimeException.class, () ->
cat.invoke(obj, exc));
}
@Test
public void testNameChange() throws Exception {
Concatenator obj = new Concatenator("-");
DynMethods.UnboundMethod cat = new DynMethods.Builder("cat")
.impl(Concatenator.class, "concat", String.class, String.class)
.buildChecked();
Assert.assertEquals("Should find 2-arg concat method",
"a-b", cat.invoke(obj, "a", "b"));
}
@Test
public void testStringClassname() throws Exception {
Concatenator obj = new Concatenator("-");
DynMethods.UnboundMethod cat = new DynMethods.Builder("concat")
.impl(Concatenator.class.getName(), String.class, String.class)
.buildChecked();
Assert.assertEquals("Should find 2-arg concat method",
"a-b", cat.invoke(obj, "a", "b"));
}
@Test
public void testHiddenMethod() throws Exception {
Concatenator obj = new Concatenator("-");
intercept(NoSuchMethodException.class, () ->
new DynMethods.Builder("setSeparator")
.impl(Concatenator.class, String.class)
.buildChecked());
DynMethods.UnboundMethod changeSep = new DynMethods.Builder("setSeparator")
.hiddenImpl(Concatenator.class, String.class)
.buildChecked();
Assert.assertNotNull("Should find hidden method with hiddenImpl",
changeSep);
changeSep.invokeChecked(obj, "/");
Assert.assertEquals("Should use separator / instead of -",
"a/b", obj.concat("a", "b"));
}
@Test
public void testBoundMethod() throws Exception {
DynMethods.UnboundMethod cat = new DynMethods.Builder("concat")
.impl(Concatenator.class, String.class, String.class)
.buildChecked();
// Unbound methods can be bound multiple times
DynMethods.BoundMethod dashCat = cat.bind(new Concatenator("-"));
DynMethods.BoundMethod underCat = cat.bind(new Concatenator("_"));
Assert.assertEquals("Should use '-' object without passing",
"a-b", dashCat.invoke("a", "b"));
Assert.assertEquals("Should use '_' object without passing",
"a_b", underCat.invoke("a", "b"));
DynMethods.BoundMethod slashCat = new DynMethods.Builder("concat")
.impl(Concatenator.class, String.class, String.class)
.buildChecked(new Concatenator("/"));
Assert.assertEquals("Should use bound object from builder without passing",
"a/b", slashCat.invoke("a", "b"));
}
@Test
public void testBindStaticMethod() throws Exception {
final DynMethods.Builder builder = new DynMethods.Builder("cat")
.impl(Concatenator.class, String[].class);
intercept(IllegalStateException.class, () ->
builder.buildChecked(new Concatenator()));
intercept(IllegalStateException.class, () ->
builder.build(new Concatenator()));
final DynMethods.UnboundMethod staticCat = builder.buildChecked();
Assert.assertTrue("Should be static", staticCat.isStatic());
intercept(IllegalStateException.class, () ->
staticCat.bind(new Concatenator()));
}
@Test
public void testStaticMethod() throws Exception {
DynMethods.StaticMethod staticCat = new DynMethods.Builder("cat")
.impl(Concatenator.class, String[].class)
.buildStaticChecked();
Assert.assertEquals("Should call varargs static method cat(String...)",
"abcde", staticCat.invokeChecked(
(Object) new String[]{"a", "b", "c", "d", "e"}));
}
@Test
public void testNonStaticMethod() throws Exception {
final DynMethods.Builder builder = new DynMethods.Builder("concat")
.impl(Concatenator.class, String.class, String.class);
intercept(IllegalStateException.class, builder::buildStatic);
intercept(IllegalStateException.class, builder::buildStaticChecked);
final DynMethods.UnboundMethod cat2 = builder.buildChecked();
Assert.assertFalse("concat(String,String) should not be static",
cat2.isStatic());
intercept(IllegalStateException.class, cat2::asStatic);
}
@Test
public void testConstructorImpl() throws Exception {
final DynMethods.Builder builder = new DynMethods.Builder("newConcatenator")
.ctorImpl(Concatenator.class, String.class)
.impl(Concatenator.class, String.class);
DynMethods.UnboundMethod newConcatenator = builder.buildChecked();
Assert.assertTrue("Should find constructor implementation",
newConcatenator instanceof DynConstructors.Ctor);
Assert.assertTrue("Constructor should be a static method",
newConcatenator.isStatic());
Assert.assertFalse("Constructor should not be NOOP",
newConcatenator.isNoop());
// constructors cannot be bound
intercept(IllegalStateException.class, () ->
builder.buildChecked(new Concatenator()));
intercept(IllegalStateException.class, () ->
builder.build(new Concatenator()));
Concatenator concatenator = newConcatenator.asStatic().invoke("*");
Assert.assertEquals("Should function as a concatenator",
"a*b", concatenator.concat("a", "b"));
concatenator = newConcatenator.asStatic().invokeChecked("@");
Assert.assertEquals("Should function as a concatenator",
"a@b", concatenator.concat("a", "b"));
}
@Test
public void testConstructorImplAfterFactoryMethod() throws Exception {
DynMethods.UnboundMethod newConcatenator = new DynMethods.Builder("newConcatenator")
.impl(Concatenator.class, String.class)
.ctorImpl(Concatenator.class, String.class)
.buildChecked();
Assert.assertFalse("Should find factory method before constructor method",
newConcatenator instanceof DynConstructors.Ctor);
}
@Test
public void testNoop() throws Exception {
// noop can be unbound, bound, or static
DynMethods.UnboundMethod noop = new DynMethods.Builder("concat")
.impl("not.a.RealClass", String.class, String.class)
.orNoop()
.buildChecked();
Assert.assertTrue("No implementation found, should return NOOP",
noop.isNoop());
Assert.assertNull("NOOP should always return null",
noop.invoke(new Concatenator(), "a"));
Assert.assertNull("NOOP can be called with null",
noop.invoke(null, "a"));
Assert.assertNull("NOOP can be bound",
noop.bind(new Concatenator()).invoke("a"));
Assert.assertNull("NOOP can be bound to null",
noop.bind(null).invoke("a"));
Assert.assertNull("NOOP can be static",
noop.asStatic().invoke("a"));
}
}

View File

@ -18,8 +18,10 @@
package org.apache.hadoop.util.functional;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.function.Function;
import org.assertj.core.api.Assertions;
import org.junit.Test;
@ -28,6 +30,7 @@
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.util.functional.FunctionalIO.extractIOExceptions;
import static org.apache.hadoop.util.functional.FunctionalIO.toUncheckedFunction;
import static org.apache.hadoop.util.functional.FunctionalIO.toUncheckedIOExceptionSupplier;
import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions;
@ -94,4 +97,15 @@ public void testUncheckAndExtract() throws Throwable {
.isSameAs(raised);
}
@Test
public void testUncheckedFunction() throws Throwable {
// java function which should raise a FileNotFoundException
// wrapped into an unchecked exeption
final Function<String, Object> fn =
toUncheckedFunction((String a) -> {
throw new FileNotFoundException(a);
});
intercept(UncheckedIOException.class, "missing", () ->
fn.apply("missing"));
}
}

View File

@ -15,4 +15,6 @@ log4j.rootLogger=info,stdout
log4j.threshold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
log4j.logger.org.apache.hadoop.util.dynamic.BindingUtils=DEBUG

View File

@ -0,0 +1,49 @@
/*
* 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.contract.hdfs;
import java.io.IOException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.io.wrappedio.impl.TestWrappedIO;
/**
* Test WrappedIO access to HDFS, especially ByteBufferPositionedReadable.
*/
public class TestDFSWrappedIO extends TestWrappedIO {
@BeforeClass
public static void createCluster() throws IOException {
HDFSContract.createCluster();
}
@AfterClass
public static void teardownCluster() throws IOException {
HDFSContract.destroyCluster();
}
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new HDFSContract(conf);
}
}

View File

@ -28,6 +28,7 @@
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.CommonPathCapabilities;
import org.apache.hadoop.fs.aliyun.oss.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.aliyun.oss.statistics.impl.OutputStreamStatistics;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
@ -62,6 +63,7 @@
import static org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.longOption;
import static org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.objectRepresentsDirectory;
import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
/**
* Implementation of {@link FileSystem} for <a href="https://oss.aliyun.com">
@ -782,4 +784,19 @@ OSSDataBlocks.BlockFactory getBlockFactory() {
BlockOutputStreamStatistics getBlockOutputStreamStatistics() {
return blockOutputStreamStatistics;
}
@Override
public boolean hasPathCapability(final Path path, final String capability)
throws IOException {
final Path p = makeQualified(path);
String cap = validatePathCapabilityArgs(p, capability);
switch (cap) {
// block locations are generated locally
case CommonPathCapabilities.VIRTUAL_BLOCK_LOCATIONS:
return true;
default:
return super.hasPathCapability(p, cap);
}
}
}

View File

@ -5560,10 +5560,12 @@ public boolean hasPathCapability(final Path path, final String capability)
case DIRECTORY_LISTING_INCONSISTENT:
return s3ExpressStore;
// etags are avaialable in listings, but they
// etags are available in listings, but they
// are not consistent across renames.
// therefore, only availability is declared
case CommonPathCapabilities.ETAGS_AVAILABLE:
// block locations are generated locally
case CommonPathCapabilities.VIRTUAL_BLOCK_LOCATIONS:
return true;
/*

View File

@ -26,7 +26,14 @@
import org.apache.hadoop.classification.InterfaceStability;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_AVRO;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_COLUMNAR;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_CSV;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_HBASE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_JSON;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ORC;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_PARQUET;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_VECTOR;
@ -81,7 +88,8 @@ boolean isAdaptive() {
* Choose an access policy.
* @param name strategy name from a configuration option, etc.
* @param defaultPolicy default policy to fall back to.
* @return the chosen strategy
* @return the chosen strategy or null if there was no match and
* the value of {@code defaultPolicy} was "null".
*/
public static S3AInputPolicy getPolicy(
String name,
@ -93,11 +101,23 @@ public static S3AInputPolicy getPolicy(
case Constants.INPUT_FADV_NORMAL:
return Normal;
// all these options currently map to random IO.
// all these options currently map to random IO.
case FS_OPTION_OPENFILE_READ_POLICY_HBASE:
case FS_OPTION_OPENFILE_READ_POLICY_RANDOM:
case FS_OPTION_OPENFILE_READ_POLICY_VECTOR:
return Random;
// columnar formats currently map to random IO,
// though in future this may be enhanced.
case FS_OPTION_OPENFILE_READ_POLICY_COLUMNAR:
case FS_OPTION_OPENFILE_READ_POLICY_ORC:
case FS_OPTION_OPENFILE_READ_POLICY_PARQUET:
return Random;
// handle the sequential formats.
case FS_OPTION_OPENFILE_READ_POLICY_AVRO:
case FS_OPTION_OPENFILE_READ_POLICY_CSV:
case FS_OPTION_OPENFILE_READ_POLICY_JSON:
case FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL:
case FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE:
return Sequential;

View File

@ -56,6 +56,7 @@
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_PARQUET;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_VECTOR;
import static org.apache.hadoop.fs.contract.ContractTestUtils.range;
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
@ -233,7 +234,8 @@ public void testNormalReadVsVectoredReadStatsCollection() throws Exception {
fs.openFile(path(VECTORED_READ_FILE_NAME))
.withFileStatus(fileStatus)
.opt(FS_OPTION_OPENFILE_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_VECTOR)
FS_OPTION_OPENFILE_READ_POLICY_PARQUET
+ ", " + FS_OPTION_OPENFILE_READ_POLICY_VECTOR)
.build();
try (FSDataInputStream in = builder.get()) {
in.readVectored(fileRanges, getAllocate());

View File

@ -0,0 +1,35 @@
/*
* 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.contract.s3a;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.io.wrappedio.impl.TestWrappedIO;
/**
* Test S3A access through the wrapped operations class.
*/
public class ITestS3AWrappedIO extends TestWrappedIO {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new S3AContract(conf);
}
}

View File

@ -43,14 +43,26 @@
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_AVRO;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_COLUMNAR;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_CSV;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_HBASE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_JSON;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ORC;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_PARQUET;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_VECTOR;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ASYNC_DRAIN_THRESHOLD;
import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
import static org.apache.hadoop.fs.s3a.S3AInputPolicy.Normal;
import static org.apache.hadoop.fs.s3a.S3AInputPolicy.Random;
import static org.apache.hadoop.fs.s3a.S3AInputPolicy.Sequential;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
@ -69,7 +81,7 @@ public class TestOpenFileSupport extends HadoopTestBase {
private static final String USERNAME = "hadoop";
public static final S3AInputPolicy INPUT_POLICY = S3AInputPolicy.Sequential;
public static final S3AInputPolicy INPUT_POLICY = Sequential;
public static final String TESTFILE = "s3a://bucket/name";
@ -142,7 +154,7 @@ public void testSeekRandomIOPolicy() throws Throwable {
// is picked up
assertOpenFile(INPUT_FADVISE, option)
.extracting(f -> f.getInputPolicy())
.isEqualTo(S3AInputPolicy.Random);
.isEqualTo(Random);
// and as neither status nor length was set: no file status
assertOpenFile(INPUT_FADVISE, option)
.extracting(f -> f.getStatus())
@ -161,7 +173,7 @@ public void testSeekPolicyAdaptive() throws Throwable {
assertOpenFile(FS_OPTION_OPENFILE_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE)
.extracting(f -> f.getInputPolicy())
.isEqualTo(S3AInputPolicy.Normal);
.isEqualTo(Normal);
}
/**
@ -184,7 +196,7 @@ public void testSeekPolicyListS3AOption() throws Throwable {
// fall back to the second seek policy if the first is unknown
assertOpenFile(INPUT_FADVISE, "hbase, random")
.extracting(f -> f.getInputPolicy())
.isEqualTo(S3AInputPolicy.Random);
.isEqualTo(Random);
}
/**
@ -199,14 +211,14 @@ public void testSeekPolicyExtractionFromList() throws Throwable {
FS_OPTION_OPENFILE_READ_POLICY);
Assertions.assertThat(S3AInputPolicy.getFirstSupportedPolicy(options, null))
.describedAs("Policy from " + plist)
.isEqualTo(S3AInputPolicy.Random);
.isEqualTo(Random);
}
@Test
public void testAdaptiveSeekPolicyRecognized() throws Throwable {
Assertions.assertThat(S3AInputPolicy.getPolicy("adaptive", null))
.describedAs("adaptive")
.isEqualTo(S3AInputPolicy.Normal);
.isEqualTo(Normal);
}
@Test
@ -222,11 +234,20 @@ public void testUnknownSeekPolicyFallback() throws Throwable {
@Test
public void testInputPolicyMapping() throws Throwable {
Object[][] policyMapping = {
{"normal", S3AInputPolicy.Normal},
{FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE, S3AInputPolicy.Normal},
{FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, S3AInputPolicy.Normal},
{FS_OPTION_OPENFILE_READ_POLICY_RANDOM, S3AInputPolicy.Random},
{FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, S3AInputPolicy.Sequential},
{"normal", Normal},
{FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE, Normal},
{FS_OPTION_OPENFILE_READ_POLICY_AVRO, Sequential},
{FS_OPTION_OPENFILE_READ_POLICY_COLUMNAR, Random},
{FS_OPTION_OPENFILE_READ_POLICY_CSV, Sequential},
{FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, Normal},
{FS_OPTION_OPENFILE_READ_POLICY_HBASE, Random},
{FS_OPTION_OPENFILE_READ_POLICY_JSON, Sequential},
{FS_OPTION_OPENFILE_READ_POLICY_ORC, Random},
{FS_OPTION_OPENFILE_READ_POLICY_PARQUET, Random},
{FS_OPTION_OPENFILE_READ_POLICY_RANDOM, Random},
{FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, Sequential},
{FS_OPTION_OPENFILE_READ_POLICY_VECTOR, Random},
{FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE, Sequential},
};
for (Object[] mapping : policyMapping) {
String name = (String) mapping[0];

View File

@ -42,6 +42,7 @@
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.statistics.IOStatistics;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FOOTER_CACHE;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
@ -180,6 +181,7 @@ public void testStreamIsNotChecksummed() throws Throwable {
fs.openFile(testFile)
.must(FS_OPTION_OPENFILE_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
.must(FS_OPTION_OPENFILE_FOOTER_CACHE, false)
.mustLong(FS_OPTION_OPENFILE_LENGTH, fileLength)
.build()
.get(),

View File

@ -1692,7 +1692,8 @@ public boolean hasPathCapability(final Path path, final String capability)
switch (validatePathCapabilityArgs(p, capability)) {
case CommonPathCapabilities.FS_PERMISSIONS:
case CommonPathCapabilities.FS_APPEND:
case CommonPathCapabilities.ETAGS_AVAILABLE:
// block locations are generated locally
case CommonPathCapabilities.VIRTUAL_BLOCK_LOCATIONS:
return true;
case CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME:

View File

@ -0,0 +1,53 @@
/*
* 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 org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.io.wrappedio.impl.TestWrappedIO;
/**
* Test WrappedIO access to ABFS.
*/
public class ITestAbfsWrappedIO extends TestWrappedIO {
private final boolean isSecure;
private final ABFSContractTestBinding binding;
public ITestAbfsWrappedIO() 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 AbstractFSContract createContract(final Configuration conf) {
return new AbfsFileSystemContract(conf, isSecure);
}
}