HADOOP-17450. Add Public IOStatistics API. (#2577)

This is the API and implementation classes of HADOOP-16830,
which allows callers to query IO object instances
(filesystems, streams, remote iterators, ...) and other classes
for statistics on their I/O Usage: operation count and min/max/mean
durations.

New Packages

org.apache.hadoop.fs.statistics.
  Public API, including:
    IOStatisticsSource
    IOStatistics
    IOStatisticsSnapshot (seralizable to java objects and json)
    +helper classes for logging and integration
    BufferedIOStatisticsInputStream
       implements IOStatisticsSource and StreamCapabilities
     BufferedIOStatisticsOutputStream
       implements IOStatisticsSource, Syncable and StreamCapabilities

org.apache.hadoop.fs.statistics.impl
  Implementation classes for internal use.

org.apache.hadoop.util.functional
  functional programming support for RemoteIterators and
  other operations which raise IOEs; all wrapper classes
  implement and propagate IOStatisticsSource

Contributed by Steve Loughran.

Change-Id: If56e8db2981613ff689c39239135e44feb25f78e
This commit is contained in:
Steve Loughran 2020-12-31 11:52:42 +00:00
parent 89452682a5
commit 57abfae136
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
72 changed files with 9890 additions and 109 deletions

View File

@ -46,9 +46,13 @@ import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.StreamCapabilitiesPolicy;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.util.StringUtils;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* CryptoInputStream decrypts data. It is not thread-safe. AES CTR mode is
* required in order to ensure that the plain text and cipher text have a 1:1
@ -66,7 +70,7 @@ public class CryptoInputStream extends FilterInputStream implements
Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor,
CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess,
ReadableByteChannel, CanUnbuffer, StreamCapabilities,
ByteBufferPositionedReadable {
ByteBufferPositionedReadable, IOStatisticsSource {
private final byte[] oneByteBuf = new byte[1];
private final CryptoCodec codec;
private final Decryptor decryptor;
@ -867,8 +871,16 @@ public class CryptoInputStream extends FilterInputStream implements
+ " does not expose its stream capabilities.");
}
return ((StreamCapabilities) in).hasCapability(capability);
case StreamCapabilities.IOSTATISTICS:
return (in instanceof StreamCapabilities)
&& ((StreamCapabilities) in).hasCapability(capability);
default:
return false;
}
}
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(in);
}
}

View File

@ -28,9 +28,13 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* CryptoOutputStream encrypts data. It is not thread-safe. AES CTR mode is
* required in order to ensure that the plain text and cipher text have a 1:1
@ -48,7 +52,7 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class CryptoOutputStream extends FilterOutputStream implements
Syncable, CanSetDropBehind, StreamCapabilities {
Syncable, CanSetDropBehind, StreamCapabilities, IOStatisticsSource {
private final byte[] oneByteBuf = new byte[1];
private final CryptoCodec codec;
private final Encryptor encryptor;
@ -313,4 +317,9 @@ public class CryptoOutputStream extends FilterOutputStream implements
}
return false;
}
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(out);
}
}

View File

@ -24,6 +24,10 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
@ -33,7 +37,8 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class BufferedFSInputStream extends BufferedInputStream
implements Seekable, PositionedReadable, HasFileDescriptor {
implements Seekable, PositionedReadable, HasFileDescriptor,
IOStatisticsSource, StreamCapabilities {
/**
* Creates a <code>BufferedFSInputStream</code>
* with the specified buffer size,
@ -126,4 +131,26 @@ implements Seekable, PositionedReadable, HasFileDescriptor {
return null;
}
}
/**
* If the inner stream supports {@link StreamCapabilities},
* forward the probe to it.
* Otherwise: return false.
*
* @param capability string to query the stream support for.
* @return true if a capability is known to be supported.
*/
@Override
public boolean hasCapability(final String capability) {
if (in instanceof StreamCapabilities) {
return ((StreamCapabilities) in).hasCapability(capability);
} else {
return false;
}
}
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(in);
}
}

View File

@ -38,6 +38,9 @@ import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
import org.apache.hadoop.fs.impl.OpenFileParameters;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.LambdaUtils;
import org.apache.hadoop.util.Progressable;
@ -134,7 +137,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
* For open()'s FSInputStream
* It verifies that data matches checksums.
*******************************************************/
private static class ChecksumFSInputChecker extends FSInputChecker {
private static class ChecksumFSInputChecker extends FSInputChecker implements
IOStatisticsSource {
private ChecksumFileSystem fs;
private FSDataInputStream datas;
private FSDataInputStream sums;
@ -270,6 +274,17 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
}
return nread;
}
/**
* Get the IO Statistics of the nested stream, falling back to
* null if the stream does not implement the interface
* {@link IOStatisticsSource}.
* @return an IOStatistics instance or null
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(datas);
}
}
private static class FSDataBoundedInputStream extends FSDataInputStream {
@ -395,7 +410,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
/** This class provides an output stream for a checksummed file.
* It generates checksums for data. */
private static class ChecksumFSOutputSummer extends FSOutputSummer {
private static class ChecksumFSOutputSummer extends FSOutputSummer
implements IOStatisticsSource, StreamCapabilities {
private FSDataOutputStream datas;
private FSDataOutputStream sums;
private static final float CHKSUM_AS_FRACTION = 0.01f;
@ -449,6 +465,28 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
throw new ClosedChannelException();
}
}
/**
* Get the IO Statistics of the nested stream, falling back to
* null if the stream does not implement the interface
* {@link IOStatisticsSource}.
* @return an IOStatistics instance or null
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(datas);
}
/**
* Probe the inner stream for a capability.
*
* @param capability string to query the stream support for.
* @return true if a capability is known to be supported.
*/
@Override
public boolean hasCapability(final String capability) {
return datas.hasCapability(capability);
}
}
@Override

View File

@ -29,6 +29,9 @@ import java.util.EnumSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.util.IdentityHashStore;
@ -40,7 +43,7 @@ public class FSDataInputStream extends DataInputStream
implements Seekable, PositionedReadable,
ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead,
HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities,
ByteBufferPositionedReadable {
ByteBufferPositionedReadable, IOStatisticsSource {
/**
* Map ByteBuffers that we have handed out to readers to ByteBufferPool
* objects
@ -267,4 +270,15 @@ public class FSDataInputStream extends DataInputStream
"unsupported by " + in.getClass().getCanonicalName());
}
}
/**
* Get the IO Statistics of the nested stream, falling back to
* null if the stream does not implement the interface
* {@link IOStatisticsSource}.
* @return an IOStatistics instance or null
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(in);
}
}

View File

@ -24,13 +24,17 @@ import java.io.OutputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
/** Utility that wraps a {@link OutputStream} in a {@link DataOutputStream}.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FSDataOutputStream extends DataOutputStream
implements Syncable, CanSetDropBehind, StreamCapabilities {
implements Syncable, CanSetDropBehind, StreamCapabilities,
IOStatisticsSource {
private final OutputStream wrappedStream;
private static class PositionCache extends FilterOutputStream {
@ -155,4 +159,15 @@ public class FSDataOutputStream extends DataOutputStream
"not support setting the drop-behind caching setting.");
}
}
/**
* Get the IO Statistics of the nested stream, falling back to
* empty statistics if the stream does not implement the interface
* {@link IOStatisticsSource}.
* @return an IOStatistics instance.
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(wrappedStream);
}
}

View File

@ -24,6 +24,9 @@ import java.io.InputStream;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -134,4 +137,23 @@ public abstract class FSInputStream extends InputStream
throws IOException {
readFully(position, buffer, 0, buffer.length);
}
/**
* toString method returns the superclass toString, but if the subclass
* implements {@link IOStatisticsSource} then those statistics are
* extracted and included in the output.
* That is: statistics of subclasses are automatically reported.
* @return a string value.
*/
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(super.toString());
sb.append('{');
if (this instanceof IOStatisticsSource) {
sb.append(IOStatisticsLogging.ioStatisticsSourceToString(
(IOStatisticsSource) this));
}
sb.append('}');
return sb.toString();
}
}

View File

@ -26,14 +26,20 @@ import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
/**
* MultipartUploader is an interface for copying files multipart and across
* multiple nodes.
* <p></p>
* The interface extends {@link IOStatisticsSource} so that there is no
* need to cast an instance to see if is a source of statistics.
* However, implementations MAY return null for their actual statistics.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public interface MultipartUploader extends Closeable {
public interface MultipartUploader extends Closeable,
IOStatisticsSource {
/**

View File

@ -40,6 +40,7 @@ import java.nio.file.attribute.BasicFileAttributeView;
import java.nio.file.attribute.FileTime;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.Locale;
import java.util.Optional;
import java.util.StringTokenizer;
@ -47,6 +48,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.BufferedIOStatisticsOutputStream;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Progressable;
@ -54,6 +59,14 @@ import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
/****************************************************************
* Implement the FileSystem API for the raw local filesystem.
@ -107,10 +120,23 @@ public class RawLocalFileSystem extends FileSystem {
/*******************************************************
* For open()'s FSInputStream.
*******************************************************/
class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor {
class LocalFSFileInputStream extends FSInputStream implements
HasFileDescriptor, IOStatisticsSource, StreamCapabilities {
private FileInputStream fis;
private long position;
/**
* Minimal set of counters.
*/
private final IOStatisticsStore ioStatistics = iostatisticsStore()
.withCounters(
STREAM_READ_BYTES,
STREAM_READ_EXCEPTIONS,
STREAM_READ_SEEK_OPERATIONS,
STREAM_READ_SKIP_OPERATIONS,
STREAM_READ_SKIP_BYTES)
.build();
public LocalFSFileInputStream(Path f) throws IOException {
fis = new FileInputStream(pathToFile(f));
}
@ -152,9 +178,11 @@ public class RawLocalFileSystem extends FileSystem {
if (value >= 0) {
this.position++;
statistics.incrementBytesRead(1);
ioStatistics.incrementCounter(STREAM_READ_BYTES);
}
return value;
} catch (IOException e) { // unexpected exception
ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@ -168,9 +196,11 @@ public class RawLocalFileSystem extends FileSystem {
if (value > 0) {
this.position += value;
statistics.incrementBytesRead(value);
ioStatistics.incrementCounter(STREAM_READ_BYTES, value);
}
return value;
} catch (IOException e) { // unexpected exception
ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@ -189,18 +219,22 @@ public class RawLocalFileSystem extends FileSystem {
int value = fis.getChannel().read(bb, position);
if (value > 0) {
statistics.incrementBytesRead(value);
ioStatistics.incrementCounter(STREAM_READ_BYTES, value);
}
return value;
} catch (IOException e) {
ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS);
throw new FSError(e);
}
}
@Override
public long skip(long n) throws IOException {
ioStatistics.incrementCounter(STREAM_READ_SKIP_OPERATIONS);
long value = fis.skip(n);
if (value > 0) {
this.position += value;
ioStatistics.incrementCounter(STREAM_READ_SKIP_BYTES, value);
}
return value;
}
@ -209,6 +243,23 @@ public class RawLocalFileSystem extends FileSystem {
public FileDescriptor getFileDescriptor() throws IOException {
return fis.getFD();
}
@Override
public boolean hasCapability(String capability) {
// a bit inefficient, but intended to make it easier to add
// new capabilities.
switch (capability.toLowerCase(Locale.ENGLISH)) {
case StreamCapabilities.IOSTATISTICS:
return true;
default:
return false;
}
}
@Override
public IOStatistics getIOStatistics() {
return ioStatistics;
}
}
@Override
@ -233,9 +284,19 @@ public class RawLocalFileSystem extends FileSystem {
/*********************************************************
* For create()'s FSOutputStream.
*********************************************************/
class LocalFSFileOutputStream extends OutputStream {
final class LocalFSFileOutputStream extends OutputStream implements
IOStatisticsSource, StreamCapabilities {
private FileOutputStream fos;
/**
* Minimal set of counters.
*/
private final IOStatisticsStore ioStatistics = iostatisticsStore()
.withCounters(
STREAM_WRITE_BYTES,
STREAM_WRITE_EXCEPTIONS)
.build();
private LocalFSFileOutputStream(Path f, boolean append,
FsPermission permission) throws IOException {
File file = pathToFile(f);
@ -275,7 +336,9 @@ public class RawLocalFileSystem extends FileSystem {
public void write(byte[] b, int off, int len) throws IOException {
try {
fos.write(b, off, len);
ioStatistics.incrementCounter(STREAM_WRITE_BYTES, len);
} catch (IOException e) { // unexpected exception
ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@ -284,10 +347,29 @@ public class RawLocalFileSystem extends FileSystem {
public void write(int b) throws IOException {
try {
fos.write(b);
ioStatistics.incrementCounter(STREAM_WRITE_BYTES);
} catch (IOException e) { // unexpected exception
ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@Override
public boolean hasCapability(String capability) {
// a bit inefficient, but intended to make it easier to add
// new capabilities.
switch (capability.toLowerCase(Locale.ENGLISH)) {
case StreamCapabilities.IOSTATISTICS:
return true;
default:
return false;
}
}
@Override
public IOStatistics getIOStatistics() {
return ioStatistics;
}
}
@Override
@ -320,8 +402,8 @@ public class RawLocalFileSystem extends FileSystem {
if (parent != null && !mkdirs(parent)) {
throw new IOException("Mkdirs failed to create " + parent.toString());
}
return new FSDataOutputStream(new BufferedOutputStream(
createOutputStreamWithMode(f, false, permission), bufferSize),
return new FSDataOutputStream(new BufferedIOStatisticsOutputStream(
createOutputStreamWithMode(f, false, permission), bufferSize, true),
statistics);
}
@ -342,8 +424,8 @@ public class RawLocalFileSystem extends FileSystem {
if (exists(f) && !flags.contains(CreateFlag.OVERWRITE)) {
throw new FileAlreadyExistsException("File already exists: " + f);
}
return new FSDataOutputStream(new BufferedOutputStream(
createOutputStreamWithMode(f, false, permission), bufferSize),
return new FSDataOutputStream(new BufferedIOStatisticsOutputStream(
createOutputStreamWithMode(f, false, permission), bufferSize, true),
statistics);
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.fs;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import java.util.Iterator;
@ -27,15 +28,16 @@ import java.util.Iterator;
* instance.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public abstract class StorageStatistics {
/**
* These are common statistic names.
*
* <p>
* The following names are considered general and preserved across different
* StorageStatistics classes. When implementing a new StorageStatistics, it is
* highly recommended to use the common statistic names.
*
* <p>
* When adding new common statistic name constants, please make them unique.
* By convention, they are implicitly unique:
* <ul>
@ -43,39 +45,46 @@ public abstract class StorageStatistics {
* underscores.</li>
* <li>the value of the constants are lowercase of the constant names.</li>
* </ul>
* See {@link StoreStatisticNames} for the field names used here
* and elsewhere.
*/
@InterfaceStability.Evolving
public interface CommonStatisticNames {
// The following names are for file system operation invocations
String OP_APPEND = "op_append";
String OP_COPY_FROM_LOCAL_FILE = "op_copy_from_local_file";
String OP_CREATE = "op_create";
String OP_CREATE_NON_RECURSIVE = "op_create_non_recursive";
String OP_DELETE = "op_delete";
String OP_EXISTS = "op_exists";
String OP_GET_CONTENT_SUMMARY = "op_get_content_summary";
String OP_GET_DELEGATION_TOKEN = "op_get_delegation_token";
String OP_GET_FILE_CHECKSUM = "op_get_file_checksum";
String OP_GET_FILE_STATUS = "op_get_file_status";
String OP_GET_STATUS = "op_get_status";
String OP_GLOB_STATUS = "op_glob_status";
String OP_IS_FILE = "op_is_file";
String OP_IS_DIRECTORY = "op_is_directory";
String OP_LIST_FILES = "op_list_files";
String OP_LIST_LOCATED_STATUS = "op_list_located_status";
String OP_LIST_STATUS = "op_list_status";
String OP_MKDIRS = "op_mkdirs";
String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries";
String OP_OPEN = "op_open";
String OP_REMOVE_ACL = "op_remove_acl";
String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries";
String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl";
String OP_RENAME = "op_rename";
String OP_SET_ACL = "op_set_acl";
String OP_SET_OWNER = "op_set_owner";
String OP_SET_PERMISSION = "op_set_permission";
String OP_SET_TIMES = "op_set_times";
String OP_TRUNCATE = "op_truncate";
String OP_APPEND = StoreStatisticNames.OP_APPEND;
String OP_COPY_FROM_LOCAL_FILE =
StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE;
String OP_CREATE = StoreStatisticNames.OP_CREATE;
String OP_CREATE_NON_RECURSIVE =
StoreStatisticNames.OP_CREATE_NON_RECURSIVE;
String OP_DELETE = StoreStatisticNames.OP_DELETE;
String OP_EXISTS = StoreStatisticNames.OP_EXISTS;
String OP_GET_CONTENT_SUMMARY =
StoreStatisticNames.OP_GET_CONTENT_SUMMARY;
String OP_GET_DELEGATION_TOKEN =
StoreStatisticNames.OP_GET_DELEGATION_TOKEN;
String OP_GET_FILE_CHECKSUM = StoreStatisticNames.OP_GET_FILE_CHECKSUM;
String OP_GET_FILE_STATUS = StoreStatisticNames.OP_GET_FILE_STATUS;
String OP_GET_STATUS = StoreStatisticNames.OP_GET_STATUS;
String OP_GLOB_STATUS = StoreStatisticNames.OP_GLOB_STATUS;
String OP_IS_FILE = StoreStatisticNames.OP_IS_FILE;
String OP_IS_DIRECTORY = StoreStatisticNames.OP_IS_DIRECTORY;
String OP_LIST_FILES = StoreStatisticNames.OP_LIST_FILES;
String OP_LIST_LOCATED_STATUS =
StoreStatisticNames.OP_LIST_LOCATED_STATUS;
String OP_LIST_STATUS = StoreStatisticNames.OP_LIST_STATUS;
String OP_MKDIRS = StoreStatisticNames.OP_MKDIRS;
String OP_MODIFY_ACL_ENTRIES = StoreStatisticNames.OP_MODIFY_ACL_ENTRIES;
String OP_OPEN = StoreStatisticNames.OP_OPEN;
String OP_REMOVE_ACL = StoreStatisticNames.OP_REMOVE_ACL;
String OP_REMOVE_ACL_ENTRIES = StoreStatisticNames.OP_REMOVE_ACL_ENTRIES;
String OP_REMOVE_DEFAULT_ACL = StoreStatisticNames.OP_REMOVE_DEFAULT_ACL;
String OP_RENAME = StoreStatisticNames.OP_RENAME;
String OP_SET_ACL = StoreStatisticNames.OP_SET_ACL;
String OP_SET_OWNER = StoreStatisticNames.OP_SET_OWNER;
String OP_SET_PERMISSION = StoreStatisticNames.OP_SET_PERMISSION;
String OP_SET_TIMES = StoreStatisticNames.OP_SET_TIMES;
String OP_TRUNCATE = StoreStatisticNames.OP_TRUNCATE;
}
/**

View File

@ -71,6 +71,11 @@ public interface StreamCapabilities {
*/
String PREADBYTEBUFFER = "in:preadbytebuffer";
/**
* IOStatisticsSource API.
*/
String IOSTATISTICS = "iostatistics";
/**
* Capabilities that a stream can support and be queried for.
*/

View File

@ -24,7 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Evolving support for functional programming/lambda-expressions.
* Support for functional programming/lambda-expressions.
* @deprecated use {@code org.apache.hadoop.util.functional}
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@ -37,6 +38,7 @@ public final class FunctionsRaisingIOE {
* Function of arity 1 which may raise an IOException.
* @param <T> type of arg1
* @param <R> type of return value.
* @deprecated use {@link org.apache.hadoop.util.functional.FunctionRaisingIOE}
*/
@FunctionalInterface
public interface FunctionRaisingIOE<T, R> {
@ -49,6 +51,7 @@ public final class FunctionsRaisingIOE {
* @param <T> type of arg1
* @param <U> type of arg2
* @param <R> type of return value.
* @deprecated use {@link org.apache.hadoop.util.functional.BiFunctionRaisingIOE}
*/
@FunctionalInterface
public interface BiFunctionRaisingIOE<T, U, R> {
@ -59,6 +62,7 @@ public final class FunctionsRaisingIOE {
/**
* This is a callable which only raises an IOException.
* @param <R> return type
* @deprecated use {@link org.apache.hadoop.util.functional.CallableRaisingIOE}
*/
@FunctionalInterface
public interface CallableRaisingIOE<R> {

View File

@ -32,9 +32,16 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSBuilder;
import org.apache.hadoop.util.functional.CallableRaisingIOE;
import org.apache.hadoop.util.functional.FutureIO;
/**
* Support for future IO and the FS Builder subclasses.
* If methods in here are needed for applications, promote
* to {@link FutureIO} for public use -with the original
* method relaying to it. This is to ensure that external
* filesystem implementations can safely use these methods
* without linkage problems surfacing.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@ -55,14 +62,7 @@ public final class FutureIOSupport {
*/
public static <T> T awaitFuture(final Future<T> future)
throws InterruptedIOException, IOException, RuntimeException {
try {
return future.get();
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException(e.toString())
.initCause(e);
} catch (ExecutionException e) {
return raiseInnerCause(e);
}
return FutureIO.awaitFuture(future);
}
@ -82,18 +82,9 @@ public final class FutureIOSupport {
final TimeUnit unit)
throws InterruptedIOException, IOException, RuntimeException,
TimeoutException {
try {
return future.get(timeout, unit);
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException(e.toString())
.initCause(e);
} catch (ExecutionException e) {
return raiseInnerCause(e);
}
return FutureIO.awaitFuture(future, timeout, unit);
}
/**
* From the inner cause of an execution exception, extract the inner cause
* if it is an IOE or RTE.
@ -110,7 +101,7 @@ public final class FutureIOSupport {
*/
public static <T> T raiseInnerCause(final ExecutionException e)
throws IOException {
throw unwrapInnerException(e);
return FutureIO.raiseInnerCause(e);
}
/**
@ -125,41 +116,7 @@ public final class FutureIOSupport {
*/
public static <T> T raiseInnerCause(final CompletionException e)
throws IOException {
throw unwrapInnerException(e);
}
/**
* From the inner cause of an execution exception, extract the inner cause.
* If it is an RTE: throw immediately.
* If it is an IOE: Return.
* If it is a WrappedIOException: Unwrap and return
* Else: create a new IOException.
*
* Recursively handles wrapped Execution and Completion Exceptions in
* case something very complicated has happened.
* @param e exception.
* @return an IOException extracted or built from the cause.
* @throws RuntimeException if that is the inner cause.
*/
private static IOException unwrapInnerException(final Throwable e) {
Throwable cause = e.getCause();
if (cause instanceof IOException) {
return (IOException) cause;
} else if (cause instanceof WrappedIOException) {
return ((WrappedIOException) cause).getCause();
} else if (cause instanceof CompletionException) {
return unwrapInnerException(cause);
} else if (cause instanceof ExecutionException) {
return unwrapInnerException(cause);
} else if (cause instanceof RuntimeException) {
throw (RuntimeException) cause;
} else if (cause != null) {
// other type: wrap with a new IOE
return new IOException(cause);
} else {
// this only happens if there was no cause.
return new IOException(e);
}
return FutureIO.raiseInnerCause(e);
}
/**
@ -236,7 +193,7 @@ public final class FutureIOSupport {
* @throws IllegalArgumentException invalid argument
*/
public static <T> CompletableFuture<T> eval(
FunctionsRaisingIOE.CallableRaisingIOE<T> callable) {
CallableRaisingIOE<T> callable) {
CompletableFuture<T> result = new CompletableFuture<>();
try {
result.complete(callable.apply());

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.fs.impl;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
@ -33,10 +34,12 @@ import org.apache.hadoop.classification.InterfaceStability;
*
* The constructor signature guarantees the cause will be an IOException,
* and as it checks for a null-argument, non-null.
* @deprecated use the {@code UncheckedIOException}.
*/
@Deprecated
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class WrappedIOException extends RuntimeException {
public class WrappedIOException extends UncheckedIOException {
private static final long serialVersionUID = 2510210974235779294L;

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.fs.statistics;
import java.io.BufferedInputStream;
import java.io.InputStream;
import org.apache.hadoop.fs.StreamCapabilities;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* An extension of {@code BufferedInputStream} which implements
* {@link IOStatisticsSource} and forwards requests for the
* {@link IOStatistics} to the wrapped stream.
*
* This should be used when any input stream needs buffering while
* allowing the inner stream to be a source of statistics.
*
* It also implements {@link StreamCapabilities} and forwards the probe
* to the inner stream, if possible.
*/
public class BufferedIOStatisticsInputStream
extends BufferedInputStream
implements IOStatisticsSource, StreamCapabilities {
/**
* Buffer an input stream with the default buffer size of 8k.
* @param in input stream
*/
public BufferedIOStatisticsInputStream(final InputStream in) {
super(in);
}
/**
* Buffer an input stream with the chosen buffer size.
* @param in input stream
* @param size buffer size
*/
public BufferedIOStatisticsInputStream(final InputStream in, final int size) {
super(in, size);
}
/**
* Return any IOStatistics offered by the inner stream.
* @return inner IOStatistics or null
*/
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(in);
}
/**
* If the inner stream supports {@link StreamCapabilities},
* forward the probe to it.
* Otherwise: return false.
*
* @param capability string to query the stream support for.
* @return true if a capability is known to be supported.
*/
@Override
public boolean hasCapability(final String capability) {
if (in instanceof StreamCapabilities) {
return ((StreamCapabilities) in).hasCapability(capability);
} else {
return false;
}
}
}

View File

@ -0,0 +1,157 @@
/*
* 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.statistics;
import java.io.BufferedOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* An extension of {@code BufferedOutputStream} which implements
* {@link IOStatisticsSource} and forwards requests for the
* {@link IOStatistics} to the wrapped stream.
*
* This should be used when any output stream needs buffering while
* allowing the inner stream to be a source of statistics.
*
* It also implements {@link StreamCapabilities}
* and {@link Syncable} and forwards to to the inner stream,
* if possible.
*/
public class BufferedIOStatisticsOutputStream
extends BufferedOutputStream
implements IOStatisticsSource, Syncable, StreamCapabilities {
/**
* Should calls to Syncable downgrade to flush if the underlying
* stream does not support it?
* While that breaks a core contract requirement of Syncable:
* "Sync.sync() guarantees durability", downgrading is
* the default behavior of FsDataOutputStream.
*/
private final boolean downgradeSyncable;
/**
* Construct with default buffer size.
* @param out output stream to buffer
* @param downgradeSyncable should Syncable calls downgrade?
*/
public BufferedIOStatisticsOutputStream(
final OutputStream out,
final boolean downgradeSyncable) {
super(out);
this.downgradeSyncable = downgradeSyncable;
}
/**
* Construct with custom buffer size.
*
* @param out output stream to buffer
* @param size buffer.
* @param downgradeSyncable should Syncable calls downgrade?
*/
public BufferedIOStatisticsOutputStream(
final OutputStream out,
final int size,
final boolean downgradeSyncable) {
super(out, size);
this.downgradeSyncable = downgradeSyncable;
}
/**
* Ask the inner stream for their IOStatistics.
* @return any IOStatistics offered by the inner stream.
*/
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(out);
}
/**
* If the inner stream supports {@link StreamCapabilities},
* forward the probe to it.
* Otherwise: return false.
*
* @param capability string to query the stream support for.
* @return true if a capability is known to be supported.
*/
@Override
public boolean hasCapability(final String capability) {
if (out instanceof StreamCapabilities) {
return ((StreamCapabilities) out).hasCapability(capability);
} else {
return false;
}
}
/**
* If the inner stream is Syncable, flush the buffer and then
* invoke the inner stream's hflush() operation.
*
* Otherwise: throw an exception, unless the stream was constructed with
* {@link #downgradeSyncable} set to true, in which case the stream
* is just flushed.
* @throws IOException IO Problem
* @throws UnsupportedOperationException if the inner class is not syncable
*/
@Override
public void hflush() throws IOException {
if (out instanceof Syncable) {
flush();
((Syncable) out).hflush();
} else {
if (!downgradeSyncable) {
throw new UnsupportedOperationException("hflush not supported by "
+ out);
} else {
flush();
}
}
}
/**
* If the inner stream is Syncable, flush the buffer and then
* invoke the inner stream's hsync() operation.
*
* Otherwise: throw an exception, unless the stream was constructed with
* {@link #downgradeSyncable} set to true, in which case the stream
* is just flushed.
* @throws IOException IO Problem
* @throws UnsupportedOperationException if the inner class is not syncable
*/
@Override
public void hsync() throws IOException {
if (out instanceof Syncable) {
flush();
((Syncable) out).hsync();
} else {
if (!downgradeSyncable) {
throw new UnsupportedOperationException("hsync not supported by "
+ out);
} else {
flush();
}
}
}
}

View File

@ -0,0 +1,154 @@
/*
* 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.statistics;
import javax.annotation.Nullable;
import java.io.Serializable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
/**
* Summary of duration tracking statistics
* as extracted from an IOStatistics instance.
* <p>
* This is for reporting and testing.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class DurationStatisticSummary implements Serializable {
private static final long serialVersionUID = 6776381340896518486L;
/** Statistic key. */
private final String key;
/** Are these success or failure statistics. */
private final boolean success;
/** Count of operation invocations. */
private final long count;
/** Max duration; -1 if unknown. */
private final long max;
/** Min duration; -1 if unknown. */
private final long min;
/** Mean duration -may be null. */
private final MeanStatistic mean;
/**
* Constructor.
* @param key Statistic key.
* @param success Are these success or failure statistics.
* @param count Count of operation invocations.
* @param max Max duration; -1 if unknown.
* @param min Min duration; -1 if unknown.
* @param mean Mean duration -may be null. (will be cloned)
*/
public DurationStatisticSummary(final String key,
final boolean success,
final long count,
final long max,
final long min,
@Nullable final MeanStatistic mean) {
this.key = key;
this.success = success;
this.count = count;
this.max = max;
this.min = min;
this.mean = mean == null ? null : mean.clone();
}
public String getKey() {
return key;
}
public boolean isSuccess() {
return success;
}
public long getCount() {
return count;
}
public long getMax() {
return max;
}
public long getMin() {
return min;
}
public MeanStatistic getMean() {
return mean;
}
@Override
public String toString() {
return "DurationStatisticSummary{" +
"key='" + key + '\'' +
", success=" + success +
", counter=" + count +
", max=" + max +
", mean=" + mean +
'}';
}
/**
* Fetch the duration timing summary of success or failure operations
* from an IO Statistics source.
* If the duration key is unknown, the summary will be incomplete.
* @param source source of data
* @param key duration statistic key
* @param success fetch success statistics, or if false, failure stats.
* @return a summary of the statistics.
*/
public static DurationStatisticSummary fetchDurationSummary(
IOStatistics source,
String key,
boolean success) {
String fullkey = success ? key : key + SUFFIX_FAILURES;
return new DurationStatisticSummary(key, success,
source.counters().getOrDefault(fullkey, 0L),
source.maximums().getOrDefault(fullkey + SUFFIX_MAX, -1L),
source.minimums().getOrDefault(fullkey + SUFFIX_MIN, -1L),
source.meanStatistics()
.get(fullkey + SUFFIX_MEAN));
}
/**
* Fetch the duration timing summary from an IOStatistics source.
* If the duration key is unknown, the summary will be incomplete.
* @param source source of data
* @param key duration statistic key
* @return a summary of the statistics.
*/
public static DurationStatisticSummary fetchSuccessSummary(
IOStatistics source,
String key) {
return fetchDurationSummary(source, key, true);
}
}

View File

@ -0,0 +1,54 @@
/*
* 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.statistics;
import java.time.Duration;
/**
* Interface to be implemented by objects which can track duration.
* It extends AutoCloseable to fit into a try-with-resources statement,
* but then strips out the {@code throws Exception} aspect of the signature
* so it doesn't force code to add extra handling for any failures.
*
* If a duration is declared as "failed()" then the failure counters
* will be updated.
*/
public interface DurationTracker extends AutoCloseable {
/**
* The operation failed. Failure statistics will be updated.
*/
void failed();
/**
* Finish tracking: update the statistics with the timings.
*/
void close();
/**
* Get the duration of an operation as a java Duration
* instance. If the duration tracker hasn't completed,
* or its duration tracking doesn't actually measure duration,
* returns Duration.ZERO.
* @return a duration, value of ZERO until close().
*/
default Duration asDuration() {
return Duration.ZERO;
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.statistics;
/**
* Interface for a source of duration tracking.
*
* This is intended for uses where it can be passed into classes
* which update operation durations, without tying those
* classes to internal implementation details.
*/
public interface DurationTrackerFactory {
/**
* Initiate a duration tracking operation by creating/returning
* an object whose {@code close()} call will
* update the statistics.
*
* The statistics counter with the key name will be incremented
* by the given count.
*
* The expected use is within a try-with-resources clause.
* @param key statistic key prefix
* @param count #of times to increment the matching counter in this
* operation.
* @return an object to close after an operation completes.
*/
DurationTracker trackDuration(String key, long count);
/**
* Initiate a duration tracking operation by creating/returning
* an object whose {@code close()} call will
* update the statistics.
* The expected use is within a try-with-resources clause.
* @param key statistic key
* @return an object to close after an operation completes.
*/
default DurationTracker trackDuration(String key) {
return trackDuration(key, 1);
}
}

View File

@ -0,0 +1,78 @@
/*
* 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.statistics;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* IO Statistics.
* <p>
* These are low-cost per-instance statistics provided by any Hadoop
* I/O class instance.
* <p>
* Consult the filesystem specification document for the requirements
* of an implementation of this interface.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public interface IOStatistics {
/**
* Map of counters.
* @return the current map of counters.
*/
Map<String, Long> counters();
/**
* Map of gauges.
* @return the current map of gauges.
*/
Map<String, Long> gauges();
/**
* Map of minimums.
* @return the current map of minimums.
*/
Map<String, Long> minimums();
/**
* Map of maximums.
* @return the current map of maximums.
*/
Map<String, Long> maximums();
/**
* Map of meanStatistics.
* @return the current map of MeanStatistic statistics.
*/
Map<String, MeanStatistic> meanStatistics();
/**
* Value when a minimum value has never been set.
*/
long MIN_UNSET_VALUE = -1;
/**
* Value when a max value has never been set.
*/
long MAX_UNSET_VALUE = -1;
}

View File

@ -0,0 +1,48 @@
/*
* 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.statistics;
import javax.annotation.Nullable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Interface exported by classes which support
* aggregation of {@link IOStatistics}.
* Implementations MAY aggregate all statistics
* exported by the IOStatistics reference passed in to
* {@link #aggregate(IOStatistics)}, or they
* may selectively aggregate specific values/classes
* of statistics.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface IOStatisticsAggregator {
/**
* Aggregate the supplied statistics into the current
* set.
*
* @param statistics statistics; may be null
* @return true if the statistics reference was not null and
* so aggregated.
*/
boolean aggregate(@Nullable IOStatistics statistics);
}

View File

@ -0,0 +1,301 @@
/*
* 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.statistics;
import javax.annotation.Nullable;
import java.util.Map;
import java.util.TreeMap;
import java.util.function.Predicate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* Utility operations convert IO Statistics sources/instances
* to strings, especially for robustly logging.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class IOStatisticsLogging {
private static final Logger LOG =
LoggerFactory.getLogger(IOStatisticsLogging.class);
private IOStatisticsLogging() {
}
/**
* Extract the statistics from a source object -or ""
* if it is not an instance of {@link IOStatistics},
* {@link IOStatisticsSource} or the retrieved
* statistics are null.
* <p>
* Exceptions are caught and downgraded to debug logging.
* @param source source of statistics.
* @return a string for logging.
*/
public static String ioStatisticsSourceToString(@Nullable Object source) {
try {
return ioStatisticsToString(retrieveIOStatistics(source));
} catch (RuntimeException e) {
LOG.debug("Ignoring", e);
return "";
}
}
/**
* Convert IOStatistics to a string form.
* @param statistics A statistics instance.
* @return string value or the empty string if null
*/
public static String ioStatisticsToString(
@Nullable final IOStatistics statistics) {
if (statistics != null) {
StringBuilder sb = new StringBuilder();
mapToString(sb, "counters", statistics.counters(), " ");
mapToString(sb, "gauges", statistics.gauges(), " ");
mapToString(sb, "minimums", statistics.minimums(), " ");
mapToString(sb, "maximums", statistics.maximums(), " ");
mapToString(sb, "means", statistics.meanStatistics(), " ");
return sb.toString();
} else {
return "";
}
}
/**
* Convert IOStatistics to a string form, with all the metrics sorted
* and empty value stripped.
* This is more expensive than the simple conversion, so should only
* be used for logging/output where it's known/highly likely that the
* caller wants to see the values. Not for debug logging.
* @param statistics A statistics instance.
* @return string value or the empty string if null
*/
public static String ioStatisticsToPrettyString(
@Nullable final IOStatistics statistics) {
if (statistics != null) {
StringBuilder sb = new StringBuilder();
mapToSortedString(sb, "counters", statistics.counters(),
p -> p == 0);
mapToSortedString(sb, "\ngauges", statistics.gauges(),
p -> p == 0);
mapToSortedString(sb, "\nminimums", statistics.minimums(),
p -> p < 0);
mapToSortedString(sb, "\nmaximums", statistics.maximums(),
p -> p < 0);
mapToSortedString(sb, "\nmeans", statistics.meanStatistics(),
MeanStatistic::isEmpty);
return sb.toString();
} else {
return "";
}
}
/**
* Given a map, add its entryset to the string.
* The entries are only sorted if the source entryset
* iterator is sorted, such as from a TreeMap.
* @param sb string buffer to append to
* @param type type (for output)
* @param map map to evaluate
* @param separator separator
* @param <E> type of values of the map
*/
private static <E> void mapToString(StringBuilder sb,
final String type,
final Map<String, E> map,
final String separator) {
int count = 0;
sb.append(type);
sb.append("=(");
for (Map.Entry<String, E> entry : map.entrySet()) {
if (count > 0) {
sb.append(separator);
}
count++;
sb.append(IOStatisticsBinding.entryToString(
entry.getKey(), entry.getValue()));
}
sb.append(");\n");
}
/**
* Given a map, produce a string with all the values, sorted.
* Needs to create a treemap and insert all the entries.
* @param sb string buffer to append to
* @param type type (for output)
* @param map map to evaluate
* @param <E> type of values of the map
*/
private static <E> void mapToSortedString(StringBuilder sb,
final String type,
final Map<String, E> map,
final Predicate<E> isEmpty) {
mapToString(sb, type, sortedMap(map, isEmpty), "\n");
}
/**
* Create a sorted (tree) map from an unsorted map.
* This incurs the cost of creating a map and that
* of inserting every object into the tree.
* @param source source map
* @param <E> value type
* @return a treemap with all the entries.
*/
private static <E> Map<String, E> sortedMap(
final Map<String, E> source,
final Predicate<E> isEmpty) {
Map<String, E> tm = new TreeMap<>();
for (Map.Entry<String, E> entry : source.entrySet()) {
if (!isEmpty.test(entry.getValue())) {
tm.put(entry.getKey(), entry.getValue());
}
}
return tm;
}
/**
* On demand stringifier of an IOStatisticsSource instance.
* <p>
* Whenever this object's toString() method is called, it evaluates the
* statistics.
* <p>
* This is designed to affordable to use in log statements.
* @param source source of statistics -may be null.
* @return an object whose toString() operation returns the current values.
*/
public static Object demandStringifyIOStatisticsSource(
@Nullable IOStatisticsSource source) {
return new SourceToString(source);
}
/**
* On demand stringifier of an IOStatistics instance.
* <p>
* Whenever this object's toString() method is called, it evaluates the
* statistics.
* <p>
* This is for use in log statements where for the cost of creation
* of this entry is low; it is affordable to use in log statements.
* @param statistics statistics to stringify -may be null.
* @return an object whose toString() operation returns the current values.
*/
public static Object demandStringifyIOStatistics(
@Nullable IOStatistics statistics) {
return new StatisticsToString(statistics);
}
/**
* Extract any statistics from the source and log at debug, if
* the log is set to log at debug.
* No-op if logging is not at debug or the source is null/of
* the wrong type/doesn't provide statistics.
* @param log log to log to
* @param message message for log -this must contain "{}" for the
* statistics report to actually get logged.
* @param source source object
*/
public static void logIOStatisticsAtDebug(
Logger log,
String message,
Object source) {
if (log.isDebugEnabled()) {
// robust extract and convert to string
String stats = ioStatisticsSourceToString(source);
if (!stats.isEmpty()) {
log.debug(message, stats);
}
}
}
/**
* Extract any statistics from the source and log to
* this class's log at debug, if
* the log is set to log at debug.
* No-op if logging is not at debug or the source is null/of
* the wrong type/doesn't provide statistics.
* @param message message for log -this must contain "{}" for the
* statistics report to actually get logged.
* @param source source object
*/
public static void logIOStatisticsAtDebug(
String message,
Object source) {
logIOStatisticsAtDebug(LOG, message, source);
}
/**
* On demand stringifier.
* <p>
* Whenever this object's toString() method is called, it
* retrieves the latest statistics instance and re-evaluates it.
*/
private static final class SourceToString {
private final IOStatisticsSource source;
private SourceToString(@Nullable IOStatisticsSource source) {
this.source = source;
}
@Override
public String toString() {
return source != null
? ioStatisticsSourceToString(source)
: IOStatisticsBinding.NULL_SOURCE;
}
}
/**
* Stringifier of statistics: low cost to instantiate and every
* toString/logging will re-evaluate the statistics.
*/
private static final class StatisticsToString {
private final IOStatistics statistics;
/**
* Constructor.
* @param statistics statistics
*/
private StatisticsToString(@Nullable IOStatistics statistics) {
this.statistics = statistics;
}
/**
* Evaluate and stringify the statistics.
* @return a string value.
*/
@Override
public String toString() {
return statistics != null
? ioStatisticsToString(statistics)
: IOStatisticsBinding.NULL_SOURCE;
}
}
}

View File

@ -0,0 +1,285 @@
/*
* 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.statistics;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
import org.apache.hadoop.util.JsonSerialization;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaps;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.snapshotMap;
/**
* Snapshot of statistics from a different source.
* <p>
* It is serializable so that frameworks which can use java serialization
* to propagate data (Spark, Flink...) can send the statistics
* back. For this reason, TreeMaps are explicitly used as field types,
* even though IDEs can recommend use of Map instead.
* For security reasons, untrusted java object streams should never be
* deserialized. If for some reason this is required, use
* {@link #requiredSerializationClasses()} to get the list of classes
* used when deserializing instances of this object.
* <p>
* <p>
* It is annotated for correct serializations with jackson2.
* </p>
*/
@SuppressWarnings("CollectionDeclaredAsConcreteClass")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class IOStatisticsSnapshot
implements IOStatistics, Serializable, IOStatisticsAggregator {
private static final long serialVersionUID = -1762522703841538084L;
/**
* List of chasses needed to deserialize.
*/
private static final Class[] DESERIALIZATION_CLASSES = {
IOStatisticsSnapshot.class,
TreeMap.class,
Long.class,
MeanStatistic.class,
};
/**
* Counters.
*/
@JsonProperty
private transient Map<String, Long> counters;
/**
* Gauges.
*/
@JsonProperty
private transient Map<String, Long> gauges;
/**
* Minimum values.
*/
@JsonProperty
private transient Map<String, Long> minimums;
/**
* Maximum values.
*/
@JsonProperty
private transient Map<String, Long> maximums;
/**
* mean statistics. The JSON key is all lower case..
*/
@JsonProperty("meanstatistics")
private transient Map<String, MeanStatistic> meanStatistics;
/**
* Construct.
*/
public IOStatisticsSnapshot() {
createMaps();
}
/**
* Construct, taking a snapshot of the source statistics data
* if the source is non-null.
* If the source is null, the empty maps are created
* @param source statistics source. Nullable.
*/
public IOStatisticsSnapshot(IOStatistics source) {
if (source != null) {
snapshot(source);
} else {
createMaps();
}
}
/**
* Create the maps.
*/
private synchronized void createMaps() {
counters = new ConcurrentHashMap<>();
gauges = new ConcurrentHashMap<>();
minimums = new ConcurrentHashMap<>();
maximums = new ConcurrentHashMap<>();
meanStatistics = new ConcurrentHashMap<>();
}
/**
* Clear all the maps.
*/
public synchronized void clear() {
counters.clear();
gauges.clear();
minimums.clear();
maximums.clear();
meanStatistics.clear();
}
/**
* Take a snapshot.
*
* This completely overwrites the map data with the statistics
* from the source.
* @param source statistics source.
*/
public synchronized void snapshot(IOStatistics source) {
checkNotNull(source);
counters = snapshotMap(source.counters());
gauges = snapshotMap(source.gauges());
minimums = snapshotMap(source.minimums());
maximums = snapshotMap(source.maximums());
meanStatistics = snapshotMap(source.meanStatistics(),
MeanStatistic::copy);
}
/**
* Aggregate the current statistics with the
* source reference passed in.
*
* The operation is synchronized.
* @param source source; may be null
* @return true if a merge took place.
*/
@Override
public synchronized boolean aggregate(
@Nullable IOStatistics source) {
if (source == null) {
return false;
}
aggregateMaps(counters, source.counters(),
IOStatisticsBinding::aggregateCounters,
IOStatisticsBinding::passthroughFn);
aggregateMaps(gauges, source.gauges(),
IOStatisticsBinding::aggregateGauges,
IOStatisticsBinding::passthroughFn);
aggregateMaps(minimums, source.minimums(),
IOStatisticsBinding::aggregateMinimums,
IOStatisticsBinding::passthroughFn);
aggregateMaps(maximums, source.maximums(),
IOStatisticsBinding::aggregateMaximums,
IOStatisticsBinding::passthroughFn);
aggregateMaps(meanStatistics, source.meanStatistics(),
IOStatisticsBinding::aggregateMeanStatistics, MeanStatistic::copy);
return true;
}
@Override
public synchronized Map<String, Long> counters() {
return counters;
}
@Override
public synchronized Map<String, Long> gauges() {
return gauges;
}
@Override
public synchronized Map<String, Long> minimums() {
return minimums;
}
@Override
public synchronized Map<String, Long> maximums() {
return maximums;
}
@Override
public synchronized Map<String, MeanStatistic> meanStatistics() {
return meanStatistics;
}
@Override
public String toString() {
return ioStatisticsToString(this);
}
/**
* Get a JSON serializer for this class.
* @return a serializer.
*/
public static JsonSerialization<IOStatisticsSnapshot> serializer() {
return new JsonSerialization<>(IOStatisticsSnapshot.class, false, true);
}
/**
* Serialize by converting each map to a TreeMap, and saving that
* to the stream.
*/
private synchronized void writeObject(ObjectOutputStream s)
throws IOException {
// Write out the core
s.defaultWriteObject();
s.writeObject(new TreeMap<String, Long>(counters));
s.writeObject(new TreeMap<String, Long>(gauges));
s.writeObject(new TreeMap<String, Long>(minimums));
s.writeObject(new TreeMap<String, Long>(maximums));
s.writeObject(new TreeMap<String, MeanStatistic>(meanStatistics));
}
/**
* Deserialize by loading each TreeMap, and building concurrent
* hash maps from them.
*/
private void readObject(final ObjectInputStream s)
throws IOException, ClassNotFoundException {
// read in core
s.defaultReadObject();
// and rebuild a concurrent hashmap from every serialized tree map
// read back from the stream.
counters = new ConcurrentHashMap<>(
(TreeMap<String, Long>) s.readObject());
gauges = new ConcurrentHashMap<>(
(TreeMap<String, Long>) s.readObject());
minimums = new ConcurrentHashMap<>(
(TreeMap<String, Long>) s.readObject());
maximums = new ConcurrentHashMap<>(
(TreeMap<String, Long>) s.readObject());
meanStatistics = new ConcurrentHashMap<>(
(TreeMap<String, MeanStatistic>) s.readObject());
}
/**
* What classes are needed to deserialize this class?
* Needed to securely unmarshall this from untrusted sources.
* @return a list of required classes to deserialize the data.
*/
public static List<Class> requiredSerializationClasses() {
return Arrays.stream(DESERIALIZATION_CLASSES)
.collect(Collectors.toList());
}
}

View File

@ -0,0 +1,47 @@
/*
* 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.statistics;
import org.apache.hadoop.classification.InterfaceStability;
/**
* A source of IO statistics.
* <p>
* These statistics MUST be instance specific, not thread local.
* </p>
*/
@InterfaceStability.Unstable
public interface IOStatisticsSource {
/**
* Return a statistics instance.
* <p>
* It is not a requirement that the same instance is returned every time.
* {@link IOStatisticsSource}.
* <p>
* If the object implementing this is Closeable, this method
* may return null if invoked on a closed object, even if
* it returns a valid instance when called earlier.
* @return an IOStatistics instance or null
*/
default IOStatistics getIOStatistics() {
return null;
}
}

View File

@ -0,0 +1,107 @@
/*
* 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.statistics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.impl.StubDurationTracker;
import org.apache.hadoop.fs.statistics.impl.StubDurationTrackerFactory;
/**
* Support for working with IOStatistics.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class IOStatisticsSupport {
private IOStatisticsSupport() {
}
/**
* Take a snapshot of the current statistics state.
* <p>
* This is not an atomic option.
* <p>
* The instance can be serialized, and its
* {@code toString()} method lists all the values.
* @param statistics statistics
* @return a snapshot of the current values.
*/
public static IOStatisticsSnapshot
snapshotIOStatistics(IOStatistics statistics) {
return new IOStatisticsSnapshot(statistics);
}
/**
* Create a snapshot statistics instance ready to aggregate data.
*
* The instance can be serialized, and its
* {@code toString()} method lists all the values.
* @return an empty snapshot
*/
public static IOStatisticsSnapshot
snapshotIOStatistics() {
return new IOStatisticsSnapshot();
}
/**
* Get the IOStatistics of the source, casting it
* if it is of the relevant type, otherwise,
* if it implements {@link IOStatisticsSource}
* extracting the value.
*
* Returns null if the source isn't of the write type
* or the return value of
* {@link IOStatisticsSource#getIOStatistics()} was null.
* @return an IOStatistics instance or null
*/
public static IOStatistics retrieveIOStatistics(
final Object source) {
if (source instanceof IOStatistics) {
return (IOStatistics) source;
} else if (source instanceof IOStatisticsSource) {
return ((IOStatisticsSource) source).getIOStatistics();
} else {
// null source or interface not implemented
return null;
}
}
/**
* Return a stub duration tracker factory whose returned trackers
* are always no-ops.
*
* As singletons are returned, this is very low-cost to use.
* @return a duration tracker factory.
*/
public static DurationTrackerFactory stubDurationTrackerFactory() {
return StubDurationTrackerFactory.STUB_DURATION_TRACKER_FACTORY;
}
/**
* Get a stub duration tracker.
* @return a stub tracker.
*/
public static DurationTracker stubDurationTracker() {
return StubDurationTracker.STUB_DURATION_TRACKER;
}
}

View File

@ -0,0 +1,290 @@
/*
* 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.statistics;
import java.io.Serializable;
import java.util.Objects;
import com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* A mean statistic represented as the sum and the sample count;
* the mean is calculated on demand.
* <p>
* It can be used to accrue values so as to dynamically update
* the mean. If so, know that there is no synchronization
* on the methods.
* </p>
* <p>
* If a statistic has 0 samples then it is considered to be empty.
* </p>
* <p>
* All 'empty' statistics are equivalent, independent of the sum value.
* </p>
* <p>
* For non-empty statistics, sum and sample values must match
* for equality.
* </p>
* <p>
* It is serializable and annotated for correct serializations with jackson2.
* </p>
* <p>
* Thread safety. The operations to add/copy sample data, are thread safe.
* </p>
* <ol>
* <li>{@link #add(MeanStatistic)}</li>
* <li>{@link #addSample(long)} </li>
* <li>{@link #clear()} </li>
* <li>{@link #setSamplesAndSum(long, long)}</li>
* <li>{@link #set(MeanStatistic)}</li>
* <li>{@link #setSamples(long)} and {@link #setSum(long)}</li>
* </ol>
* <p>
* So is the {@link #mean()} method. This ensures that when
* used to aggregated statistics, the aggregate value and sample
* count are set and evaluated consistently.
* </p>
* <p>
* Other methods marked as synchronized because Findbugs overreacts
* to the idea that some operations to update sum and sample count
* are synchronized, but that things like equals are not.
* </p>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class MeanStatistic implements Serializable, Cloneable {
private static final long serialVersionUID = 567888327998615425L;
/**
* Number of samples used to calculate
* the mean.
*/
private long samples;
/**
* sum of the values.
*/
private long sum;
/**
* Constructor, with some resilience against invalid sample counts.
* If the sample count is 0 or less, the sum is set to 0 and
* the sample count to 0.
* @param samples sample count.
* @param sum sum value
*/
public MeanStatistic(final long samples, final long sum) {
if (samples > 0) {
this.sum = sum;
this.samples = samples;
}
}
/**
* Create from another statistic.
* @param that source
*/
public MeanStatistic(MeanStatistic that) {
synchronized (that) {
set(that);
}
}
/**
* Create an empty statistic.
*/
public MeanStatistic() {
}
/**
* Get the sum of samples.
* @return the sum
*/
public synchronized long getSum() {
return sum;
}
/**
* Get the sample count.
* @return the sample count; 0 means empty
*/
public synchronized long getSamples() {
return samples;
}
/**
* Is a statistic empty?
* @return true if the sample count is 0
*/
@JsonIgnore
public synchronized boolean isEmpty() {
return samples == 0;
}
/**
* Set the values to 0.
*/
public void clear() {
setSamplesAndSum(0, 0);
}
/**
* Set the sum and samples.
* Synchronized.
* @param sampleCount new sample count.
* @param newSum new sum
*/
public synchronized void setSamplesAndSum(long sampleCount,
long newSum) {
setSamples(sampleCount);
setSum(newSum);
}
/**
* Set the statistic to the values of another.
* Synchronized.
* @param other the source.
*/
public void set(final MeanStatistic other) {
setSamplesAndSum(other.getSamples(), other.getSum());
}
/**
* Set the sum.
* @param sum new sum
*/
public synchronized void setSum(final long sum) {
this.sum = sum;
}
/**
* Set the sample count.
*
* If this is less than zero, it is set to zero.
* This stops an ill-formed JSON entry from
* breaking deserialization, or get an invalid sample count
* into an entry.
* @param samples sample count.
*/
public synchronized void setSamples(final long samples) {
if (samples < 0) {
this.samples = 0;
} else {
this.samples = samples;
}
}
/**
* Get the arithmetic mean value.
* @return the mean
*/
public synchronized double mean() {
return samples > 0
? ((double) sum) / samples
: 0.0d;
}
/**
* Add another MeanStatistic.
* @param other other value
*/
public synchronized MeanStatistic add(final MeanStatistic other) {
if (other.isEmpty()) {
return this;
}
long otherSamples;
long otherSum;
synchronized (other) {
otherSamples = other.samples;
otherSum = other.sum;
}
if (isEmpty()) {
samples = otherSamples;
sum = otherSum;
return this;
}
samples += otherSamples;
sum += otherSum;
return this;
}
/**
* Add a sample.
* Thread safe.
* @param value value to add to the sum
*/
public synchronized void addSample(long value) {
samples++;
sum += value;
}
/**
* The hash code is derived from the mean
* and sample count: if either is changed
* the statistic cannot be used as a key
* for hash tables/maps.
* @return a hash value
*/
@Override
public synchronized int hashCode() {
return Objects.hash(sum, samples);
}
@Override
public synchronized boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
MeanStatistic that = (MeanStatistic) o;
if (isEmpty()) {
// if we are empty, then so must the other.
return that.isEmpty();
}
return getSum() == that.getSum() &&
getSamples() == that.getSamples();
}
@Override
public MeanStatistic clone() {
return copy();
}
/**
* Create a copy of this instance.
* @return copy.
*
*/
public MeanStatistic copy() {
return new MeanStatistic(this);
}
@Override
public String toString() {
return String.format("(samples=%d, sum=%d, mean=%.4f)",
samples, sum, mean());
}
}

View File

@ -0,0 +1,337 @@
/*
* 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.statistics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Common statistic names for object store operations..
* <p>
* When adding new common statistic name constants, please make them unique.
* By convention:
* </p>
* <ul>
* <li>the name of the constants are uppercase, words separated by
* underscores.</li>
* <li>the value of the constants are lowercase of the constant names.</li>
* </ul>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class StoreStatisticNames {
/** {@value}. */
public static final String OP_APPEND = "op_append";
/** {@value}. */
public static final String OP_COPY_FROM_LOCAL_FILE =
"op_copy_from_local_file";
/** {@value}. */
public static final String OP_CREATE = "op_create";
/** {@value}. */
public static final String OP_CREATE_NON_RECURSIVE =
"op_create_non_recursive";
/** {@value}. */
public static final String OP_DELETE = "op_delete";
/** {@value}. */
public static final String OP_EXISTS = "op_exists";
/** {@value}. */
public static final String OP_GET_CONTENT_SUMMARY =
"op_get_content_summary";
/** {@value}. */
public static final String OP_GET_DELEGATION_TOKEN =
"op_get_delegation_token";
/** {@value}. */
public static final String OP_GET_FILE_CHECKSUM =
"op_get_file_checksum";
/** {@value}. */
public static final String OP_GET_FILE_STATUS = "op_get_file_status";
/** {@value}. */
public static final String OP_GET_STATUS = "op_get_status";
/** {@value}. */
public static final String OP_GLOB_STATUS = "op_glob_status";
/** {@value}. */
public static final String OP_IS_FILE = "op_is_file";
/** {@value}. */
public static final String OP_IS_DIRECTORY = "op_is_directory";
/** {@value}. */
public static final String OP_LIST_FILES = "op_list_files";
/** {@value}. */
public static final String OP_LIST_LOCATED_STATUS =
"op_list_located_status";
/** {@value}. */
public static final String OP_LIST_STATUS = "op_list_status";
/** {@value}. */
public static final String OP_MKDIRS = "op_mkdirs";
/** {@value}. */
public static final String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries";
/** {@value}. */
public static final String OP_OPEN = "op_open";
/** {@value}. */
public static final String OP_REMOVE_ACL = "op_remove_acl";
/** {@value}. */
public static final String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries";
/** {@value}. */
public static final String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl";
/** {@value}. */
public static final String OP_RENAME = "op_rename";
/** {@value}. */
public static final String OP_SET_ACL = "op_set_acl";
/** {@value}. */
public static final String OP_SET_OWNER = "op_set_owner";
/** {@value}. */
public static final String OP_SET_PERMISSION = "op_set_permission";
/** {@value}. */
public static final String OP_SET_TIMES = "op_set_times";
/** {@value}. */
public static final String OP_TRUNCATE = "op_truncate";
/** {@value}. */
public static final String DELEGATION_TOKENS_ISSUED
= "delegation_tokens_issued";
/** Requests throttled and retried: {@value}. */
public static final String STORE_IO_THROTTLED
= "store_io_throttled";
/** Requests made of a store: {@value}. */
public static final String STORE_IO_REQUEST
= "store_io_request";
/**
* IO retried: {@value}.
*/
public static final String STORE_IO_RETRY
= "store_io_retry";
/**
* A store's equivalent of a paged LIST request was initiated: {@value}.
*/
public static final String OBJECT_LIST_REQUEST
= "object_list_request";
/**
* Number of continued object listings made.
* Value :{@value}.
*/
public static final String OBJECT_CONTINUE_LIST_REQUEST =
"object_continue_list_request";
/**
* A bulk DELETE request was made: {@value}.
* A separate statistic from {@link #OBJECT_DELETE_REQUEST}
* so that metrics on duration of the operations can
* be distinguished.
*/
public static final String OBJECT_BULK_DELETE_REQUEST
= "object_bulk_delete_request";
/**
* A store's equivalent of a DELETE request was made: {@value}.
* This may be an HTTP DELETE verb, or it may be some custom
* operation which takes a list of objects to delete.
*/
public static final String OBJECT_DELETE_REQUEST
= "object_delete_request";
/**
* The count of objects deleted in delete requests.
*/
public static final String OBJECT_DELETE_OBJECTS
= "object_delete_objects";
/**
* Object multipart upload initiated.
* Value :{@value}.
*/
public static final String OBJECT_MULTIPART_UPLOAD_INITIATED =
"object_multipart_initiated";
/**
* Object multipart upload aborted.
* Value :{@value}.
*/
public static final String OBJECT_MULTIPART_UPLOAD_ABORTED =
"object_multipart_aborted";
/**
* Object put/multipart upload count.
* Value :{@value}.
*/
public static final String OBJECT_PUT_REQUEST =
"object_put_request";
/**
* Object put/multipart upload completed count.
* Value :{@value}.
*/
public static final String OBJECT_PUT_REQUEST_COMPLETED =
"object_put_request_completed";
/**
* Current number of active put requests.
* Value :{@value}.
*/
public static final String OBJECT_PUT_REQUEST_ACTIVE =
"object_put_request_active";
/**
* number of bytes uploaded.
* Value :{@value}.
*/
public static final String OBJECT_PUT_BYTES =
"object_put_bytes";
/**
* number of bytes queued for upload/being actively uploaded.
* Value :{@value}.
*/
public static final String OBJECT_PUT_BYTES_PENDING =
"object_put_bytes_pending";
/**
* Count of S3 Select (or similar) requests issued.
* Value :{@value}.
*/
public static final String OBJECT_SELECT_REQUESTS =
"object_select_requests";
/**
* Suffix to use for a minimum value when
* the same key is shared across min/mean/max
* statistics.
*
* Value {@value}.
*/
public static final String SUFFIX_MIN = ".min";
/**
* Suffix to use for a maximum value when
* the same key is shared across max/mean/max
* statistics.
*
* Value {@value}.
*/
public static final String SUFFIX_MAX = ".max";
/**
* Suffix to use for a mean statistic value when
* the same key is shared across mean/mean/max
* statistics.
*
* Value {@value}.
*/
public static final String SUFFIX_MEAN = ".mean";
/**
* String to add to counters and other stats to track failures.
* This comes before the .min/.mean//max suffixes.
*
* Value {@value}.
*/
public static final String SUFFIX_FAILURES = ".failures";
/**
* The name of the statistic collected for executor acquisition if
* a duration tracker factory is passed in to the constructor.
* {@value}.
*/
public static final String ACTION_EXECUTOR_ACQUIRED =
"action_executor_acquired";
/**
* An HTTP HEAD request was made: {@value}.
*/
public static final String ACTION_HTTP_HEAD_REQUEST
= "action_http_head_request";
/**
* An HTTP GET request was made: {@value}.
*/
public static final String ACTION_HTTP_GET_REQUEST
= "action_http_get_request";
/**
* An HTTP HEAD request was made: {@value}.
*/
public static final String OBJECT_METADATA_REQUESTS
= "object_metadata_request";
public static final String OBJECT_COPY_REQUESTS
= "object_copy_requests";
public static final String STORE_IO_THROTTLE_RATE
= "store_io_throttle_rate";
public static final String DELEGATION_TOKEN_ISSUED
= "delegation_token_issued";
public static final String MULTIPART_UPLOAD_INSTANTIATED
= "multipart_instantiated";
public static final String MULTIPART_UPLOAD_PART_PUT
= "multipart_upload_part_put";
public static final String MULTIPART_UPLOAD_PART_PUT_BYTES
= "multipart_upload_part_put_bytes";
public static final String MULTIPART_UPLOAD_ABORTED
= "multipart_upload_aborted";
public static final String MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED
= "multipart_upload_abort_under_path_invoked";
public static final String MULTIPART_UPLOAD_COMPLETED
= "multipart_upload_completed";
public static final String MULTIPART_UPLOAD_STARTED
= "multipart_upload_started";
private StoreStatisticNames() {
}
}

View File

@ -0,0 +1,292 @@
/*
* 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.statistics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* These are common statistic names.
* <p>
* When adding new common statistic name constants, please make them unique.
* By convention, they are implicitly unique:
* <ul>
* <li>
* The name of the constants are uppercase, words separated by
* underscores.
* </li>
* <li>
* The value of the constants are lowercase of the constant names.
* </li>
* </ul>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class StreamStatisticNames {
/**
* Count of times the TCP stream was aborted.
* Value: {@value}.
*/
public static final String STREAM_READ_ABORTED = "stream_aborted";
/**
* Bytes read from an input stream in read() calls.
* Does not include bytes read and then discarded in seek/close etc.
* These are the bytes returned to the caller.
* Value: {@value}.
*/
public static final String STREAM_READ_BYTES
= "stream_read_bytes";
/**
* Count of bytes discarded by aborting an input stream .
* Value: {@value}.
*/
public static final String STREAM_READ_BYTES_DISCARDED_ABORT
= "stream_read_bytes_discarded_in_abort";
/**
* Count of bytes read and discarded when closing an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_BYTES_DISCARDED_CLOSE
= "stream_read_bytes_discarded_in_close";
/**
* Count of times the TCP stream was closed.
* Value: {@value}.
*/
public static final String STREAM_READ_CLOSED = "stream_read_closed";
/**
* Total count of times an attempt to close an input stream was made
* Value: {@value}.
*/
public static final String STREAM_READ_CLOSE_OPERATIONS
= "stream_read_close_operations";
/**
* Total count of times an input stream to was opened.
* For object stores, that means the count a GET request was initiated.
* Value: {@value}.
*/
public static final String STREAM_READ_OPENED = "stream_read_opened";
/**
* Count of exceptions raised during input stream reads.
* Value: {@value}.
*/
public static final String STREAM_READ_EXCEPTIONS =
"stream_read_exceptions";
/**
* Count of readFully() operations in an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_FULLY_OPERATIONS
= "stream_read_fully_operations";
/**
* Count of read() operations in an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_OPERATIONS =
"stream_read_operations";
/**
* Count of incomplete read() operations in an input stream,
* that is, when the bytes returned were less than that requested.
* Value: {@value}.
*/
public static final String STREAM_READ_OPERATIONS_INCOMPLETE
= "stream_read_operations_incomplete";
/**
* Count of version mismatches encountered while reading an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_VERSION_MISMATCHES
= "stream_read_version_mismatches";
/**
* Count of executed seek operations which went backwards in a stream.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS =
"stream_read_seek_backward_operations";
/**
* Count of bytes moved backwards during seek operations
* in an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_BYTES_BACKWARDS
= "stream_read_bytes_backwards_on_seek";
/**
* Count of bytes read and discarded during seek() in an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_BYTES_DISCARDED =
"stream_read_seek_bytes_discarded";
/**
* Count of bytes skipped during forward seek operations.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_BYTES_SKIPPED
= "stream_read_seek_bytes_skipped";
/**
* Count of executed seek operations which went forward in
* an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS
= "stream_read_seek_forward_operations";
/**
* Count of times the seek policy was dynamically changed
* in an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_POLICY_CHANGED =
"stream_read_seek_policy_changed";
/**
* Count of seek operations in an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_SEEK_OPERATIONS =
"stream_read_seek_operations";
/**
* Count of {@code InputStream.skip()} calls.
* Value: {@value}.
*/
public static final String STREAM_READ_SKIP_OPERATIONS =
"stream_read_skip_operations";
/**
* Count bytes skipped in {@code InputStream.skip()} calls.
* Value: {@value}.
*/
public static final String STREAM_READ_SKIP_BYTES =
"stream_read_skip_bytes";
/**
* Total count of bytes read from an input stream.
* Value: {@value}.
*/
public static final String STREAM_READ_TOTAL_BYTES
= "stream_read_total_bytes";
/**
* Count of calls of {@code CanUnbuffer.unbuffer()}.
* Value: {@value}.
*/
public static final String STREAM_READ_UNBUFFERED
= "stream_read_unbuffered";
/**
* "Count of stream write failures reported.
* Value: {@value}.
*/
public static final String STREAM_WRITE_EXCEPTIONS =
"stream_write_exceptions";
/**
* Count of failures when finalizing a multipart upload:
* {@value}.
*/
public static final String STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS =
"stream_write_exceptions_completing_upload";
/**
* Count of block/partition uploads complete.
* Value: {@value}.
*/
public static final String STREAM_WRITE_BLOCK_UPLOADS
= "stream_write_block_uploads";
/**
* Count of number of block uploads aborted.
* Value: {@value}.
*/
public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED
= "stream_write_block_uploads_aborted";
/**
* Count of block/partition uploads active.
* Value: {@value}.
*/
public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE
= "stream_write_block_uploads_active";
/**
* Gauge of data queued to be written.
* Value: {@value}.
*/
public static final String STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING =
"stream_write_block_uploads_data_pending";
/**
* Count of number of block uploads committed.
* Value: {@value}.
*/
public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED
= "stream_write_block_uploads_committed";
/**
* Gauge of block/partitions uploads queued to be written.
* Value: {@value}.
*/
public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING
= "stream_write_block_uploads_pending";
/**
* "Count of bytes written to output stream including all not yet uploaded.
* {@value}.
*/
public static final String STREAM_WRITE_BYTES
= "stream_write_bytes";
/**
* Count of total time taken for uploads to complete.
* {@value}.
*/
public static final String STREAM_WRITE_TOTAL_TIME
= "stream_write_total_time";
/**
* Total queue duration of all block uploads.
* {@value}.
*/
public static final String STREAM_WRITE_QUEUE_DURATION
= "stream_write_queue_duration";
public static final String STREAM_WRITE_TOTAL_DATA
= "stream_write_total_data";
private StreamStatisticNames() {
}
}

View File

@ -0,0 +1,30 @@
/*
* 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.statistics.impl;
import org.apache.hadoop.fs.statistics.IOStatistics;
/**
* Base implementation in case common methods/fields need to be added
* in future.
*/
public abstract class AbstractIOStatisticsImpl implements IOStatistics {
}

View File

@ -0,0 +1,132 @@
/*
* 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.statistics.impl;
import java.util.Collections;
import java.util.Map;
import java.util.function.Function;
import org.apache.hadoop.fs.statistics.MeanStatistic;
/**
* These statistics are dynamically evaluated by the supplied
* String -&gt; type functions.
*
* This allows statistic sources to supply a list of callbacks used to
* generate the statistics on demand; similar to some of the Coda Hale metrics.
*
* The evaluation actually takes place during the iteration's {@code next()}
* call.
*/
final class DynamicIOStatistics
extends AbstractIOStatisticsImpl {
/**
* Counter evaluators.
*/
private final EvaluatingStatisticsMap<Long> counters
= new EvaluatingStatisticsMap<>();
private final EvaluatingStatisticsMap<Long> gauges
= new EvaluatingStatisticsMap<>();
private final EvaluatingStatisticsMap<Long> minimums
= new EvaluatingStatisticsMap<>();
private final EvaluatingStatisticsMap<Long> maximums
= new EvaluatingStatisticsMap<>();
private final EvaluatingStatisticsMap<MeanStatistic> meanStatistics
= new EvaluatingStatisticsMap<>(MeanStatistic::copy);
DynamicIOStatistics() {
}
@Override
public Map<String, Long> counters() {
return Collections.unmodifiableMap(counters);
}
@Override
public Map<String, Long> gauges() {
return Collections.unmodifiableMap(gauges);
}
@Override
public Map<String, Long> minimums() {
return Collections.unmodifiableMap(minimums);
}
@Override
public Map<String, Long> maximums() {
return Collections.unmodifiableMap(maximums);
}
@Override
public Map<String, MeanStatistic> meanStatistics() {
return Collections.unmodifiableMap(meanStatistics);
}
/**
* add a mapping of a key to a counter function.
* @param key the key
* @param eval the evaluator
*/
void addCounterFunction(String key, Function<String, Long> eval) {
counters.addFunction(key, eval);
}
/**
* add a mapping of a key to a gauge function.
* @param key the key
* @param eval the evaluator
*/
void addGaugeFunction(String key, Function<String, Long> eval) {
gauges.addFunction(key, eval);
}
/**
* add a mapping of a key to a minimum function.
* @param key the key
* @param eval the evaluator
*/
void addMinimumFunction(String key, Function<String, Long> eval) {
minimums.addFunction(key, eval);
}
/**
* add a mapping of a key to a maximum function.
* @param key the key
* @param eval the evaluator
*/
void addMaximumFunction(String key, Function<String, Long> eval) {
maximums.addFunction(key, eval);
}
/**
* add a mapping of a key to a meanStatistic function.
* @param key the key
* @param eval the evaluator
*/
void addMeanStatisticFunction(String key,
Function<String, MeanStatistic> eval) {
meanStatistics.addFunction(key, eval);
}
}

View File

@ -0,0 +1,248 @@
/*
* 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.statistics.impl;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.function.ToLongFunction;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.MeanStatistic;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState;
/**
* Builder of {@link DynamicIOStatistics}.
*
* Instantiate through
* {@link IOStatisticsBinding#dynamicIOStatistics()}.
*/
public class DynamicIOStatisticsBuilder {
/**
* the instance being built up. Will be null after the (single)
* call to {@link #build()}.
*/
private DynamicIOStatistics instance = new DynamicIOStatistics();
/**
* Build the IOStatistics instance.
* @return an instance.
* @throws IllegalStateException if the builder has already been built.
*/
public IOStatistics build() {
final DynamicIOStatistics stats = activeInstance();
// stop the builder from working any more.
instance = null;
return stats;
}
/**
* Get the statistics instance.
* @return the instance to build/return
* @throws IllegalStateException if the builder has already been built.
*/
private DynamicIOStatistics activeInstance() {
checkState(instance != null, "Already built");
return instance;
}
/**
* Add a new evaluator to the counter statistics.
* @param key key of this statistic
* @param eval evaluator for the statistic
* @return the builder.
*/
public DynamicIOStatisticsBuilder withLongFunctionCounter(String key,
ToLongFunction<String> eval) {
activeInstance().addCounterFunction(key, eval::applyAsLong);
return this;
}
/**
* Add a counter statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic long counter
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicLongCounter(String key,
AtomicLong source) {
withLongFunctionCounter(key, s -> source.get());
return this;
}
/**
* Add a counter statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic int counter
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicIntegerCounter(String key,
AtomicInteger source) {
withLongFunctionCounter(key, s -> source.get());
return this;
}
/**
* Build a dynamic counter statistic from a
* {@link MutableCounterLong}.
* @param key key of this statistic
* @param source mutable long counter
* @return the builder.
*/
public DynamicIOStatisticsBuilder withMutableCounter(String key,
MutableCounterLong source) {
withLongFunctionCounter(key, s -> source.value());
return this;
}
/**
* Add a new evaluator to the gauge statistics.
* @param key key of this statistic
* @param eval evaluator for the statistic
* @return the builder.
*/
public DynamicIOStatisticsBuilder withLongFunctionGauge(String key,
ToLongFunction<String> eval) {
activeInstance().addGaugeFunction(key, eval::applyAsLong);
return this;
}
/**
* Add a gauge statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic long gauge
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicLongGauge(String key,
AtomicLong source) {
withLongFunctionGauge(key, s -> source.get());
return this;
}
/**
* Add a gauge statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic int gauge
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicIntegerGauge(String key,
AtomicInteger source) {
withLongFunctionGauge(key, s -> source.get());
return this;
}
/**
* Add a new evaluator to the minimum statistics.
* @param key key of this statistic
* @param eval evaluator for the statistic
* @return the builder.
*/
public DynamicIOStatisticsBuilder withLongFunctionMinimum(String key,
ToLongFunction<String> eval) {
activeInstance().addMinimumFunction(key, eval::applyAsLong);
return this;
}
/**
* Add a minimum statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic long minimum
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicLongMinimum(String key,
AtomicLong source) {
withLongFunctionMinimum(key, s -> source.get());
return this;
}
/**
* Add a minimum statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic int minimum
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicIntegerMinimum(String key,
AtomicInteger source) {
withLongFunctionMinimum(key, s -> source.get());
return this;
}
/**
* Add a new evaluator to the maximum statistics.
* @param key key of this statistic
* @param eval evaluator for the statistic
* @return the builder.
*/
public DynamicIOStatisticsBuilder withLongFunctionMaximum(String key,
ToLongFunction<String> eval) {
activeInstance().addMaximumFunction(key, eval::applyAsLong);
return this;
}
/**
* Add a maximum statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic long maximum
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicLongMaximum(String key,
AtomicLong source) {
withLongFunctionMaximum(key, s -> source.get());
return this;
}
/**
* Add a maximum statistic to dynamically return the
* latest value of the source.
* @param key key of this statistic
* @param source atomic int maximum
* @return the builder.
*/
public DynamicIOStatisticsBuilder withAtomicIntegerMaximum(String key,
AtomicInteger source) {
withLongFunctionMaximum(key, s -> source.get());
return this;
}
/**
* Add a new evaluator to the mean statistics.
*
* This is a function which must return the mean and the sample count.
* @param key key of this statistic
* @param eval evaluator for the statistic
* @return the builder.
*/
public DynamicIOStatisticsBuilder withMeanStatisticFunction(String key,
Function<String, MeanStatistic> eval) {
activeInstance().addMeanStatisticFunction(key, eval);
return this;
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.statistics.impl;
import java.util.Map;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.MeanStatistic;
import static java.util.Collections.emptyMap;
/**
* An empty IO Statistics implementation for classes which always
* want to return a non-null set of statistics.
*/
final class EmptyIOStatistics extends AbstractIOStatisticsImpl {
/**
* The sole instance of this class.
*/
private static final EmptyIOStatistics INSTANCE = new EmptyIOStatistics();
private EmptyIOStatistics() {
}
@Override
public Map<String, Long> counters() {
return emptyMap();
}
@Override
public Map<String, Long> gauges() {
return emptyMap();
}
@Override
public Map<String, Long> minimums() {
return emptyMap();
}
@Override
public Map<String, Long> maximums() {
return emptyMap();
}
@Override
public Map<String, MeanStatistic> meanStatistics() {
return emptyMap();
}
/**
* Get the single instance of this class.
* @return a shared, empty instance.
*/
public static IOStatistics getInstance() {
return INSTANCE;
}
}

View File

@ -0,0 +1,202 @@
/*
* 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.statistics.impl;
import java.io.Serializable;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
* A map of functions which can be invoked to dynamically
* create the value of an entry.
* @param <E> type of entry value.
*/
final class EvaluatingStatisticsMap<E extends Serializable> implements
Map<String, E> {
/**
* Functions to invoke when evaluating keys.
*/
private final Map<String, Function<String, E>> evaluators
= new ConcurrentHashMap<>();
/**
* Function to use when copying map values.
*/
private final Function<E, E> copyFn;
/**
* Construct with the copy function being simple passthrough.
*/
EvaluatingStatisticsMap() {
this(IOStatisticsBinding::passthroughFn);
}
/**
* Construct with the copy function being that supplied in.
* @param copyFn copy function.
*/
EvaluatingStatisticsMap(final Function<E, E> copyFn) {
this.copyFn = copyFn;
}
/**
* add a mapping of a key to a function.
* @param key the key
* @param eval the evaluator
*/
void addFunction(String key, Function<String, E> eval) {
evaluators.put(key, eval);
}
@Override
public int size() {
return evaluators.size();
}
@Override
public boolean isEmpty() {
return evaluators.isEmpty();
}
@Override
public boolean containsKey(final Object key) {
return evaluators.containsKey(key);
}
@Override
public boolean containsValue(final Object value) {
throw new UnsupportedOperationException();
}
@Override
public E get(final Object key) {
Function<String, E> fn = evaluators.get(key);
return fn != null
? fn.apply((String) key)
: null;
}
@Override
public E put(final String key, final E value) {
throw new UnsupportedOperationException();
}
@Override
public E remove(final Object key) {
throw new UnsupportedOperationException();
}
@Override
public void putAll(final Map<? extends String, ? extends E> m) {
throw new UnsupportedOperationException();
}
@Override
public void clear() {
throw new UnsupportedOperationException();
}
@Override
public Set<String> keySet() {
return evaluators.keySet();
}
/**
* Evaluate all the entries and provide a list of the results.
*
* This is not a snapshot, so if the evaluators actually return
* references to mutable objects (e.g. a MeanStatistic instance)
* then that value may still change.
* @return the current list of evaluated results.
*/
@Override
public Collection<E> values() {
Set<Entry<String, Function<String, E>>> evalEntries =
evaluators.entrySet();
return evalEntries.parallelStream().map((e) ->
e.getValue().apply(e.getKey()))
.collect(Collectors.toList());
}
/**
* Take a snapshot.
* @return a map snapshot.
*/
public Map<String, E> snapshot() {
return IOStatisticsBinding.snapshotMap(this, copyFn);
}
/**
* Creating the entry set forces an evaluation of the functions.
*
* This is not a snapshot, so if the evaluators actually return
* references to mutable objects (e.g. a MeanStatistic instance)
* then that value may still change.
*
* The evaluation may be parallelized.
* @return an evaluated set of values
*/
@Override
public synchronized Set<Entry<String, E>> entrySet() {
Set<Entry<String, Function<String, E>>> evalEntries =
evaluators.entrySet();
Set<Entry<String, E>> r = evalEntries.parallelStream().map((e) ->
new EntryImpl<>(e.getKey(), e.getValue().apply(e.getKey())))
.collect(Collectors.toSet());
return r;
}
/**
* Simple entry.
* @param <E> entry type
*/
private static final class EntryImpl<E> implements Entry<String, E> {
private String key;
private E value;
private EntryImpl(final String key, final E value) {
this.key = key;
this.value = value;
}
@Override
public String getKey() {
return key;
}
@Override
public E getValue() {
return value;
}
@Override
public E setValue(final E val) {
this.value = val;
return val;
}
}
}

View File

@ -0,0 +1,616 @@
/*
* 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.statistics.impl;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.Serializable;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.BiFunction;
import java.util.function.Function;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.fs.StorageStatistics;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.MeanStatistic;
import org.apache.hadoop.util.functional.CallableRaisingIOE;
import org.apache.hadoop.util.functional.ConsumerRaisingIOE;
import org.apache.hadoop.util.functional.FunctionRaisingIOE;
import org.apache.hadoop.util.functional.InvocationRaisingIOE;
import static org.apache.hadoop.fs.statistics.IOStatistics.MIN_UNSET_VALUE;
import static org.apache.hadoop.fs.statistics.impl.StubDurationTracker.STUB_DURATION_TRACKER;
/**
* Support for implementing IOStatistics interfaces.
*/
public final class IOStatisticsBinding {
/** Pattern used for each entry. */
public static final String ENTRY_PATTERN = "(%s=%s)";
/** String to return when a source is null. */
@VisibleForTesting
public static final String NULL_SOURCE = "()";
private IOStatisticsBinding() {
}
/**
* Create IOStatistics from a storage statistics instance.
*
* This will be updated as the storage statistics change.
* @param storageStatistics source data.
* @return an IO statistics source.
*/
public static IOStatistics fromStorageStatistics(
StorageStatistics storageStatistics) {
DynamicIOStatisticsBuilder builder = dynamicIOStatistics();
Iterator<StorageStatistics.LongStatistic> it = storageStatistics
.getLongStatistics();
while (it.hasNext()) {
StorageStatistics.LongStatistic next = it.next();
builder.withLongFunctionCounter(next.getName(),
k -> storageStatistics.getLong(k));
}
return builder.build();
}
/**
* Create a builder for dynamic IO Statistics.
* @return a builder to be completed.
*/
public static DynamicIOStatisticsBuilder dynamicIOStatistics() {
return new DynamicIOStatisticsBuilder();
}
/**
* Get the shared instance of the immutable empty statistics
* object.
* @return an empty statistics object.
*/
public static IOStatistics emptyStatistics() {
return EmptyIOStatistics.getInstance();
}
/**
* Take an IOStatistics instance and wrap it in a source.
* @param statistics statistics.
* @return a source which will return the values
*/
public static IOStatisticsSource wrap(IOStatistics statistics) {
return new SourceWrappedStatistics(statistics);
}
/**
* Create a builder for an {@link IOStatisticsStore}.
*
* @return a builder instance.
*/
public static IOStatisticsStoreBuilder iostatisticsStore() {
return new IOStatisticsStoreBuilderImpl();
}
/**
* Convert an entry to the string format used in logging.
*
* @param entry entry to evaluate
* @param <E> entry type
* @return formatted string
*/
public static <E> String entryToString(
final Map.Entry<String, E> entry) {
return entryToString(entry.getKey(), entry.getValue());
}
/**
* Convert entry values to the string format used in logging.
*
* @param name statistic name
* @param value stat value
* @return formatted string
*/
public static <E> String entryToString(
final String name, final E value) {
return String.format(
ENTRY_PATTERN,
name,
value);
}
/**
* Copy into the dest map all the source entries.
* The destination is cleared first.
* @param <E> entry type
* @param dest destination of the copy
* @param source source
* @param copyFn function to copy entries
* @return the destination.
*/
private static <E> Map<String, E> copyMap(
Map<String, E> dest,
Map<String, E> source,
Function<E, E> copyFn) {
// we have to clone the values so that they aren't
// bound to the original values
dest.clear();
source.entrySet()
.forEach(entry ->
dest.put(entry.getKey(), copyFn.apply(entry.getValue())));
return dest;
}
/**
* A passthrough copy operation suitable for immutable
* types, including numbers.
* @param src source object
* @return the source object
*/
public static <E extends Serializable> E passthroughFn(E src) {
return src;
}
/**
* Take a snapshot of a supplied map, where the copy option simply
* uses the existing value.
*
* For this to be safe, the map must refer to immutable objects.
* @param source source map
* @param <E> type of values.
* @return a new map referencing the same values.
*/
public static <E extends Serializable> Map<String, E> snapshotMap(
Map<String, E> source) {
return snapshotMap(source,
IOStatisticsBinding::passthroughFn);
}
/**
* Take a snapshot of a supplied map, using the copy function
* to replicate the source values.
* @param source source map
* @param copyFn function to copy the value
* @param <E> type of values.
* @return a concurrent hash map referencing the same values.
*/
public static <E extends Serializable>
ConcurrentHashMap<String, E> snapshotMap(
Map<String, E> source,
Function<E, E> copyFn) {
ConcurrentHashMap<String, E> dest = new ConcurrentHashMap<>();
copyMap(dest, source, copyFn);
return dest;
}
/**
* Aggregate two maps so that the destination.
* @param <E> type of values
* @param dest destination map.
* @param other other map
* @param aggregateFn function to aggregate the values.
* @param copyFn function to copy the value
*/
public static <E> void aggregateMaps(
Map<String, E> dest,
Map<String, E> other,
BiFunction<E, E, E> aggregateFn,
Function<E, E> copyFn) {
// scan through the other hand map; copy
// any values not in the left map,
// aggregate those for which there is already
// an entry
other.entrySet().forEach(entry -> {
String key = entry.getKey();
E rVal = entry.getValue();
E lVal = dest.get(key);
if (lVal == null) {
dest.put(key, copyFn.apply(rVal));
} else {
dest.put(key, aggregateFn.apply(lVal, rVal));
}
});
}
/**
* Aggregate two counters.
* @param l left value
* @param r right value
* @return the aggregate value
*/
public static Long aggregateCounters(Long l, Long r) {
return Math.max(l, 0) + Math.max(r, 0);
}
/**
* Add two gauges.
* @param l left value
* @param r right value
* @return aggregate value
*/
public static Long aggregateGauges(Long l, Long r) {
return l + r;
}
/**
* Aggregate two minimum values.
* @param l left
* @param r right
* @return the new minimum.
*/
public static Long aggregateMinimums(Long l, Long r) {
if (l == MIN_UNSET_VALUE) {
return r;
} else if (r == MIN_UNSET_VALUE) {
return l;
} else {
return Math.min(l, r);
}
}
/**
* Aggregate two maximum values.
* @param l left
* @param r right
* @return the new minimum.
*/
public static Long aggregateMaximums(Long l, Long r) {
if (l == MIN_UNSET_VALUE) {
return r;
} else if (r == MIN_UNSET_VALUE) {
return l;
} else {
return Math.max(l, r);
}
}
/**
* Aggregate the mean statistics.
* This returns a new instance.
* @param l left value
* @param r right value
* @return aggregate value
*/
public static MeanStatistic aggregateMeanStatistics(
MeanStatistic l, MeanStatistic r) {
MeanStatistic res = l.copy();
res.add(r);
return res;
}
/**
* Update a maximum value tracked in an atomic long.
* This is thread safe -it uses compareAndSet to ensure
* that Thread T1 whose sample is greater than the current
* value never overwrites an update from thread T2 whose
* sample was also higher -and which completed first.
* @param dest destination for all changes.
* @param sample sample to update.
*/
public static void maybeUpdateMaximum(AtomicLong dest, long sample) {
boolean done;
do {
long current = dest.get();
if (sample > current) {
done = dest.compareAndSet(current, sample);
} else {
done = true;
}
} while (!done);
}
/**
* Update a maximum value tracked in an atomic long.
* This is thread safe -it uses compareAndSet to ensure
* that Thread T1 whose sample is greater than the current
* value never overwrites an update from thread T2 whose
* sample was also higher -and which completed first.
* @param dest destination for all changes.
* @param sample sample to update.
*/
public static void maybeUpdateMinimum(AtomicLong dest, long sample) {
boolean done;
do {
long current = dest.get();
if (current == MIN_UNSET_VALUE || sample < current) {
done = dest.compareAndSet(current, sample);
} else {
done = true;
}
} while (!done);
}
/**
* Given an IOException raising function/lambda expression,
* return a new one which wraps the inner and tracks
* the duration of the operation, including whether
* it passes/fails.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param inputFn input function
* @param <A> type of argument to the input function.
* @param <B> return type.
* @return a new function which tracks duration and failure.
*/
public static <A, B> FunctionRaisingIOE<A, B> trackFunctionDuration(
@Nullable DurationTrackerFactory factory,
String statistic,
FunctionRaisingIOE<A, B> inputFn) {
return (x) -> {
// create the tracker outside try-with-resources so
// that failures can be set in the catcher.
DurationTracker tracker = createTracker(factory, statistic);
try {
// exec the input function and return its value
return inputFn.apply(x);
} catch (IOException | RuntimeException e) {
// input function failed: note it
tracker.failed();
// and rethrow
throw e;
} finally {
// update the tracker.
// this is called after the catch() call would have
// set the failed flag.
tracker.close();
}
};
}
/**
* Given a java function/lambda expression,
* return a new one which wraps the inner and tracks
* the duration of the operation, including whether
* it passes/fails.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param inputFn input function
* @param <A> type of argument to the input function.
* @param <B> return type.
* @return a new function which tracks duration and failure.
*/
public static <A, B> Function<A, B> trackJavaFunctionDuration(
@Nullable DurationTrackerFactory factory,
String statistic,
Function<A, B> inputFn) {
return (x) -> {
// create the tracker outside try-with-resources so
// that failures can be set in the catcher.
DurationTracker tracker = createTracker(factory, statistic);
try {
// exec the input function and return its value
return inputFn.apply(x);
} catch (RuntimeException e) {
// input function failed: note it
tracker.failed();
// and rethrow
throw e;
} finally {
// update the tracker.
// this is called after the catch() call would have
// set the failed flag.
tracker.close();
}
};
}
/**
* Given an IOException raising callable/lambda expression,
* execute it and update the relevant statistic.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param input input callable.
* @param <B> return type.
* @return the result of the operation.
*/
public static <B> B trackDuration(
DurationTrackerFactory factory,
String statistic,
CallableRaisingIOE<B> input) throws IOException {
return trackDurationOfOperation(factory, statistic, input).apply();
}
/**
* Given an IOException raising callable/lambda expression,
* execute it and update the relevant statistic.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param input input callable.
*/
public static void trackDurationOfInvocation(
DurationTrackerFactory factory,
String statistic,
InvocationRaisingIOE input) throws IOException {
// create the tracker outside try-with-resources so
// that failures can be set in the catcher.
DurationTracker tracker = createTracker(factory, statistic);
try {
// exec the input function and return its value
input.apply();
} catch (IOException | RuntimeException e) {
// input function failed: note it
tracker.failed();
// and rethrow
throw e;
} finally {
// update the tracker.
// this is called after the catch() call would have
// set the failed flag.
tracker.close();
}
}
/**
* Given an IOException raising callable/lambda expression,
* return a new one which wraps the inner and tracks
* the duration of the operation, including whether
* it passes/fails.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param input input callable.
* @param <B> return type.
* @return a new callable which tracks duration and failure.
*/
public static <B> CallableRaisingIOE<B> trackDurationOfOperation(
@Nullable DurationTrackerFactory factory,
String statistic,
CallableRaisingIOE<B> input) {
return () -> {
// create the tracker outside try-with-resources so
// that failures can be set in the catcher.
DurationTracker tracker = createTracker(factory, statistic);
try {
// exec the input function and return its value
return input.apply();
} catch (IOException | RuntimeException e) {
// input function failed: note it
tracker.failed();
// and rethrow
throw e;
} finally {
// update the tracker.
// this is called after the catch() call would have
// set the failed flag.
tracker.close();
}
};
}
/**
* Given an IOException raising Consumer,
* return a new one which wraps the inner and tracks
* the duration of the operation, including whether
* it passes/fails.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param input input callable.
* @param <B> return type.
* @return a new consumer which tracks duration and failure.
*/
public static <B> ConsumerRaisingIOE<B> trackDurationConsumer(
@Nullable DurationTrackerFactory factory,
String statistic,
ConsumerRaisingIOE<B> input) {
return (B t) -> {
// create the tracker outside try-with-resources so
// that failures can be set in the catcher.
DurationTracker tracker = createTracker(factory, statistic);
try {
// exec the input function and return its value
input.accept(t);
} catch (IOException | RuntimeException e) {
// input function failed: note it
tracker.failed();
// and rethrow
throw e;
} finally {
// update the tracker.
// this is called after the catch() call would have
// set the failed flag.
tracker.close();
}
};
}
/**
* Given a callable/lambda expression,
* return a new one which wraps the inner and tracks
* the duration of the operation, including whether
* it passes/fails.
* @param factory factory of duration trackers
* @param statistic statistic key
* @param input input callable.
* @param <B> return type.
* @return a new callable which tracks duration and failure.
*/
public static <B> Callable<B> trackDurationOfCallable(
@Nullable DurationTrackerFactory factory,
String statistic,
Callable<B> input) {
return () -> {
// create the tracker outside try-with-resources so
// that failures can be set in the catcher.
DurationTracker tracker = createTracker(factory, statistic);
try {
// exec the input function and return its value
return input.call();
} catch (RuntimeException e) {
// input function failed: note it
tracker.failed();
// and rethrow
throw e;
} finally {
// update the tracker.
// this is called after any catch() call will have
// set the failed flag.
tracker.close();
}
};
}
/**
* Create the tracker. If the factory is null, a stub
* tracker is returned.
* @param factory tracker factory
* @param statistic statistic to track
* @return a duration tracker.
*/
private static DurationTracker createTracker(
@Nullable final DurationTrackerFactory factory,
final String statistic) {
return factory != null
? factory.trackDuration(statistic)
: STUB_DURATION_TRACKER;
}
/**
* Create a DurationTrackerFactory which aggregates the tracking
* of two other factories.
* @param first first tracker factory
* @param second second tracker factory
* @return a factory
*/
public static DurationTrackerFactory pairedTrackerFactory(
final DurationTrackerFactory first,
final DurationTrackerFactory second) {
return new PairedDurationTrackerFactory(first, second);
}
/**
* Publish the IOStatistics as a set of storage statistics.
* This is dynamic.
* @param name storage statistics name.
* @param scheme FS scheme; may be null.
* @param source IOStatistics source.
* @return a dynamic storage statistics object.
*/
public static StorageStatistics publishAsStorageStatistics(
String name, String scheme, IOStatistics source) {
return new StorageStatisticsFromIOStatistics(name, scheme, source);
}
}

View File

@ -0,0 +1,258 @@
/*
* 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.statistics.impl;
import java.time.Duration;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
import org.apache.hadoop.fs.statistics.MeanStatistic;
/**
* Interface of an IOStatistics store intended for
* use in classes which track statistics for reporting.
*/
public interface IOStatisticsStore extends IOStatistics,
IOStatisticsAggregator,
DurationTrackerFactory {
/**
* Increment a counter by one.
*
* No-op if the counter is unknown.
* @param key statistics key
* @return old value or, if the counter is unknown: 0
*/
default long incrementCounter(String key) {
return incrementCounter(key, 1);
}
/**
* Increment a counter.
*
* No-op if the counter is unknown.
* If the value is negative, it is ignored.
* @param key statistics key
* @param value value to increment
* @return the updated value or, if the counter is unknown: 0
*/
long incrementCounter(String key, long value);
/**
* Set a counter.
*
* No-op if the counter is unknown.
* @param key statistics key
* @param value value to set
*/
void setCounter(String key, long value);
/**
* Set a gauge.
*
* No-op if the gauge is unknown.
* @param key statistics key
* @param value value to set
*/
void setGauge(String key, long value);
/**
* Increment a gauge.
* <p>
* No-op if the gauge is unknown.
* </p>
* @param key statistics key
* @param value value to increment
* @return new value or 0 if the key is unknown
*/
long incrementGauge(String key, long value);
/**
* Set a maximum.
* No-op if the maximum is unknown.
* @param key statistics key
* @param value value to set
*/
void setMaximum(String key, long value);
/**
* Increment a maximum.
* <p>
* No-op if the maximum is unknown.
* </p>
* @param key statistics key
* @param value value to increment
* @return new value or 0 if the key is unknown
*/
long incrementMaximum(String key, long value);
/**
* Set a minimum.
* <p>
* No-op if the minimum is unknown.
* </p>
* @param key statistics key
* @param value value to set
*/
void setMinimum(String key, long value);
/**
* Increment a minimum.
* <p>
* No-op if the minimum is unknown.
* </p>
* @param key statistics key
* @param value value to increment
* @return new value or 0 if the key is unknown
*/
long incrementMinimum(String key, long value);
/**
* Add a minimum sample: if less than the current value,
* updates the value.
* <p>
* No-op if the minimum is unknown.
* </p>
* @param key statistics key
* @param value sample value
*/
void addMinimumSample(String key, long value);
/**
* Add a maximum sample: if greater than the current value,
* updates the value.
* <p>
* No-op if the key is unknown.
* </p>
* @param key statistics key
* @param value sample value
*/
void addMaximumSample(String key, long value);
/**
* Set a mean statistic to a given value.
* <p>
* No-op if the key is unknown.
* </p>
* @param key statistic key
* @param value new value.
*/
void setMeanStatistic(String key, MeanStatistic value);
/**
* Add a sample to the mean statistics.
* <p>
* No-op if the key is unknown.
* </p>
* @param key key
* @param value sample value.
*/
void addMeanStatisticSample(String key, long value);
/**
* Reset all statistics.
* Unsynchronized.
*/
void reset();
/**
* Get a reference to the atomic instance providing the
* value for a specific counter. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
AtomicLong getCounterReference(String key);
/**
* Get a reference to the atomic instance providing the
* value for a specific maximum. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
AtomicLong getMaximumReference(String key);
/**
* Get a reference to the atomic instance providing the
* value for a specific minimum. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
AtomicLong getMinimumReference(String key);
/**
* Get a reference to the atomic instance providing the
* value for a specific gauge. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
AtomicLong getGaugeReference(String key);
/**
* Get a reference to the atomic instance providing the
* value for a specific meanStatistic. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
MeanStatistic getMeanStatistic(String key);
/**
* Add a duration to the min/mean/max statistics, using the
* given prefix and adding a suffix for each specific value.
*
* The update is not-atomic, even though each individual statistic
* is updated thread-safely. If two threads update the values
* simultaneously, at the end of each operation the state will
* be correct. It is only during the sequence that the statistics
* may be observably inconsistent.
* @param prefix statistic prefix
* @param durationMillis duration in milliseconds.
*/
void addTimedOperation(String prefix, long durationMillis);
/**
* Add a duration to the min/mean/max statistics, using the
* given prefix and adding a suffix for each specific value.;
* increment tha counter whose name == prefix.
*
* If any of the statistics are not registered, that part of
* the sequence will be omitted -the rest will proceed.
*
* The update is not-atomic, even though each individual statistic
* is updated thread-safely. If two threads update the values
* simultaneously, at the end of each operation the state will
* be correct. It is only during the sequence that the statistics
* may be observably inconsistent.
* @param prefix statistic prefix
* @param duration duration
*/
void addTimedOperation(String prefix, Duration duration);
}

View File

@ -0,0 +1,75 @@
/*
* 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.statistics.impl;
/**
* Builder of the {@link IOStatisticsStore} implementation.
*/
public interface IOStatisticsStoreBuilder {
/**
* Declare a varargs list of counters to add.
* @param keys names of statistics.
* @return this builder.
*/
IOStatisticsStoreBuilder withCounters(String... keys);
/**
* Declare a varargs list of gauges to add.
* @param keys names of statistics.
* @return this builder.
*/
IOStatisticsStoreBuilder withGauges(String... keys);
/**
* Declare a varargs list of maximums to add.
* @param keys names of statistics.
* @return this builder.
*/
IOStatisticsStoreBuilder withMaximums(String... keys);
/**
* Declare a varargs list of minimums to add.
* @param keys names of statistics.
* @return this builder.
*/
IOStatisticsStoreBuilder withMinimums(String... keys);
/**
* Declare a varargs list of means to add.
* @param keys names of statistics.
* @return this builder.
*/
IOStatisticsStoreBuilder withMeanStatistics(String... keys);
/**
* Add a statistic in the counter, min, max and mean maps for each
* declared statistic prefix.
* @param prefixes prefixes for the stats.
* @return this
*/
IOStatisticsStoreBuilder withDurationTracking(
String... prefixes);
/**
* Build the collector.
* @return a new collector.
*/
IOStatisticsStore build();
}

View File

@ -0,0 +1,100 @@
/*
* 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.statistics.impl;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
/**
* Builder for an IOStatistics store..
*/
final class IOStatisticsStoreBuilderImpl implements
IOStatisticsStoreBuilder {
private final List<String> counters = new ArrayList<>();
private final List<String> gauges = new ArrayList<>();
private final List<String> minimums = new ArrayList<>();
private final List<String> maximums = new ArrayList<>();
private final List<String> meanStatistics = new ArrayList<>();
@Override
public IOStatisticsStoreBuilderImpl withCounters(final String... keys) {
counters.addAll(Arrays.asList(keys));
return this;
}
@Override
public IOStatisticsStoreBuilderImpl withGauges(final String... keys) {
gauges.addAll(Arrays.asList(keys));
return this;
}
@Override
public IOStatisticsStoreBuilderImpl withMaximums(final String... keys) {
maximums.addAll(Arrays.asList(keys));
return this;
}
@Override
public IOStatisticsStoreBuilderImpl withMinimums(final String... keys) {
minimums.addAll(Arrays.asList(keys));
return this;
}
@Override
public IOStatisticsStoreBuilderImpl withMeanStatistics(
final String... keys) {
meanStatistics.addAll(Arrays.asList(keys));
return this;
}
@Override
public IOStatisticsStoreBuilderImpl withDurationTracking(
final String... prefixes) {
for (String p : prefixes) {
withCounters(p, p + SUFFIX_FAILURES);
withMinimums(
p + SUFFIX_MIN,
p + SUFFIX_FAILURES + SUFFIX_MIN);
withMaximums(
p + SUFFIX_MAX,
p + SUFFIX_FAILURES + SUFFIX_MAX);
withMeanStatistics(
p + SUFFIX_MEAN,
p + SUFFIX_FAILURES + SUFFIX_MEAN);
}
return this;
}
@Override
public IOStatisticsStore build() {
return new IOStatisticsStoreImpl(counters, gauges, minimums,
maximums, meanStatistics);
}
}

View File

@ -0,0 +1,469 @@
/*
* 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.statistics.impl;
import javax.annotation.Nullable;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.MeanStatistic;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaximums;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMinimums;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.maybeUpdateMaximum;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.maybeUpdateMinimum;
/**
* Implementation of {@link IOStatisticsStore}.
* <p>
* A ConcurrentHashMap of each set of statistics is created;
* the AtomicLong/MeanStatistic entries are fetched as required.
* When the statistics are updated, the referenced objects
* are updated rather than new values set in the map.
* </p>
*/
final class IOStatisticsStoreImpl extends WrappedIOStatistics
implements IOStatisticsStore {
/**
* Log changes at debug.
* Noisy, but occasionally useful.
*/
private static final Logger LOG =
LoggerFactory.getLogger(IOStatisticsStoreImpl.class);
/** All the counters are atomic longs. */
private final Map<String, AtomicLong> counterMap = new ConcurrentHashMap<>();
/** All the gauges are atomic longs. */
private final Map<String, AtomicLong> gaugeMap = new ConcurrentHashMap<>();
/** All the minimum values are atomic longs. */
private final Map<String, AtomicLong> minimumMap = new ConcurrentHashMap<>();
/** All the maximum values are atomic longs. */
private final Map<String, AtomicLong> maximumMap = new ConcurrentHashMap<>();
/**
* The mean statistics.
* Relies on the MeanStatistic operations being synchronized.
*/
private final Map<String, MeanStatistic> meanStatisticMap
= new ConcurrentHashMap<>();
/**
* Constructor invoked via the builder.
* @param counters keys to use for the counter statistics.
* @param gauges names of gauges
* @param minimums names of minimums
* @param maximums names of maximums
* @param meanStatistics names of mean statistics.
*/
IOStatisticsStoreImpl(
final List<String> counters,
final List<String> gauges,
final List<String> minimums,
final List<String> maximums,
final List<String> meanStatistics) {
// initially create the superclass with no wrapped mapping;
super(null);
// now construct a dynamic statistics source mapping to
// the various counters, gauges etc dynamically created
// into maps
DynamicIOStatisticsBuilder builder = dynamicIOStatistics();
if (counters != null) {
for (String key : counters) {
AtomicLong counter = new AtomicLong();
counterMap.put(key, counter);
builder.withAtomicLongCounter(key, counter);
}
}
if (gauges != null) {
for (String key : gauges) {
AtomicLong gauge = new AtomicLong();
gaugeMap.put(key, gauge);
builder.withAtomicLongGauge(key, gauge);
}
}
if (maximums != null) {
for (String key : maximums) {
AtomicLong maximum = new AtomicLong(MAX_UNSET_VALUE);
maximumMap.put(key, maximum);
builder.withAtomicLongMaximum(key, maximum);
}
}
if (minimums != null) {
for (String key : minimums) {
AtomicLong minimum = new AtomicLong(MIN_UNSET_VALUE);
minimumMap.put(key, minimum);
builder.withAtomicLongMinimum(key, minimum);
}
}
if (meanStatistics != null) {
for (String key : meanStatistics) {
meanStatisticMap.put(key, new MeanStatistic());
builder.withMeanStatisticFunction(key, k -> meanStatisticMap.get(k));
}
}
setWrapped(builder.build());
}
/**
* Set an atomic long to a value.
* @param aLong atomic long; may be null
* @param value value to set to
*/
private void setAtomicLong(final AtomicLong aLong, final long value) {
if (aLong != null) {
aLong.set(value);
}
}
/**
* increment an atomic long and return its value;
* null long is no-op returning 0.
* @param aLong atomic long; may be null
* param increment amount to increment; negative for a decrement
* @return final value or 0 if the long is null
*/
private long incAtomicLong(final AtomicLong aLong,
final long increment) {
if (aLong != null) {
// optimization: zero is a get rather than addAndGet()
return increment != 0
? aLong.addAndGet(increment)
: aLong.get();
} else {
return 0;
}
}
@Override
public void setCounter(final String key, final long value) {
setAtomicLong(counterMap.get(key), value);
LOG.debug("Setting counter {} to {}", key, value);
}
@Override
public long incrementCounter(final String key, final long value) {
AtomicLong counter = counterMap.get(key);
if (counter == null) {
LOG.debug("Ignoring counter increment for unknown counter {}",
key);
return 0;
}
if (value < 0) {
LOG.debug("Ignoring negative increment value {} for counter {}",
value, key);
// returns old value
return counter.get();
} else {
long l = incAtomicLong(counter, value);
LOG.debug("Incrementing counter {} by {} with final value {}",
key, value, l);
return l;
}
}
@Override
public void setMaximum(final String key, final long value) {
setAtomicLong(maximumMap.get(key), value);
}
@Override
public long incrementMaximum(final String key, final long value) {
return incAtomicLong(maximumMap.get(key), value);
}
@Override
public void setMinimum(final String key, final long value) {
setAtomicLong(minimumMap.get(key), value);
}
@Override
public long incrementMinimum(final String key, final long value) {
return incAtomicLong(minimumMap.get(key), value);
}
@Override
public void addMinimumSample(final String key, final long value) {
AtomicLong min = minimumMap.get(key);
if (min != null) {
maybeUpdateMinimum(min, value);
}
}
@Override
public void addMaximumSample(final String key, final long value) {
AtomicLong max = maximumMap.get(key);
if (max != null) {
maybeUpdateMaximum(max, value);
}
}
@Override
public void setGauge(final String key, final long value) {
setAtomicLong(gaugeMap.get(key), value);
}
@Override
public long incrementGauge(final String key, final long value) {
return incAtomicLong(gaugeMap.get(key), value);
}
@Override
public void setMeanStatistic(final String key, final MeanStatistic value) {
final MeanStatistic ref = meanStatisticMap.get(key);
if (ref != null) {
ref.set(value);
}
}
@Override
public void addMeanStatisticSample(final String key, final long value) {
final MeanStatistic ref = meanStatisticMap.get(key);
if (ref != null) {
ref.addSample(value);
}
}
/**
* Reset all statistics.
*/
@Override
public synchronized void reset() {
counterMap.values().forEach(a -> a.set(0));
gaugeMap.values().forEach(a -> a.set(0));
minimumMap.values().forEach(a -> a.set(0));
maximumMap.values().forEach(a -> a.set(0));
meanStatisticMap.values().forEach(a -> a.clear());
}
/**
* Aggregate those statistics which the store is tracking;
* ignore the rest.
*
* @param source statistics; may be null
* @return true if a statistics reference was supplied/aggregated.
*/
@Override
public synchronized boolean aggregate(
@Nullable final IOStatistics source) {
if (source == null) {
return false;
}
// counters: addition
Map<String, Long> sourceCounters = source.counters();
counterMap.entrySet().
forEach(e -> {
Long sourceValue = lookupQuietly(sourceCounters, e.getKey());
if (sourceValue != null) {
e.getValue().addAndGet(sourceValue);
}
});
// gauge: add positive values only
Map<String, Long> sourceGauges = source.gauges();
gaugeMap.entrySet().forEach(e -> {
Long sourceGauge = lookupQuietly(sourceGauges, e.getKey());
if (sourceGauge != null && sourceGauge > 0) {
e.getValue().addAndGet(sourceGauge);
}
});
// min: min of current and source
Map<String, Long> sourceMinimums = source.minimums();
minimumMap.entrySet().forEach(e -> {
Long sourceValue = lookupQuietly(sourceMinimums, e.getKey());
if (sourceValue != null) {
AtomicLong dest = e.getValue();
dest.set(aggregateMaximums(dest.get(), sourceValue));
dest.set(aggregateMinimums(dest.get(), sourceValue));
}
});
// max: max of current and source
Map<String, Long> sourceMaximums = source.maximums();
maximumMap.entrySet().forEach(e -> {
Long sourceValue = lookupQuietly(sourceMaximums, e.getKey());
if (sourceValue != null) {
AtomicLong dest = e.getValue();
dest.set(aggregateMaximums(dest.get(), sourceValue));
}
});
// the most complex
Map<String, MeanStatistic> sourceMeans = source.meanStatistics();
meanStatisticMap.entrySet().forEach(e -> {
MeanStatistic current = e.getValue();
MeanStatistic sourceValue = lookupQuietly(
sourceMeans, e.getKey());
if (sourceValue != null) {
current.add(sourceValue);
}
});
return true;
}
/**
* Get a reference to the map type providing the
* value for a specific key, raising an exception if
* there is no entry for that key.
* @param <T> type of map/return type.
* @param map map to look up
* @param key statistic name
* @return the value
* @throws NullPointerException if there is no entry of that name
*/
private static <T> T lookup(final Map<String, T> map, String key) {
T val = map.get(key);
requireNonNull(val, () -> ("unknown statistic " + key));
return val;
}
/**
* Get a reference to the map type providing the
* value for a specific key, returning null if it not found.
* @param <T> type of map/return type.
* @param map map to look up
* @param key statistic name
* @return the value
*/
private static <T> T lookupQuietly(final Map<String, T> map, String key) {
return map.get(key);
}
/**
* Get a reference to the atomic instance providing the
* value for a specific counter. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
@Override
public AtomicLong getCounterReference(String key) {
return lookup(counterMap, key);
}
/**
* Get a reference to the atomic instance providing the
* value for a specific maximum. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
@Override
public AtomicLong getMaximumReference(String key) {
return lookup(maximumMap, key);
}
/**
* Get a reference to the atomic instance providing the
* value for a specific minimum. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
@Override
public AtomicLong getMinimumReference(String key) {
return lookup(minimumMap, key);
}
/**
* Get a reference to the atomic instance providing the
* value for a specific gauge. This is useful if
* the value is passed around.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
@Override
public AtomicLong getGaugeReference(String key) {
return lookup(gaugeMap, key);
}
/**
* Get a mean statistic.
* @param key statistic name
* @return the reference
* @throws NullPointerException if there is no entry of that name
*/
@Override
public MeanStatistic getMeanStatistic(String key) {
return lookup(meanStatisticMap, key);
}
/**
* Add a duration to the min/mean/max statistics, using the
* given prefix and adding a suffix for each specific value.
* <p>
* The update is non -atomic, even though each individual statistic
* is updated thread-safely. If two threads update the values
* simultaneously, at the end of each operation the state will
* be correct. It is only during the sequence that the statistics
* may be observably inconsistent.
* </p>
* @param prefix statistic prefix
* @param durationMillis duration in milliseconds.
*/
@Override
public void addTimedOperation(String prefix, long durationMillis) {
addMeanStatisticSample(prefix + SUFFIX_MEAN, durationMillis);
addMinimumSample(prefix + SUFFIX_MIN, durationMillis);
addMaximumSample(prefix + SUFFIX_MAX, durationMillis);
}
@Override
public void addTimedOperation(String prefix, Duration duration) {
addTimedOperation(prefix, duration.toMillis());
}
/**
* If the store is tracking the given key, return the
* duration tracker for it. If not tracked, return the
* stub tracker.
* @param key statistic key prefix
* @param count #of times to increment the matching counter in this
* operation.
* @return a tracker.
*/
@Override
public DurationTracker trackDuration(final String key, final long count) {
if (counterMap.containsKey(key)) {
return new StatisticDurationTracker(this, key, count);
} else {
return stubDurationTracker();
}
}
}

View File

@ -0,0 +1,93 @@
/*
* 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.statistics.impl;
import java.time.Duration;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
/**
* A duration tracker factory which aggregates two other trackers
* to have the same lifecycle.
*
* This is to ease having instance-level tracking alongside global
* values, such as an input stream and a filesystem.
*
* It's got some inefficiencies -assuming system time is used for
* the tracking, System.currentTimeMillis will be invoked twice
* at each point of the process -and the results may actually be different.
* However, it enables multiple duration tracker factories to be given the
* opportunity to collect the statistics.
*/
final class PairedDurationTrackerFactory implements DurationTrackerFactory {
private final DurationTrackerFactory local;
private final DurationTrackerFactory global;
PairedDurationTrackerFactory(final DurationTrackerFactory local,
final DurationTrackerFactory global) {
this.local = local;
this.global = global;
}
@Override
public DurationTracker trackDuration(final String key, final long count) {
return new PairedDurationTracker(
global.trackDuration(key, count),
local.trackDuration(key, count));
}
/**
* Tracker which wraps the two duration trackers created for the operation.
*/
private static final class PairedDurationTracker
implements DurationTracker {
private final DurationTracker firstDuration;
private final DurationTracker secondDuration;
private PairedDurationTracker(
final DurationTracker firstDuration,
final DurationTracker secondDuration) {
this.firstDuration = firstDuration;
this.secondDuration = secondDuration;
}
@Override
public void failed() {
firstDuration.failed();
secondDuration.failed();
}
@Override
public void close() {
firstDuration.close();
secondDuration.close();
}
/**
* @return the global duration
*/
@Override
public Duration asDuration() {
return firstDuration.asDuration();
}
}
}

View File

@ -0,0 +1,44 @@
/*
* 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.statistics.impl;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
/**
* Wrap a statistics instance with an {@link IOStatisticsSource}
* instance which will then serve up the statistics when asked.
*/
public class SourceWrappedStatistics implements IOStatisticsSource {
private final IOStatistics source;
/**
* Constructor.
* @param source source of statistics.
*/
public SourceWrappedStatistics(final IOStatistics source) {
this.source = source;
}
@Override
public IOStatistics getIOStatistics() {
return source;
}
}

View File

@ -0,0 +1,106 @@
/*
* 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.statistics.impl;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import org.apache.hadoop.util.OperationDuration;
/**
* Track the duration of an object.
*
* When closed the
* min/max/mean statistics are updated.
*
* In the constructor, the counter with name of 'key' is
* incremented -default is by 1, but can be set to other
* values, including 0.
*/
public class StatisticDurationTracker extends OperationDuration
implements DurationTracker {
/**
* Statistics to update.
*/
private final IOStatisticsStore iostats;
/**
* Key to use as prefix of values.
*/
private final String key;
/**
* Flag to indicate the operation failed.
*/
private boolean failed;
/**
* Constructor -increments the counter by 1.
* @param iostats statistics to update
* @param key prefix of values.
*/
public StatisticDurationTracker(
final IOStatisticsStore iostats,
final String key) {
this(iostats, key, 1);
}
/**
* Constructor.
* If the supplied count is greater than zero, the counter
* of the key name is updated.
* @param iostats statistics to update
* @param key Key to use as prefix of values.
* @param count #of times to increment the matching counter.
*/
public StatisticDurationTracker(
final IOStatisticsStore iostats,
final String key,
final long count) {
this.iostats = iostats;
this.key = key;
if (count > 0) {
iostats.incrementCounter(key, count);
}
}
@Override
public void failed() {
failed = true;
}
/**
* Set the finished time and then update the statistics.
* If the operation failed then the key + .failures counter will be
* incremented by one.
* The operation min/mean/max values will be updated with the duration;
* on a failure these will all be the .failures metrics.
*/
@Override
public void close() {
finished();
String name = key;
if (failed) {
// failure:
name = key + StoreStatisticNames.SUFFIX_FAILURES;
iostats.incrementCounter(name);
}
iostats.addTimedOperation(name, asDuration());
}
}

View File

@ -0,0 +1,98 @@
/*
* 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.statistics.impl;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.StorageStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
/**
* Returns all the counters of an IOStatistics instance as StorageStatistics.
* This is dynamic.
* The {@link #reset()} is downgraded to a no-op.
*/
public class StorageStatisticsFromIOStatistics
extends StorageStatistics
implements Iterable<StorageStatistics.LongStatistic> {
private final IOStatistics ioStatistics;
private final String scheme;
/**
* Instantiate.
* @param name storage statistics name.
* @param scheme FS scheme; may be null.
* @param ioStatistics IOStatistics source.
*/
public StorageStatisticsFromIOStatistics(
final String name,
final String scheme,
final IOStatistics ioStatistics) {
super(name);
this.scheme = scheme;
this.ioStatistics = ioStatistics;
}
@Override
public Iterator<LongStatistic> iterator() {
return getLongStatistics();
}
/**
* Take a snapshot of the current counter values
* and return an iterator over them.
* @return all the counter statistics.
*/
@Override
public Iterator<LongStatistic> getLongStatistics() {
final Set<Map.Entry<String, Long>> counters = counters()
.entrySet();
return counters.stream().map(e ->
new StorageStatistics.LongStatistic(e.getKey(), e.getValue()))
.collect(Collectors.toSet()).iterator();
}
private Map<String, Long> counters() {
return ioStatistics.counters();
}
@Override
public Long getLong(final String key) {
return counters().get(key);
}
@Override
public boolean isTracked(final String key) {
return counters().containsKey(key);
}
@Override
public void reset() {
/* no-op */
}
@Override
public String getScheme() {
return scheme;
}
}

View File

@ -0,0 +1,51 @@
/*
* 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.statistics.impl;
import java.time.Duration;
import org.apache.hadoop.fs.statistics.DurationTracker;
/**
* A simple stub duration tracker which can be issued in interfaces
* and other places where full duration tracking is not implemented.
*/
public final class StubDurationTracker implements DurationTracker {
public static final DurationTracker STUB_DURATION_TRACKER =
new StubDurationTracker();
private StubDurationTracker() {
}
@Override
public void failed() {
}
@Override
public void close() {
}
@Override
public Duration asDuration() {
return Duration.ZERO;
}
}

View File

@ -0,0 +1,44 @@
/*
* 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.statistics.impl;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
/**
* This is a stub factory which always returns no-op duration
* trackers. Allows for code to always be handed a factory.
*/
public final class StubDurationTrackerFactory
implements DurationTrackerFactory {
/**
* Single instance.
*/
public static final StubDurationTrackerFactory STUB_DURATION_TRACKER_FACTORY
= new StubDurationTrackerFactory();
private StubDurationTrackerFactory() {
}
@Override
public DurationTracker trackDuration(final String key, final long count) {
return StubDurationTracker.STUB_DURATION_TRACKER;
}
}

View File

@ -0,0 +1,108 @@
/*
* 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.statistics.impl;
import java.util.Map;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.MeanStatistic;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
/**
* Wrap IOStatistics source with another (dynamic) wrapper.
*/
public class WrappedIOStatistics extends AbstractIOStatisticsImpl {
/**
* The wrapped statistics.
*/
private IOStatistics wrapped;
/**
* Instantiate.
* @param wrapped nullable wrapped statistics.
*/
public WrappedIOStatistics(final IOStatistics wrapped) {
this.wrapped = wrapped;
}
/**
* Instantiate without setting the statistics.
* This is for subclasses which build up the map during their own
* construction.
*/
protected WrappedIOStatistics() {
}
@Override
public Map<String, Long> counters() {
return getWrapped().counters();
}
/**
* Get at the wrapped inner statistics.
* @return the wrapped value
*/
protected IOStatistics getWrapped() {
return wrapped;
}
/**
* Set the wrapped statistics.
* Will fail if the field is already set.
* @param wrapped new value
*/
protected void setWrapped(final IOStatistics wrapped) {
Preconditions.checkState(this.wrapped == null,
"Attempted to overwrite existing wrapped statistics");
this.wrapped = wrapped;
}
@Override
public Map<String, Long> gauges() {
return getWrapped().gauges();
}
@Override
public Map<String, Long> minimums() {
return getWrapped().minimums();
}
@Override
public Map<String, Long> maximums() {
return getWrapped().maximums();
}
@Override
public Map<String, MeanStatistic> meanStatistics() {
return getWrapped().meanStatistics();
}
/**
* Return the statistics dump of the wrapped statistics.
* @return the statistics for logging.
*/
@Override
public String toString() {
return ioStatisticsToString(wrapped);
}
}

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.
*/
/**
* Implementation support for statistics.
* For use internally; external filesystems MAY use this if the implementors
* accept that it is unstable and that incompatible changes may take
* place over minor point releases.
*/
@InterfaceAudience.LimitedPrivate("Filesystems")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.statistics.impl;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,134 @@
/*
* 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.
*/
/**
* This package contains support for statistic collection and reporting.
* This is the public API; implementation classes are to be kept elsewhere.
* <p>
* This package defines two interfaces:
* <p>
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}:
* a source of statistic data, which can be retrieved
* through a call to
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSource#getIOStatistics()} .
* <p>
* {@link org.apache.hadoop.fs.statistics.IOStatistics} the statistics retrieved
* from a statistics source.
* <p>
* The retrieved statistics may be an immutable snapshot -in which case to get
* updated statistics another call to
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSource#getIOStatistics()}
* must be made. Or they may be dynamic -in which case every time a specific
* statistic is retrieved, the latest version is returned. Callers should assume
* that if a statistics instance is dynamic, there is no atomicity when querying
* multiple statistics. If the statistics source was a closeable object (e.g. a
* stream), the statistics MUST remain valid after the stream is closed.
* <p>
* Use pattern:
* <p>
* An application probes an object (filesystem, stream etc) to see if it
* implements {@code IOStatisticsSource}, and, if it is,
* calls {@code getIOStatistics()} to get its statistics.
* If this is non-null, the client has statistics on the current
* state of the statistics.
* <p>
* The expectation is that a statistics source is dynamic: when a value is
* looked up the most recent values are returned.
* When iterating through the set, the values of the iterator SHOULD
* be frozen at the time the iterator was requested.
* <p>
* These statistics can be used to: log operations, profile applications,
* and make assertions about the state of the output.
* <p>
* The names of statistics are a matter of choice of the specific source.
* However, {@link org.apache.hadoop.fs.statistics.StoreStatisticNames}
* contains a
* set of names recommended for object store operations.
* {@link org.apache.hadoop.fs.statistics.StreamStatisticNames} declares
* recommended names for statistics provided for
* input and output streams.
* <p>
* Utility classes:
* <ul>
* <li>
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSupport}.
* General support, including the ability to take a serializable
* snapshot of the current state of an IOStatistics instance.
* </li>
* <li>
* {@link org.apache.hadoop.fs.statistics.IOStatisticsLogging}.
* Methods for robust/on-demand string conversion, designed
* for use in logging statements and {@code toString()} implementations.
* </li>
* <li>
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSnapshot}.
* A static snaphot of statistics which can be marshalled via
* java serialization or as JSON via jackson. It supports
* aggregation, so can be used to generate aggregate statistics.
* </li>
* </ul>
*
* <p>
* Implementors notes:
* <ol>
* <li>
* IOStatistics keys SHOULD be standard names where possible.
* </li>
* <li>
* An IOStatistics instance MUST be unique to that specific instance of
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}.
* (i.e. not shared the way StorageStatistics are)
* </li>
* <li>
* MUST return the same values irrespective of which thread the statistics are
* retrieved or its keys evaluated.
* </li>
* <li>
* MUST NOT remove keys once a statistic instance has been created.
* </li>
* <li>
* MUST NOT add keys once a statistic instance has been created.
* </li>
* <li>
* MUST NOT block for long periods of time while blocking operations
* (reads, writes) are taking place in the source.
* That is: minimal synchronization points (AtomicLongs etc.) may be
* used to share values, but retrieval of statistics should
* be fast and return values even while slow/blocking remote IO is underway.
* </li>
* <li>
* MUST support value enumeration and retrieval after the source has been
* closed.
* </li>
* <li>
* SHOULD NOT have back-references to potentially expensive objects
* (filesystem instances etc.)
* </li>
* <li>
* SHOULD provide statistics which can be added to generate aggregate
* statistics.
* </li>
* </ol>
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
package org.apache.hadoop.fs.statistics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -25,6 +25,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.PositionedReadable;
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
/**
* A compression input stream.
*
@ -34,7 +38,8 @@ import org.apache.hadoop.fs.Seekable;
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class CompressionInputStream extends InputStream implements Seekable {
public abstract class CompressionInputStream extends InputStream
implements Seekable, IOStatisticsSource {
/**
* The input stream to be compressed.
*/
@ -68,7 +73,16 @@ public abstract class CompressionInputStream extends InputStream implements Seek
}
}
}
/**
* Return any IOStatistics provided by the underlying stream.
* @return IO stats from the inner stream.
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(in);
}
/**
* Read bytes from the stream.
* Made abstract to prevent leakage to underlying stream.

View File

@ -23,13 +23,17 @@ import java.io.OutputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
/**
* A compression output stream.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class CompressionOutputStream extends OutputStream {
public abstract class CompressionOutputStream extends OutputStream
implements IOStatisticsSource {
/**
* The output stream to be compressed.
*/
@ -94,4 +98,12 @@ public abstract class CompressionOutputStream extends OutputStream {
*/
public abstract void resetState() throws IOException;
/**
* Return any IOStatistics provided by the underlying stream.
* @return IO stats from the inner stream.
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(out);
}
}

View File

@ -25,6 +25,9 @@ import java.io.InputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.io.Text;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
@ -42,7 +45,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
*/
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
public class LineReader implements Closeable {
public class LineReader implements Closeable, IOStatisticsSource {
private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
private int bufferSize = DEFAULT_BUFFER_SIZE;
private InputStream in;
@ -148,7 +151,16 @@ public class LineReader implements Closeable {
public void close() throws IOException {
in.close();
}
/**
* Return any IOStatistics provided by the source.
* @return IO stats from the input stream.
*/
@Override
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(in);
}
/**
* Read one line from the InputStream into the given Text.
*

View File

@ -18,48 +18,98 @@
package org.apache.hadoop.util;
import java.time.Duration;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Little duration counter.
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Unstable
public class OperationDuration {
/**
* Time in millis when the operation started.
*/
private final long started;
/**
* Time when the operation finished.
*/
private long finished;
/**
* Instantiate.
* The start time and finished time are both set
* to the current clock time.
*/
public OperationDuration() {
started = time();
finished = started;
}
/**
* Evaluate the system time.
* @return the current clock time.
*/
protected long time() {
return System.currentTimeMillis();
}
/**
* Update the finished time with the current system time.
*/
public void finished() {
finished = time();
}
/**
* Return the duration as {@link #humanTime(long)}.
* @return a printable duration.
*/
public String getDurationString() {
return humanTime(value());
}
/**
* Convert to a human time of minutes:seconds.millis.
* @param time time to humanize.
* @return a printable value.
*/
public static String humanTime(long time) {
long seconds = (time / 1000);
long minutes = (seconds / 60);
return String.format("%d:%02d.%03ds", minutes, seconds % 60, time % 1000);
}
/**
* Return the duration as {@link #humanTime(long)}.
* @return a printable duration.
*/
@Override
public String toString() {
return getDurationString();
}
/**
* Get the duration in milliseconds.
* <p></p>
* This will be 0 until a call
* to {@link #finished()} has been made.
* @return the currently recorded duration.
*/
public long value() {
return finished -started;
}
/**
* Get the duration of an operation as a java Duration
* instance.
* @return a duration.
*/
public Duration asDuration() {
return Duration.ofMillis(value());
}
}

View File

@ -0,0 +1,40 @@
/*
* 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.functional;
import java.io.IOException;
/**
* Function of arity 2 which may raise an IOException.
* @param <T> type of arg1
* @param <U> type of arg2
* @param <R> type of return value.
*/
@FunctionalInterface
public interface BiFunctionRaisingIOE<T, U, R> {
/**
* Apply the function.
* @param t argument 1
* @param u argument 2
* @return result
* @throws IOException Any IO failure
*/
R apply(T t, U u) throws IOException;
}

View File

@ -0,0 +1,36 @@
/*
* 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.functional;
import java.io.IOException;
/**
* This is a callable which only raises an IOException.
* @param <R> return type
*/
@FunctionalInterface
public interface CallableRaisingIOE<R> {
/**
* Apply the operation.
* @return result
* @throws IOException Any IO failure
*/
R apply() throws IOException;
}

View File

@ -0,0 +1,51 @@
/*
* 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.functional;
import java.io.IOException;
/**
* Version of java.util.function.Consumer which raises
* exceptions.
* @param <T> type of argument,.
*/
@FunctionalInterface
public interface ConsumerRaisingIOE<T> {
/**
* Process the argument.
* @param t type
* @throws IOException if needed
*/
void accept(T t) throws IOException;
/**
* after calling {@link #accept(Object)},
* invoke the next consumer in the chain.
* @param next next consumer
* @return the chain.
*/
default ConsumerRaisingIOE<T> andThen(
ConsumerRaisingIOE<? super T> next) {
return (T t) -> {
accept(t);
next.accept(t);
};
}
}

View File

@ -0,0 +1,38 @@
/*
* 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.functional;
import java.io.IOException;
/**
* Function of arity 1 which may raise an IOException.
* @param <T> type of arg1
* @param <R> type of return value.
*/
@FunctionalInterface
public interface FunctionRaisingIOE<T, R> {
/**
* Apply the function.
* @param t argument 1
* @return result
* @throws IOException Any IO failure
*/
R apply(T t) throws IOException;
}

View File

@ -0,0 +1,188 @@
/*
* 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.functional;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.UncheckedIOException;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Future IO Helper methods.
* <p>
* Contains methods promoted from
* {@link org.apache.hadoop.fs.impl.FutureIOSupport} because they
* are a key part of integrating async IO in application code.
* </p>
* <p>
* One key feature is that the {@link #awaitFuture(Future)} and
* {@link #awaitFuture(Future, long, TimeUnit)} calls will
* extract and rethrow exceptions raised in the future's execution,
* including extracting the inner IOException of any
* {@code UncheckedIOException} raised in the future.
* This makes it somewhat easier to execute IOException-raising
* code inside futures.
* </p>
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class FutureIO {
private FutureIO() {
}
/**
* Given a future, evaluate it.
* <p>
* Any exception generated in the future is
* extracted and rethrown.
* </p>
* @param future future to evaluate
* @param <T> type of the result.
* @return the result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
public static <T> T awaitFuture(final Future<T> future)
throws InterruptedIOException, IOException, RuntimeException {
try {
return future.get();
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
} catch (ExecutionException e) {
return raiseInnerCause(e);
}
}
/**
* Given a future, evaluate it.
* <p>
* Any exception generated in the future is
* extracted and rethrown.
* </p>
* @param future future to evaluate
* @param <T> type of the result.
* @return the result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
* @throws TimeoutException the future timed out.
*/
public static <T> T awaitFuture(final Future<T> future,
final long timeout,
final TimeUnit unit)
throws InterruptedIOException, IOException, RuntimeException,
TimeoutException {
try {
return future.get(timeout, unit);
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
} catch (ExecutionException e) {
return raiseInnerCause(e);
}
}
/**
* From the inner cause of an execution exception, extract the inner cause
* if it is an IOE or RTE.
* This will always raise an exception, either the inner IOException,
* an inner RuntimeException, or a new IOException wrapping the raised
* exception.
*
* @param e exception.
* @param <T> type of return value.
* @return nothing, ever.
* @throws IOException either the inner IOException, or a wrapper around
* any non-Runtime-Exception
* @throws RuntimeException if that is the inner cause.
*/
public static <T> T raiseInnerCause(final ExecutionException e)
throws IOException {
throw unwrapInnerException(e);
}
/**
* Extract the cause of a completion failure and rethrow it if an IOE
* or RTE.
* @param e exception.
* @param <T> type of return value.
* @return nothing, ever.
* @throws IOException either the inner IOException, or a wrapper around
* any non-Runtime-Exception
* @throws RuntimeException if that is the inner cause.
*/
public static <T> T raiseInnerCause(final CompletionException e)
throws IOException {
throw unwrapInnerException(e);
}
/**
* From the inner cause of an execution exception, extract the inner cause
* to an IOException, raising RuntimeExceptions and Errors immediately.
* <ol>
* <li> If it is an IOE: Return.</li>
* <li> If it is a {@link UncheckedIOException}: return the cause</li>
* <li> Completion/Execution Exceptions: extract and repeat</li>
* <li> If it is an RTE or Error: throw.</li>
* <li> Any other type: wrap in an IOE</li>
* </ol>
*
* Recursively handles wrapped Execution and Completion Exceptions in
* case something very complicated has happened.
* @param e exception.
* @return an IOException extracted or built from the cause.
* @throws RuntimeException if that is the inner cause.
* @throws Error if that is the inner cause.
*/
@SuppressWarnings("ChainOfInstanceofChecks")
public static IOException unwrapInnerException(final Throwable e) {
Throwable cause = e.getCause();
if (cause instanceof IOException) {
return (IOException) cause;
} else if (cause instanceof UncheckedIOException) {
// this is always an IOException
return ((UncheckedIOException) cause).getCause();
} else if (cause instanceof CompletionException) {
return unwrapInnerException(cause);
} else if (cause instanceof ExecutionException) {
return unwrapInnerException(cause);
} else if (cause instanceof RuntimeException) {
throw (RuntimeException) cause;
} else if (cause instanceof Error) {
throw (Error) cause;
} else if (cause != null) {
// other type: wrap with a new IOE
return new IOException(cause);
} else {
// this only happens if there was no cause.
return new IOException(e);
}
}
}

View File

@ -0,0 +1,42 @@
/*
* 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.functional;
import java.io.IOException;
/**
* This is a lambda-expression which may raises an IOException.
* This is a recurrent design patten in the hadoop codebase, e.g
* {@code LambdaTestUtils.VoidCallable} and
* the S3A {@code Invoker.VoidOperation}}. Hopefully this should
* be the last.
* Note for implementors of methods which take this as an argument:
* don't use method overloading to determine which specific functional
* interface is to be used.
*/
@FunctionalInterface
public interface InvocationRaisingIOE {
/**
* Apply the operation.
* @throws IOException Any IO failure
*/
void apply() throws IOException;
}

View File

@ -0,0 +1,698 @@
/*
* 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.functional;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Objects;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.io.IOUtils;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* A set of remote iterators supporting transformation and filtering,
* with IOStatisticsSource passthrough, and of conversions of
* the iterators to lists/arrays and of performing actions
* on the values.
* <p></p>
* This aims to make it straightforward to use lambda-expressions to
* transform the results of an iterator, without losing the statistics
* in the process, and to chain the operations together.
* <p></p>
* The closeable operation will be passed through RemoteIterators which
* wrap other RemoteIterators. This is to support any iterator which
* can be closed to release held connections, file handles etc.
* Unless client code is written to assume that RemoteIterator instances
* may be closed, this is not likely to be broadly used. It is added
* to make it possible to adopt this feature in a managed way.
* <p></p>
* One notable feature is that the
* {@link #foreach(RemoteIterator, ConsumerRaisingIOE)} method will
* LOG at debug any IOStatistics provided by the iterator, if such
* statistics are provided. There's no attempt at retrieval and logging
* if the LOG is not set to debug, so it is a zero cost feature unless
* the logger {@code org.apache.hadoop.fs.functional.RemoteIterators}
* is at DEBUG.
* <p></p>
* Based on the S3A Listing code, and some some work on moving other code
* to using iterative listings so as to pick up the statistics.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class RemoteIterators {
/**
* Log used for logging any statistics in
* {@link #foreach(RemoteIterator, ConsumerRaisingIOE)}
* at DEBUG.
*/
private static final Logger LOG = LoggerFactory.getLogger(
RemoteIterators.class);
private RemoteIterators() {
}
/**
* Create an iterator from a singleton.
* @param singleton instance
* @param <T> type
* @return a remote iterator
*/
public static <T> RemoteIterator<T> remoteIteratorFromSingleton(
@Nullable T singleton) {
return new SingletonIterator<>(singleton);
}
/**
* Create a remote iterator from a java.util.Iterator.
* @param <T> type
* @return a remote iterator
*/
public static <T> RemoteIterator<T> remoteIteratorFromIterator(
Iterator<T> iterator) {
return new WrappedJavaIterator<>(iterator);
}
/**
* Create a remote iterator from a java.util.Iterable -e.g. a list
* or other collection.
* @param <T> type
* @return a remote iterator
*/
public static <T> RemoteIterator<T> remoteIteratorFromIterable(
Iterable<T> iterable) {
return new WrappedJavaIterator<>(iterable.iterator());
}
/**
* Create a remote iterator from an array.
* @param <T> type
* @return a remote iterator
*/
public static <T> RemoteIterator<T> remoteIteratorFromArray(T[] array) {
return new WrappedJavaIterator<>(Arrays.stream(array).iterator());
}
/**
* Create an iterator from an iterator and a transformation function.
* @param <S> source type
* @param <T> result type
* @param iterator source
* @param mapper transformation
* @return a remote iterator
*/
public static <S, T> RemoteIterator<T> mappingRemoteIterator(
RemoteIterator<S> iterator,
FunctionRaisingIOE<? super S, T> mapper) {
return new MappingRemoteIterator<>(iterator, mapper);
}
/**
* Create a RemoteIterator from a RemoteIterator, casting the
* type in the process. This is to help with filesystem API
* calls where overloading causes confusion (e.g. listStatusIterator())
* @param <S> source type
* @param <T> result type
* @param iterator source
* @return a remote iterator
*/
public static <S, T> RemoteIterator<T> typeCastingRemoteIterator(
RemoteIterator<S> iterator) {
return new TypeCastingRemoteIterator<>(iterator);
}
/**
* Create a RemoteIterator from a RemoteIterator and a filter
* function which returns true for every element to be passed
* through.
* <p></p>
* Elements are filtered in the hasNext() method; if not used
* the filtering will be done on demand in the {@code next()}
* call.
* @param <S> type
* @param iterator source
* @param filter filter
* @return a remote iterator
*/
public static <S> RemoteIterator<S> filteringRemoteIterator(
RemoteIterator<S> iterator,
FunctionRaisingIOE<? super S, Boolean> filter) {
return new FilteringRemoteIterator<>(iterator, filter);
}
/**
* This adds an extra close operation alongside the passthrough
* to any Closeable.close() method supported by the source iterator.
* @param iterator source
* @param toClose extra object to close.
* @param <S> source type.
* @return a new iterator
*/
public static <S> RemoteIterator<S> closingRemoteIterator(
RemoteIterator<S> iterator,
Closeable toClose) {
return new CloseRemoteIterator<>(iterator, toClose);
}
/**
* Build a list from a RemoteIterator.
* @param <T> type
* @return a list of the values.
* @throws IOException if the source RemoteIterator raises it.
*/
public static <T> List<T> toList(RemoteIterator<T> source)
throws IOException {
List<T> l = new ArrayList<>();
foreach(source, l::add);
return l;
}
/**
* Build an array from a RemoteIterator.
* @param <T> type
* @return an array of the values.
* @throws IOException if the source RemoteIterator raises it.
*/
public static <T> T[] toArray(RemoteIterator<T> source) throws IOException {
return (T[]) toList(source).toArray();
}
/**
* Apply an operation to all values of a RemoteIterator.
* <p></p>
* If the iterator is an IOStatisticsSource returning a non-null
* set of statistics, <i>and</i> this classes log is set to DEBUG,
* then the statistics of the operation are evaluated and logged at
* debug.
* <p></p>
* The number of entries processed is returned, as it is useful to
* know this, especially during tests or when reporting values
* to users.
* <p></p>
* This does not close the iterator afterwards.
* @param source iterator source
* @param consumer consumer of the values.
* @return the number of elements processed
* @param <T> type of source
* @throws IOException if the source RemoteIterator or the consumer raise one.
*/
public static <T> long foreach(
RemoteIterator<T> source,
ConsumerRaisingIOE<? super T> consumer) throws IOException {
long count = 0;
try {
while (source.hasNext()) {
count++;
consumer.accept(source.next());
}
// maybe log the results
logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source);
} finally {
if (source instanceof Closeable) {
// source is closeable, so close.
IOUtils.cleanupWithLogger(LOG, (Closeable) source);
}
}
return count;
}
/**
* A remote iterator from a singleton. It has a single next()
* value, after which hasNext() returns false and next() fails.
* <p></p>
* If it is a source of
* remote statistics, these are returned.
* @param <T> type.
*/
private static final class SingletonIterator<T>
implements RemoteIterator<T>, IOStatisticsSource {
/**
* Single entry.
*/
private final T singleton;
/** Has the entry been processed? */
private boolean processed;
/**
* Instantiate.
* @param singleton single value...may be null
*/
private SingletonIterator(@Nullable T singleton) {
this.singleton = singleton;
// if the entry is null, consider it processed.
this.processed = singleton == null;
}
@Override
public boolean hasNext() throws IOException {
return !processed;
}
@SuppressWarnings("NewExceptionWithoutArguments")
@Override
public T next() throws IOException {
if (hasNext()) {
processed = true;
return singleton;
} else {
throw new NoSuchElementException();
}
}
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(singleton);
}
@Override
public String toString() {
return "SingletonIterator{"
+ (singleton != null ? singleton : "")
+ '}';
}
}
/**
* Create a remote iterator from a simple java.util.Iterator, or
* an iterable.
* <p> </p>
* If the iterator is a source of statistics that is passed through.
* <p></p>
* The {@link #close()} will close the source iterator if it is
* Closeable;
* @param <T> iterator type.
*/
private static final class WrappedJavaIterator<T>
implements RemoteIterator<T>, IOStatisticsSource, Closeable {
/**
* inner iterator..
*/
private final Iterator<? extends T> source;
private final Closeable sourceToClose;
/**
* Construct from an interator.
* @param source source iterator.
*/
private WrappedJavaIterator(Iterator<? extends T> source) {
this.source = requireNonNull(source);
sourceToClose = new MaybeClose(source);
}
@Override
public boolean hasNext() {
return source.hasNext();
}
@Override
public T next() {
return source.next();
}
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(source);
}
@Override
public String toString() {
return "FromIterator{" + source + '}';
}
@Override
public void close() throws IOException {
sourceToClose.close();
}
}
/**
* Wrapper of another remote iterator; IOStatistics
* and Closeable methods are passed down if implemented.
* @param <S> source type
* @param <T> type of returned value
*/
private static abstract class WrappingRemoteIterator<S, T>
implements RemoteIterator<T>, IOStatisticsSource, Closeable {
/**
* Source iterator.
*/
private final RemoteIterator<S> source;
private final Closeable sourceToClose;
protected WrappingRemoteIterator(final RemoteIterator<S> source) {
this.source = requireNonNull(source);
sourceToClose = new MaybeClose(source);
}
protected RemoteIterator<S> getSource() {
return source;
}
@Override
public IOStatistics getIOStatistics() {
return retrieveIOStatistics(source);
}
@Override
public void close() throws IOException {
sourceToClose.close();
}
/**
* Check for the source having a next element.
* If it does not, this object's close() method
* is called and false returned
* @return true if there is a new value
* @throws IOException failure to retrieve next value
*/
protected boolean sourceHasNext() throws IOException {
boolean hasNext;
try {
hasNext = getSource().hasNext();
} catch (IOException e) {
IOUtils.cleanupWithLogger(LOG, this);
throw e;
}
if (!hasNext) {
// there is nothing less so automatically close.
close();
}
return hasNext;
}
/**
* Get the next source value.
* This calls {@link #sourceHasNext()} first to verify
* that there is data.
* @return the next value
* @throws IOException failure
* @throws NoSuchElementException no more data
*/
protected S sourceNext() throws IOException {
try {
if (!sourceHasNext()) {
throw new NoSuchElementException();
}
return getSource().next();
} catch (NoSuchElementException | IOException e) {
IOUtils.cleanupWithLogger(LOG, this);
throw e;
}
}
@Override
public String toString() {
return source.toString();
}
}
/**
* Iterator taking a source and a transformational function.
* @param <S> source type
* @param <T> final output type.There
*/
private static final class MappingRemoteIterator<S, T>
extends WrappingRemoteIterator<S, T> {
/**
* Mapper to invoke.
*/
private final FunctionRaisingIOE<? super S, T> mapper;
private MappingRemoteIterator(
RemoteIterator<S> source,
FunctionRaisingIOE<? super S, T> mapper) {
super(source);
this.mapper = requireNonNull(mapper);
}
@Override
public boolean hasNext() throws IOException {
return sourceHasNext();
}
@Override
public T next() throws IOException {
return mapper.apply(sourceNext());
}
@Override
public String toString() {
return "FunctionRemoteIterator{" + getSource() + '}';
}
}
/**
* RemoteIterator which can change the type of the input.
* This is useful in some situations.
* @param <S> source type
* @param <T> final output type.
*/
private static final class TypeCastingRemoteIterator<S, T>
extends WrappingRemoteIterator<S, T> {
private TypeCastingRemoteIterator(
RemoteIterator<S> source) {
super(source);
}
@Override
public boolean hasNext() throws IOException {
return sourceHasNext();
}
@Override
public T next() throws IOException {
return (T)sourceNext();
}
@Override
public String toString() {
return getSource().toString();
}
}
/**
* Extend the wrapped iterator by filtering source values out.
* Only those values for which the filter predicate returns true
* will be returned.
* @param <S> type of iterator.
*/
@SuppressWarnings("NewExceptionWithoutArguments")
private static final class FilteringRemoteIterator<S>
extends WrappingRemoteIterator<S, S> {
/**
* Filter Predicate.
* Takes the input type or any superclass.
*/
private final FunctionRaisingIOE<? super S, Boolean>
filter;
/**
* Next value; will be null if none has been evaluated, or the
* last one was already returned by next().
*/
private S next;
/**
* An iterator which combines filtering with transformation.
* All source elements for which filter = true are returned,
* transformed via the mapper.
* @param source source iterator.
* @param filter filter predicate.
*/
private FilteringRemoteIterator(
RemoteIterator<S> source,
FunctionRaisingIOE<? super S, Boolean> filter) {
super(source);
this.filter = requireNonNull(filter);
}
/**
* Fetch: retrieve the next value.
* @return true if a new value was found after filtering.
* @throws IOException failure in retrieval from source or mapping
*/
private boolean fetch() throws IOException {
while (next == null && sourceHasNext()) {
S candidate = getSource().next();
if (filter.apply(candidate)) {
next = candidate;
return true;
}
}
return false;
}
/**
* Trigger a fetch if an entry is needed.
* @return true if there was already an entry return,
* or there was not but one could then be retrieved.set
* @throws IOException failure in fetch operation
*/
@Override
public boolean hasNext() throws IOException {
if (next != null) {
return true;
}
return fetch();
}
/**
* Return the next value.
* Will retrieve the next elements if needed.
* This is where the mapper takes place.
* @return true if there is another data element.
* @throws IOException failure in fetch operation or the transformation.
* @throws NoSuchElementException no more data
*/
@Override
public S next() throws IOException {
if (hasNext()) {
S result = next;
next = null;
return result;
}
throw new NoSuchElementException();
}
@Override
public String toString() {
return "FilteringRemoteIterator{" + getSource() + '}';
}
}
/**
* A wrapping remote iterator which adds another entry to
* close. This is to assist cleanup.
* @param <S> type
*/
private static final class CloseRemoteIterator<S>
extends WrappingRemoteIterator<S, S> {
private final MaybeClose toClose;
private boolean closed;
private CloseRemoteIterator(
final RemoteIterator<S> source,
final Closeable toClose) {
super(source);
this.toClose = new MaybeClose(Objects.requireNonNull(toClose));
}
@Override
public boolean hasNext() throws IOException {
return sourceHasNext();
}
@Override
public S next() throws IOException {
return sourceNext();
}
@Override
public void close() throws IOException {
if (closed) {
return;
}
closed = true;
LOG.debug("Closing {}", this);
try {
super.close();
} finally {
toClose.close();
}
}
}
/**
* Class to help with Closeable logic, where sources may/may not
* be closeable, only one invocation is allowed.
* On the second and later call of close(), it is a no-op.
*/
private static final class MaybeClose implements Closeable {
private Closeable toClose;
/**
* Construct.
* @param o object to close.
*/
private MaybeClose(Object o) {
this(o, true);
}
/**
* Construct -close the object if it is closeable and close==true.
* @param o object to close.
* @param close should close?
*/
private MaybeClose(Object o, boolean close) {
if (close && o instanceof Closeable) {
this.toClose = (Closeable) o;
} else {
this.toClose = null;
}
}
@Override
public void close() throws IOException {
if (toClose != null) {
try {
toClose.close();
} finally {
toClose = null;
}
}
}
}
}

View File

@ -0,0 +1,41 @@
/*
* 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 functional programming within the Hadoop APIs.
* <p></p>
* Much of this is needed simply to cope with Java's checked exceptions and
* the fact that the java.util.function can only throw runtime exceptions.
* <p></p>
* Pretty much all the Hadoop FS APIs raise IOExceptions, hence the need
* for these classes. If Java had made a different decision about the
* nature of exceptions, life would be better.
* <p></p>
* Do note that the {@link org.apache.hadoop.util.functional.RemoteIterators}
* iterators go beyond that of the java ones, in terms of declaring themselves
* Closeable and implementors of
* {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}; a chain
* of wrapped iterators can supply statistics of the inner iterators, and
* encourage close() to be called after use.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
package org.apache.hadoop.util.functional;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -38,3 +38,4 @@ HDFS as these are commonly expected by Hadoop client applications.
2. [Testing with the Filesystem specification](testing.html)
2. [Extending the specification and its tests](extending.html)
1. [Uploading a file using Multiple Parts](multipartuploader.html)
1. [IOStatistics](iostatistics.html)

View File

@ -0,0 +1,432 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# Statistic collection with the IOStatistics API
```java
@InterfaceAudience.Public
@InterfaceStability.Unstable
```
The `IOStatistics` API is intended to provide statistics on individual IO
classes -such as input and output streams, *in a standard way which
applications can query*
Many filesystem-related classes have implemented statistics gathering
and provided private/unstable ways to query this, but as they were
not common across implementations it was unsafe for applications
to reference these values. Example: `S3AInputStream` and its statistics
API. This is used in internal tests, but cannot be used downstream in
applications such as Apache Hive or Apache HBase.
The IOStatistics API is intended to
1. Be instance specific:, rather than shared across multiple instances
of a class, or thread local.
1. Be public and stable enough to be used by applications.
1. Be easy to use in applications written in Java, Scala, and, via libhdfs, C/C++
1. Have foundational interfaces and classes in the `hadoop-common` JAR.
## Core Model
Any class *may* implement `IOStatisticsSource` in order to
provide statistics.
Wrapper I/O Classes such as `FSDataInputStream` anc `FSDataOutputStream` *should*
implement the interface and forward it to the wrapped class, if they also
implement it -and return `null` if they do not.
`IOStatisticsSource` implementations `getIOStatistics()` return an
instance of `IOStatistics` enumerating the statistics of that specific
instance.
The `IOStatistics` Interface exports five kinds of statistic:
| Category | Type | Description |
|------|------|-------------|
| `counter` | `long` | a counter which may increase in value; SHOULD BE >= 0 |
| `gauge` | `long` | an arbitrary value which can down as well as up; SHOULD BE >= 0 |
| `minimum` | `long` | an minimum value; MAY BE negative |
| `maximum` | `long` | a maximum value; MAY BE negative |
| `meanStatistic` | `MeanStatistic` | an arithmetic mean and sample size; mean MAY BE negative |
Four are simple `long` values, with the variations how they are likely to
change and how they are aggregated.
#### Aggregation of Statistic Values
For the different statistic category, the result of `aggregate(x, y)` is
| Category | Aggregation |
|------------------|-------------|
| `counter` | `max(0, x) + max(0, y)` |
| `gauge` | `max(0, x) + max(0, y)` |
| `minimum` | `min(x, y)` |
| `maximum` | `max(x, y)` |
| `meanStatistic` | calculation of the mean of `x` and `y` ) |
#### Class `MeanStatistic`
## package `org.apache.hadoop.fs.statistics`
This package contains the public statistics APIs intended
for use by applications.
<!-- ============================================================= -->
<!-- Class: MeanStatistic -->
<!-- ============================================================= -->
`MeanStatistic` is a tuple of `(mean, samples)` to support aggregation.
A `MeanStatistic` with a sample of `0` is considered an empty statistic.
All `MeanStatistic` instances where `sample = 0` are considered equal,
irrespective of the `mean` value.
Algorithm to calculate the mean :
```python
if x.samples = 0:
y
else if y.samples = 0 :
x
else:
samples' = x.samples + y.samples
mean' = (x.mean * x.samples) + (y.mean * y.samples) / samples'
(samples', mean')
```
Implicitly, this means that if both samples are empty, then the aggregate value is also empty.
```java
public final class MeanStatistic implements Serializable, Cloneable {
/**
* Arithmetic mean.
*/
private double mean;
/**
* Number of samples used to calculate
* the mean.
*/
private long samples;
/**
* Get the mean value.
* @return the mean
*/
public double getMean() {
return mean;
}
/**
* Get the sample count.
* @return the sample count; 0 means empty
*/
public long getSamples() {
return samples;
}
/**
* Is a statistic empty?
* @return true if the sample count is 0
*/
public boolean isEmpty() {
return samples == 0;
}
/**
* Add another mean statistic to create a new statistic.
* When adding two statistics, if either is empty then
* a copy of the non-empty statistic is returned.
* If both are empty then a new empty statistic is returned.
*
* @param other other value
* @return the aggregate mean
*/
public MeanStatistic add(final MeanStatistic other) {
/* Implementation elided. */
}
@Override
public int hashCode() {
return Objects.hash(mean, samples);
}
@Override
public boolean equals(final Object o) {
if (this == o) { return true; }
if (o == null || getClass() != o.getClass()) { return false; }
MeanStatistic that = (MeanStatistic) o;
if (this.isEmpty()) {
return that.isEmpty();
}
return Double.compare(that.mean, mean) == 0 &&
samples == that.samples;
}
@Override
public MeanStatistic clone() {
return new MeanStatistic(this);
}
public MeanStatistic copy() {
return new MeanStatistic(this);
}
}
```
<!-- ============================================================= -->
<!-- Interface: IOStatisticsSource -->
<!-- ============================================================= -->
### class `org.apache.hadoop.fs.statistics.IOStatisticsSource`
```java
/**
* A source of IO statistics.
* These statistics MUST be instance specific, not thread local.
*/
@InterfaceStability.Unstable
public interface IOStatisticsSource {
/**
* Return a statistics instance.
* It is not a requirement that the same instance is returned every time.
* {@link IOStatisticsSource}.
* If the object implementing this is Closeable, this method
* may return null if invoked on a closed object, even if
* it returns a valid instance when called earlier.
* @return an IOStatistics instance or null
*/
IOStatistics getIOStatistics();
}
```
This is the interface which an object instance MUST implement if they are a source of
IOStatistics information.
#### Invariants
The result of `getIOStatistics()` must be one of
* `null`
* an immutable `IOStatistics` for which each map of entries is
an empty map.
* an instance of an `IOStatistics` whose statistics MUST BE unique to that
instance of the class implementing `IOStatisticsSource`.
Less formally: if the statistics maps returned are non-empty, all the statistics
must be collected from the current instance, and not from any other instances, the way
some of the `FileSystem` statistics are collected.
The result of `getIOStatistics()`, if non-null, MAY be a different instance
on every invocation.
<!-- ============================================================= -->
<!-- Interface: IOStatistics -->
<!-- ============================================================= -->
### class `org.apache.hadoop.fs.statistics.IOStatistics`
These are per-instance statistics provided by an object which
implements `IOStatisticsSource`.
```java
@InterfaceAudience.Public
@InterfaceStability.Unstable
public interface IOStatistics {
/**
* Map of counters.
* @return the current map of counters.
*/
Map<String, Long> counters();
/**
* Map of gauges.
* @return the current map of gauges.
*/
Map<String, Long> gauges();
/**
* Map of minumums.
* @return the current map of minumums.
*/
Map<String, Long> minumums();
/**
* Map of maximums.
* @return the current map of maximums.
*/
Map<String, Long> maximums();
/**
* Map of meanStatistics.
* @return the current map of MeanStatistic statistics.
*/
Map<String, MeanStatistic> meanStatistics();
}
```
### Statistic Naming
The naming policy of statistics is designed to be readable, shareable
and ideally consistent across `IOStatisticSource` implementations.
* Characters in key names MUST match the regular expression
`[a-z|0-9|_]` with the exception of the first character, which
MUST be in the range `[a-z]`. Thus the full regular expression
for a valid statistic name is:
[a-z][a-z|0-9|_]+
* Where possible, the names of statistics SHOULD be those defined
with common names.
org.apache.hadoop.fs.statistics.StreamStatisticNames
org.apache.hadoop.fs.statistics.StoreStatisticNames
Note 1.: these are evolving; for clients to safely reference their
statistics by name they SHOULD be copied to the application.
(i.e. for an application compiled hadoop 3.4.2 to link against hadoop 3.4.1,
copy the strings).
Note 2: keys defined in these classes SHALL NOT be removed
from subsequent Hadoop releases.
* A common statistic name MUST NOT be used to report any other statistic and
MUST use the pre-defined unit of measurement.
* A statistic name in one of the maps SHOULD NOT be re-used in another map.
This aids diagnostics of logged statistics.
### Statistic Maps
For each map of statistics returned:
* The operations to add/remove entries are unsupported: the map returned
MAY be mutable by the source of statistics.
* The map MAY be empty.
* The map keys each represent a measured statistic.
* The set of keys in a map SHOULD remain unchanged, and MUST NOT remove keys.
* The statistics SHOULD be dynamic: every lookup of an entry SHOULD
return the latest value.
* The values MAY change across invocations of `Map.values()` and `Map.entries()`
* The update MAY be in the `iterable()` calls of the iterators returned,
or MAY be in the actual `iterable.next()` operation. That is: there is
no guarantee as to when the evaluation takes place.
* The returned `Map.Entry` instances MUST return the same value on
repeated `getValue()` calls. (i.e once you have the entry, it is immutable).
* Queries of statistics SHOULD be fast and non-blocking to the extent
that if invoked during a long operation, they will prioritize
returning fast over most timely values.
* The statistics MAY lag; especially for statistics collected in separate
operations (e.g stream IO statistics as provided by a filesystem
instance).
* Statistics which represent time SHOULD use milliseconds as their unit.
* Statistics which represent time and use a different unit MUST document
the unit used.
### Thread Model
1. An instance of `IOStatistics` can be shared across threads;
1. Read access to the supplied statistics maps MUST be thread safe.
1. Iterators returned from the maps MUST NOT be shared across threads.
1. The statistics collected MUST include all operations which took
place across all threads performing work for the monitored object.
1. The statistics reported MUST NOT be local to the active thread.
This is different from the `FileSystem.Statistics` behavior where per-thread statistics
are collected and reported.
That mechanism supports collecting limited read/write statistics for different
worker threads sharing the same FS instance, but as the collection is thread local,
it invariably under-reports IO performed in other threads on behalf of a worker thread.
## Statisic Snapshot
A snapshot of the current statistic values MAY be obtained by calling
`IOStatisticsSupport.snapshotIOStatistics()`
```java
public static <X extends IOStatistics & Serializable> X
snapshotIOStatistics(IOStatistics statistics)
```
This snapshot is serializable through Java serialization and through
Jackson to/from JSON.
## Helper Classes
### class `org.apache.hadoop.fs.statistics.IOStatisticsSupport`
This provides helper methods to work with IOStatistics sources and instances.
Consult the javadocs for its operations.
### class `org.apache.hadoop.fs.statistics.IOStatisticsLogging`
Support for efficiently logging `IOStatistics`/`IOStatisticsSource`
instances.
These are intended for assisting logging, including only enumerating the
state of an `IOStatistics` instance when the log level needs it.
```java
LOG.info("IOStatistics after upload: {}", demandStringify(iostats));
// or even better, as it results in only a single object creations
Object latest = demandStringify(iostats);
LOG.info("IOStatistics : {}", latest);
/* do some work. */
LOG.info("IOStatistics : {}", latest);
```
## Package `org.apache.hadoop.fs.statistics.impl`
This contains implementation classes to support providing statistics to applications.
These MUST NOT BE used by applications. If a feature is needed from this package then
the provisioning of a public implementation MAY BE raised via the Hadoop development
channels.
These MAY be used by those implementations of the Hadoop `FileSystem`, `AbstractFileSystem`
and related classes which are not in the hadoop source tree. Implementors MUST BE
aware that the implementation this code is unstable and may change across
minor point releases of Hadoop.

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists;
import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.test.LambdaTestUtils.eventually;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@ -106,6 +107,8 @@ public abstract class AbstractContractMultipartUploaderTest extends
CompletableFuture<Integer> f
= uploader.abortUploadsUnderPath(teardown);
f.get();
LOG.info("Statistics {}",
ioStatisticsSourceToString(uploader));
} catch (Exception e) {
LOG.warn("Exeception in teardown", e);
}

View File

@ -0,0 +1,313 @@
/*
* 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;
import java.util.Collections;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.AfterClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
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.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
/**
* Tests {@link IOStatistics} support in input and output streams.
* <p>
* Requires both the input and output streams to offer the basic
* bytes read/written statistics.
* </p>
* If the IO is buffered, that information must be provided,
* especially the input buffer size.
*/
public abstract class AbstractContractStreamIOStatisticsTest
extends AbstractFSContractTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class);
/**
* FileSystem statistics are collected across every test case.
*/
protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS =
snapshotIOStatistics();
@Override
public void teardown() throws Exception {
final FileSystem fs = getFileSystem();
if (fs instanceof IOStatisticsSource) {
FILESYSTEM_IOSTATS.aggregate(((IOStatisticsSource)fs).getIOStatistics());
}
super.teardown();
}
/**
* Dump the filesystem statistics after the class if contains any values.
*/
@AfterClass
public static void dumpFileSystemIOStatistics() {
if (!FILESYSTEM_IOSTATS.counters().isEmpty()) {
// if there is at least one counter
LOG.info("Aggregate FileSystem Statistics {}",
ioStatisticsToPrettyString(FILESYSTEM_IOSTATS));
}
}
@Test
public void testOutputStreamStatisticKeys() throws Throwable {
describe("Look at the statistic keys of an output stream");
Path path = methodPath();
FileSystem fs = getFileSystem();
fs.mkdirs(path.getParent());
try (FSDataOutputStream out = fs.create(path, true)) {
IOStatistics statistics = extractStatistics(out);
final List<String> keys = outputStreamStatisticKeys();
Assertions.assertThat(statistics.counters().keySet())
.describedAs("statistic keys of %s", statistics)
.containsAll(keys);
Assertions.assertThat(keys)
.describedAs("Statistics supported by the stream %s", out)
.contains(STREAM_WRITE_BYTES);
} finally {
fs.delete(path, false);
}
}
/**
* If the stream writes in blocks, then counters during the write may be
* zero until a whole block is written -or the write has finished.
* @return true if writes are buffered into whole blocks.
*/
public boolean streamWritesInBlocks() {
return false;
}
@Test
public void testWriteSingleByte() throws Throwable {
describe("Write a byte to a file and verify"
+ " the stream statistics are updated");
Path path = methodPath();
FileSystem fs = getFileSystem();
fs.mkdirs(path.getParent());
boolean writesInBlocks = streamWritesInBlocks();
try (FSDataOutputStream out = fs.create(path, true)) {
IOStatistics statistics = extractStatistics(out);
// before a write, no bytes
verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 0);
out.write('0');
verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES,
writesInBlocks ? 0 : 1);
// close the stream
out.close();
// statistics are still valid after the close
// always call the output stream to check that behavior
statistics = extractStatistics(out);
final String strVal = statistics.toString();
LOG.info("Statistics = {}", strVal);
verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 1);
} finally {
fs.delete(path, false);
}
}
@Test
public void testWriteByteArrays() throws Throwable {
describe("Write byte arrays to a file and verify"
+ " the stream statistics are updated");
Path path = methodPath();
FileSystem fs = getFileSystem();
fs.mkdirs(path.getParent());
boolean writesInBlocks = streamWritesInBlocks();
try (FSDataOutputStream out = fs.create(path, true)) {
Object demandStatsString = demandStringifyIOStatisticsSource(out);
// before a write, no bytes
final byte[] bytes = ContractTestUtils.toAsciiByteArray(
"statistically-speaking");
final long len = bytes.length;
out.write(bytes);
out.flush();
LOG.info("stats {}", demandStatsString);
IOStatistics statistics = extractStatistics(out);
verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES,
writesInBlocks ? 0 : len);
out.write(bytes);
out.flush();
verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES,
writesInBlocks ? 0 : len * 2);
// close the stream
out.close();
LOG.info("stats {}", demandStatsString);
// statistics are still valid after the close
// always call the output stream to check that behavior
statistics = extractStatistics(out);
verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, len * 2);
// the to string value must contain the same counterHiCable you mean
Assertions.assertThat(demandStatsString.toString())
.contains(Long.toString(len * 2));
} finally {
fs.delete(path, false);
}
}
@Test
public void testInputStreamStatisticKeys() throws Throwable {
describe("Look at the statistic keys of an input stream");
Path path = methodPath();
FileSystem fs = getFileSystem();
ContractTestUtils.touch(fs, path);
try (FSDataInputStream in = fs.open(path)) {
IOStatistics statistics = extractStatistics(in);
final List<String> keys = inputStreamStatisticKeys();
Assertions.assertThat(statistics.counters().keySet())
.describedAs("statistic keys of %s", statistics)
.containsAll(keys);
Assertions.assertThat(keys)
.describedAs("Statistics supported by the stream %s", in)
.contains(STREAM_READ_BYTES);
verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0);
} finally {
fs.delete(path, false);
}
}
@Test
public void testInputStreamStatisticRead() throws Throwable {
describe("Read Data from an input stream");
Path path = methodPath();
FileSystem fs = getFileSystem();
final int fileLen = 1024;
final byte[] ds = dataset(fileLen, 'a', 26);
ContractTestUtils.writeDataset(fs, path, ds, fileLen, 8_000, true);
try (FSDataInputStream in = fs.open(path)) {
long current = 0;
IOStatistics statistics = extractStatistics(in);
verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0);
Assertions.assertThat(in.read()).isEqualTo('a');
int bufferSize = readBufferSize();
// either a single byte was read or a whole block
current = verifyBytesRead(statistics, current, 1, bufferSize);
final int bufferLen = 128;
byte[] buf128 = new byte[bufferLen];
in.read(buf128);
current = verifyBytesRead(statistics, current, bufferLen, bufferSize);
in.readFully(buf128);
current = verifyBytesRead(statistics, current, bufferLen, bufferSize);
in.readFully(0, buf128);
current = verifyBytesRead(statistics, current, bufferLen, bufferSize);
// seek must not increment the read counter
in.seek(256);
verifyBytesRead(statistics, current, 0, bufferSize);
// if a stream implements lazy-seek the seek operation
// may be postponed until the read
final int sublen = 32;
Assertions.assertThat(in.read(buf128, 0, sublen))
.isEqualTo(sublen);
current = verifyBytesRead(statistics, current, sublen, bufferSize);
// perform some read operations near the end of the file such that
// the buffer will not be completely read.
// skip these tests for buffered IO as it is too complex to work out
if (bufferSize == 0) {
final int pos = fileLen - sublen;
in.seek(pos);
Assertions.assertThat(in.read(buf128))
.describedAs("Read overlapping EOF")
.isEqualTo(sublen);
current = verifyStatisticCounterValue(statistics, STREAM_READ_BYTES,
current + sublen);
Assertions.assertThat(in.read(pos, buf128, 0, bufferLen))
.describedAs("Read(buffer) overlapping EOF")
.isEqualTo(sublen);
verifyStatisticCounterValue(statistics, STREAM_READ_BYTES,
current + sublen);
}
} finally {
fs.delete(path, false);
}
}
/**
* Verify the bytes read value, taking into account block size.
* @param statistics stats
* @param current current count
* @param bytesRead bytes explicitly read
* @param bufferSize buffer size of stream
* @return the current count of bytes read <i>ignoring block size</i>
*/
public long verifyBytesRead(final IOStatistics statistics,
final long current,
final int bytesRead, final int bufferSize) {
// final position. for unbuffered read, this is the expected value
long finalPos = current + bytesRead;
long expected = finalPos;
if (bufferSize > 0) {
// buffered. count of read is number of buffers already read
// plus the current buffer, multiplied by that buffer size
expected = bufferSize * (1 + (current / bufferSize));
}
verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, expected);
return finalPos;
}
/**
* Buffer size for reads.
* Filesystems performing block reads (checksum, etc)
* must return their buffer value is
* @return buffer capacity; 0 for unbuffered
*/
public int readBufferSize() {
return 0;
}
/**
* Keys which the output stream must support.
* @return a list of keys
*/
public List<String> outputStreamStatisticKeys() {
return Collections.singletonList(STREAM_WRITE_BYTES);
}
/**
* Keys which the input stream must support.
* @return a list of keys
*/
public List<String> inputStreamStatisticKeys() {
return Collections.singletonList(STREAM_READ_BYTES);
}
}

View File

@ -0,0 +1,80 @@
/*
* 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.localfs;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS;
/**
* Test IOStatistics through the local FS.
*/
public class TestLocalFSContractStreamIOStatistics extends
AbstractContractStreamIOStatisticsTest {
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new LocalFSContract(conf);
}
/**
* Keys which the input stream must support.
* @return a list of keys
*/
public List<String> inputStreamStatisticKeys() {
return Arrays.asList(STREAM_READ_BYTES,
STREAM_READ_EXCEPTIONS,
STREAM_READ_SEEK_OPERATIONS,
STREAM_READ_SKIP_OPERATIONS,
STREAM_READ_SKIP_BYTES);
}
/**
* Keys which the output stream must support.
* @return a list of keys
*/
@Override
public List<String> outputStreamStatisticKeys() {
return Arrays.asList(STREAM_WRITE_BYTES,
STREAM_WRITE_EXCEPTIONS);
}
@Override
public int readBufferSize() {
return 1024;
}
@Override
public boolean streamWritesInBlocks() {
return true;
}
}

View File

@ -0,0 +1,528 @@
/*
* 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.statistics;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.ObjectStreamClass;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.assertj.core.api.AbstractLongAssert;
import org.assertj.core.api.ObjectAssert;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import static org.assertj.core.api.Assertions.assertThat;
/**
* Assertions and any other support for IOStatistics testing.
* If used downstream: know it is unstable.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class IOStatisticAssertions {
private static final String COUNTER = "Counter";
private static final String GAUGE = "Gauge";
private static final String MINIMUM = "Minimum";
private static final String MAXIMUM = "Maxiumum";
private static final String MEAN = "Mean";
private IOStatisticAssertions() {
}
/**
* Get a required counter statistic.
* @param stats statistics source
* @param key statistic key
* @return the value
*/
public static long lookupCounterStatistic(
final IOStatistics stats,
final String key) {
return lookupStatistic(COUNTER, key,
verifyStatisticsNotNull(stats).counters());
}
/**
* Given an IOStatistics instance, verify it is not null,
* and return the value for continued use in a test.
* @param stats statistics source.
* @param <T> type of statistics
* @return the value passed in.
*/
public static <T extends IOStatistics> T
verifyStatisticsNotNull(final T stats) {
assertThat(stats)
.describedAs("IO Statistics reference")
.isNotNull();
return stats;
}
/**
* Get a required gauge statistic.
* @param stats statistics source
* @param key statistic key
* @return the value
*/
public static long lookupGaugeStatistic(
final IOStatistics stats,
final String key) {
return lookupStatistic(GAUGE, key,
verifyStatisticsNotNull(stats).gauges());
}
/**
* Get a required maximum statistic.
* @param stats statistics source
* @param key statistic key
* @return the value
*/
public static long lookupMaximumStatistic(
final IOStatistics stats,
final String key) {
return lookupStatistic(MAXIMUM, key,
verifyStatisticsNotNull(stats).maximums());
}
/**
* Get a required minimum statistic.
* @param stats statistics source
* @param key statistic key
* @return the value
*/
public static long lookupMinimumStatistic(
final IOStatistics stats,
final String key) {
return lookupStatistic(MINIMUM, key,
verifyStatisticsNotNull(stats).minimums());
}
/**
* Get a required mean statistic.
* @param stats statistics source
* @param key statistic key
* @return the value
*/
public static MeanStatistic lookupMeanStatistic(
final IOStatistics stats,
final String key) {
return lookupStatistic(MEAN, key,
verifyStatisticsNotNull(stats).meanStatistics());
}
/**
* Get a required counter statistic.
* @param <E> type of map element
* @param type type for error text
* @param key statistic key
* @param map map to probe
* @return the value
*/
private static <E> E lookupStatistic(
final String type,
final String key,
final Map<String, E> map) {
final E statistic = map.get(key);
assertThat(statistic)
.describedAs("%s named %s", type, key)
.isNotNull();
return statistic;
}
/**
* Assert that a counter has an expected value.
* @param stats statistics source
* @param key statistic key
* @param value expected value.
* @return the value (which always equals the expected value)
*/
public static long verifyStatisticCounterValue(
final IOStatistics stats,
final String key,
final long value) {
return verifyStatisticValue(COUNTER, key,
verifyStatisticsNotNull(stats).counters(), value);
}
/**
* Assert that a gauge has an expected value.
* @param stats statistics source
* @param key statistic key
* @param value expected value.
* @return the value (which always equals the expected value)
*/
public static long verifyStatisticGaugeValue(
final IOStatistics stats,
final String key,
final long value) {
return verifyStatisticValue(GAUGE, key,
verifyStatisticsNotNull(stats).gauges(), value);
}
/**
* Assert that a maximum has an expected value.
* @param stats statistics source
* @param key statistic key
* @param value expected value.
* @return the value (which always equals the expected value)
*/
public static long verifyStatisticMaximumValue(
final IOStatistics stats,
final String key,
final long value) {
return verifyStatisticValue(MAXIMUM, key,
verifyStatisticsNotNull(stats).maximums(), value);
}
/**
* Assert that a minimum has an expected value.
* @param stats statistics source
* @param key statistic key
* @param value expected value.
* @return the value (which always equals the expected value)
*/
public static long verifyStatisticMinimumValue(
final IOStatistics stats,
final String key,
final long value) {
return verifyStatisticValue(MINIMUM, key,
verifyStatisticsNotNull(stats).minimums(), value);
}
/**
* Assert that a mean has an expected value.
* @param stats statistics source
* @param key statistic key
* @param value expected value.
* @return the value (which always equals the expected value)
*/
public static MeanStatistic verifyStatisticMeanValue(
final IOStatistics stats,
final String key,
final MeanStatistic value) {
return verifyStatisticValue(MEAN, key,
verifyStatisticsNotNull(stats).meanStatistics(), value);
}
/**
* Assert that a given statistic has an expected value.
* @param type type for error text
* @param key statistic key
* @param map map to look up
* @param value expected value.
* @param <E> type of map element
* @return the value (which always equals the expected value)
*/
private static <E> E verifyStatisticValue(
final String type,
final String key,
final Map<String, E> map,
final E value) {
final E statistic = lookupStatistic(type, key, map);
assertThat(statistic)
.describedAs("%s named %s with expected value %s", type,
key, value)
.isEqualTo(value);
return statistic;
}
/**
* Assert that a given statistic has an expected value.
* @param <E> type of map element
* @param type type for error text
* @param key statistic key
* @param map map to look up
* @return an ongoing assertion
*/
private static <E> ObjectAssert<E> assertThatStatistic(
final String type,
final String key,
final Map<String, E> map) {
final E statistic = lookupStatistic(type, key, map);
return assertThat(statistic)
.describedAs("%s named %s", type, key);
}
/**
* Assert that a given statistic has an expected value.
* @param <E> type of map element
* @param type type for error text
* @param key statistic key
* @param map map to look up
* @return an ongoing assertion
*/
private static AbstractLongAssert<?> assertThatStatisticLong(
final String type,
final String key,
final Map<String, Long> map) {
final long statistic = lookupStatistic(type, key, map);
return assertThat(statistic)
.describedAs("%s named %s", type, key);
}
/**
* Start an assertion chain on
* a required counter statistic.
* @param stats statistics source
* @param key statistic key
* @return an ongoing assertion
*/
public static AbstractLongAssert<?> assertThatStatisticCounter(
final IOStatistics stats,
final String key) {
return assertThatStatisticLong(COUNTER, key,
verifyStatisticsNotNull(stats).counters());
}
/**
* Start an assertion chain on
* a required gauge statistic.
* @param stats statistics source
* @param key statistic key
* @return an ongoing assertion
*/
public static AbstractLongAssert<?> assertThatStatisticGauge(
final IOStatistics stats,
final String key) {
return assertThatStatisticLong(GAUGE, key,
verifyStatisticsNotNull(stats).gauges());
}
/**
* Start an assertion chain on
* a required minimum statistic.
* @param stats statistics source
* @param key statistic key
* @return an ongoing assertion
*/
public static AbstractLongAssert<?> assertThatStatisticMinimum(
final IOStatistics stats,
final String key) {
return assertThatStatisticLong(MINIMUM, key,
verifyStatisticsNotNull(stats).minimums());
}
/**
* Start an assertion chain on
* a required maximum statistic.
* @param stats statistics source
* @param key statistic key
* @return an ongoing assertion
*/
public static AbstractLongAssert<?> assertThatStatisticMaximum(
final IOStatistics stats,
final String key) {
return assertThatStatisticLong(MAXIMUM, key,
verifyStatisticsNotNull(stats).maximums());
}
/**
* Start an assertion chain on
* a required mean statistic.
* @param stats statistics source
* @param key statistic key
* @return an ongoing assertion
*/
public static ObjectAssert<MeanStatistic> assertThatStatisticMean(
final IOStatistics stats,
final String key) {
return assertThatStatistic(MEAN, key,
verifyStatisticsNotNull(stats).meanStatistics());
}
/**
* Start an assertion chain on
* a required mean statistic with the initial validation on the
* sample count and sum.
* @param stats statistics source
* @param key statistic key
* @return an ongoing assertion
*/
public static ObjectAssert<MeanStatistic> assertThatStatisticMeanMatches(
final IOStatistics stats,
final String key,
final long samples,
final long sum) {
return assertThatStatisticMean(stats, key)
.matches(p -> (p.getSamples() == samples),
"samples == " + samples)
.matches(p -> (p.getSum() == sum),
"sum == " + sum);
}
/**
* Assert that a given counter statistic is untracked.
* @param stats statistics source
* @param type type for error text
* @param key statistic key
* @param map map to probe
*/
private static void assertUntracked(final IOStatistics stats,
final String type,
final String key,
final Map<String, ?> map) {
assertThat(map.containsKey(key))
.describedAs("%s %s is tracked in %s", type, key, stats)
.isFalse();
}
/**
* Assert that a given counter statistic is untracked.
* @param stats statistics source
* @param type type for error text
* @param key statistic key
* @param map map to probe
*/
private static void assertTracked(final IOStatistics stats,
final String type,
final String key,
final Map<String, ?> map) {
assertThat(map.containsKey(key))
.describedAs("%s %s is not tracked in %s", type, key, stats)
.isTrue();
}
/**
* Assert that a given statistic is tracked.
* @param stats statistics source
* @param key statistic key
*/
public static void assertStatisticCounterIsTracked(
final IOStatistics stats,
final String key) {
assertTracked(stats, COUNTER, key,
verifyStatisticsNotNull(stats).counters());
}
/**
* Assert that a given counter statistic is untracked.
* @param stats statistics source
* @param key statistic key
*/
public static void assertStatisticCounterIsUntracked(
final IOStatistics stats,
final String key) {
assertUntracked(stats, COUNTER, key,
verifyStatisticsNotNull(stats).counters());
}
/**
* Assert that an object is a statistics source and that the
* statistics is not null.
* @param source source object.
*/
public static void assertIsStatisticsSource(Object source) {
assertThat(source)
.describedAs("Object %s", source)
.isInstanceOf(IOStatisticsSource.class)
.extracting(o -> ((IOStatisticsSource) o).getIOStatistics())
.isNotNull();
}
/**
* Query the source for the statistics; fails if the statistics
* returned are null or the class does not implement the API.
* @param source source object.
* @return the statistics it provides.
*/
public static IOStatistics extractStatistics(Object source) {
assertThat(source)
.describedAs("Object %s", source)
.isInstanceOf(IOStatisticsSource.class);
IOStatisticsSource ios = (IOStatisticsSource) source;
return extractStatistics(ios);
}
/**
* Get the non-null statistics.
* @param ioStatisticsSource source
* @return the statistics, guaranteed to be non null
*/
private static IOStatistics extractStatistics(
final IOStatisticsSource ioStatisticsSource) {
IOStatistics statistics = ioStatisticsSource.getIOStatistics();
assertThat(statistics)
.describedAs("Statistics from %s", ioStatisticsSource)
.isNotNull();
return statistics;
}
/**
* Perform a serialization round trip on a statistics instance.
* @param stat statistic
* @return the deserialized version.
*/
public static IOStatistics statisticsJavaRoundTrip(final IOStatistics stat)
throws IOException, ClassNotFoundException {
assertThat(stat).isInstanceOf(Serializable.class);
ByteArrayOutputStream baos = new ByteArrayOutputStream(1024);
try (ObjectOutputStream oos = new ObjectOutputStream(baos)) {
oos.writeObject(stat);
}
ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
IOStatistics deser;
try (ObjectInputStream ois = new RestrictedInput(bais,
IOStatisticsSnapshot.requiredSerializationClasses())) {
deser = (IOStatistics) ois.readObject();
}
return deser;
}
private static final class RestrictedInput extends ObjectInputStream {
private final List<String> allowedClasses;
private RestrictedInput(final InputStream in,
final List<Class> allowedClasses) throws IOException {
super(in);
this.allowedClasses = allowedClasses.stream()
.map(Class::getName)
.collect(Collectors.toList());
}
@Override
protected Class<?> resolveClass(final ObjectStreamClass desc)
throws IOException, ClassNotFoundException {
final String classname = desc.getName();
if (!allowedClasses.contains(classname)) {
throw new ClassNotFoundException("Class " + classname
+ " Not in list of allowed classes");
}
return super.resolveClass(desc);
}
}
}

View File

@ -0,0 +1,361 @@
/*
* 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.statistics;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.FutureIOSupport;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.apache.hadoop.util.functional.FunctionRaisingIOE;
import org.apache.hadoop.util.functional.FutureIO;
import static org.apache.hadoop.fs.statistics.DurationStatisticSummary.fetchDurationSummary;
import static org.apache.hadoop.fs.statistics.DurationStatisticSummary.fetchSuccessSummary;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.*;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.*;
import static org.apache.hadoop.fs.statistics.impl.StubDurationTrackerFactory.STUB_DURATION_TRACKER_FACTORY;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.assertj.core.api.Assertions.assertThat;
/**
* Test the IOStatistic DurationTracker logic.
*/
public class TestDurationTracking extends AbstractHadoopTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestDurationTracking.class);
private static final String REQUESTS = "requests";
public static final String UNKNOWN = "unknown";
private IOStatisticsStore stats;
private final AtomicInteger invocationCounter = new AtomicInteger(0);
@Before
public void setup() {
stats = iostatisticsStore()
.withDurationTracking(REQUESTS)
.build();
}
@After
public void teardown() {
LOG.info("stats {}", stats);
}
/**
* Duration tracking.
*/
@Test
public void testDurationTryWithResources() throws Throwable {
DurationTracker tracker =
stats.trackDuration(REQUESTS);
verifyStatisticCounterValue(stats, REQUESTS, 1L);
sleep();
tracker.close();
try (DurationTracker ignored =
stats.trackDuration(REQUESTS)) {
sleep();
}
LOG.info("Statistics: {}", stats);
DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS);
assertSummaryValues(summary, 2, 1, 1);
assertSummaryMean(summary, 2, 0);
}
/**
* A little sleep method; exceptions are swallowed.
* Increments {@link #invocationCounter}.
* Increments {@inheritDoc #atomicCounter}.
*/
public void sleep() {
sleepf(10);
}
/**
* A little sleep function; exceptions are swallowed.
* Increments {@link #invocationCounter}.
*/
protected int sleepf(final int millis) {
invocationCounter.incrementAndGet();
try {
Thread.sleep(millis);
} catch (InterruptedException ignored) {
}
return millis;
}
/**
* Assert that the sleep counter has been invoked
* the expected number of times.
* @param expected expected value
*/
private void assertCounterValue(final int expected) {
assertThat(invocationCounter.get())
.describedAs("Sleep invocation Counter")
.isEqualTo(expected);
}
/**
* Test that a function raising an IOE can be wrapped.
*/
@Test
public void testDurationFunctionIOE() throws Throwable {
FunctionRaisingIOE<Integer, Integer> fn =
trackFunctionDuration(stats, REQUESTS,
(Integer x) -> invocationCounter.getAndSet(x));
assertThat(fn.apply(1)).isEqualTo(0);
assertCounterValue(1);
assertSummaryValues(
fetchSuccessSummary(stats, REQUESTS),
1, 0, 0);
}
/**
* Trigger a failure and verify its the failure statistics
* which go up.
*/
@Test
public void testDurationFunctionIOEFailure() throws Throwable {
FunctionRaisingIOE<Integer, Integer> fn =
trackFunctionDuration(stats, REQUESTS,
(Integer x) -> {
sleep();
return 100 / x;
});
intercept(ArithmeticException.class,
() -> fn.apply(0));
assertSummaryValues(
fetchSuccessSummary(stats, REQUESTS),
1, -1, -1);
DurationStatisticSummary failures = fetchDurationSummary(stats, REQUESTS,
false);
assertSummaryValues(failures, 1, 0, 0);
assertSummaryMean(failures, 1, 0);
}
/**
* Trigger a failure and verify its the failure statistics
* which go up.
*/
@Test
public void testDurationJavaFunctionFailure() throws Throwable {
Function<Integer, Integer> fn =
trackJavaFunctionDuration(stats, REQUESTS,
(Integer x) -> {
return 100 / x;
});
intercept(ArithmeticException.class,
() -> fn.apply(0));
assertSummaryValues(
fetchSuccessSummary(stats, REQUESTS),
1, -1, -1);
DurationStatisticSummary failures = fetchDurationSummary(stats, REQUESTS,
false);
assertSummaryValues(failures, 1, 0, 0);
}
/**
* Test trackDurationOfCallable.
*/
@Test
public void testCallableDuration() throws Throwable {
// call the operation
assertThat(
trackDurationOfCallable(stats, REQUESTS, () -> sleepf(100)).call())
.isEqualTo(100);
DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS);
assertSummaryValues(summary, 1, 0, 0);
assertSummaryMean(summary, 1, 0);
}
/**
* Callable raising an RTE after a sleep; failure
* stats will be updated and the execution count will be
* 1.
*/
@Test
public void testCallableFailureDuration() throws Throwable {
intercept(RuntimeException.class,
trackDurationOfCallable(stats, REQUESTS, () -> {
sleepf(100);
throw new RuntimeException("oops");
}));
assertCounterValue(1);
assertSummaryValues(
fetchSuccessSummary(stats, REQUESTS),
1, -1, -1);
assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false),
1, 0, 0);
}
/**
* Duration of the successful execution of a InvocationRaisingIOE.
*/
@Test
public void testInvocationDuration() throws Throwable {
// call the operation
trackDurationOfInvocation(stats, REQUESTS, () -> {
sleepf(100);
});
assertCounterValue(1);
DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS);
assertSummaryValues(summary, 1, 0, 0);
assertSummaryMean(summary, 1, 0);
}
/**
* Duration of the successful execution of a CallableRaisingIOE.
*/
@Test
public void testCallableIOEDuration() throws Throwable {
// call the operation
assertThat(
trackDuration(stats, REQUESTS, () -> sleepf(100)))
.isEqualTo(100);
DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS);
assertSummaryValues(summary, 1, 0, 0);
assertSummaryMean(summary, 1, 0);
}
/**
* Track the duration of an IOE raising callable which fails.
*/
@Test
public void testCallableIOEFailureDuration() throws Throwable {
intercept(IOException.class,
() ->
trackDuration(stats, REQUESTS, () -> {
sleepf(100);
throw new IOException("oops");
}));
assertSummaryValues(
fetchSuccessSummary(stats, REQUESTS),
1, -1, -1);
assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false),
1, 0, 0);
}
/**
* Track the duration of an IOE raising callable which fails.
*/
@Test
public void testDurationThroughEval() throws Throwable {
CompletableFuture<Object> eval = FutureIOSupport.eval(
trackDurationOfOperation(stats, REQUESTS, () -> {
sleepf(100);
throw new FileNotFoundException("oops");
}));
intercept(FileNotFoundException.class, "oops", () ->
FutureIO.awaitFuture(eval));
assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false),
1, 0, 0);
}
/**
* It's OK to track a duration against an unknown statistic.
*/
@Test
public void testUnknownDuration() throws Throwable {
trackDurationOfCallable(stats, UNKNOWN, () -> sleepf(1)).call();
DurationStatisticSummary summary = fetchSuccessSummary(stats, UNKNOWN);
assertSummaryValues(summary, 0, -1, -1);
assertThat(summary.getMean()).isNull();
}
/**
* The stub duration tracker factory can be supplied as an input.
*/
@Test
public void testTrackDurationWithStubFactory() throws Throwable {
trackDuration(STUB_DURATION_TRACKER_FACTORY, UNKNOWN, () -> sleepf(1));
}
/**
* Make sure the tracker returned from the stub factory
* follows the basic lifecycle.
*/
@Test
public void testStubDurationLifecycle() throws Throwable {
DurationTracker tracker = STUB_DURATION_TRACKER_FACTORY
.trackDuration("k", 1);
tracker.failed();
tracker.close();
tracker.close();
}
/**
* Assert that a statistics summary has the specific values.
* @param summary summary data
* @param count count -must match exactly.
* @param minBase minimum value for the minimum field (inclusive)
* @param maxBase minimum value for the maximum field (inclusive)
*/
protected void assertSummaryValues(
final DurationStatisticSummary summary,
final int count,
final int minBase,
final int maxBase) {
assertThat(summary)
.matches(s -> s.getCount() == count, "Count value")
.matches(s -> s.getMax() >= maxBase, "Max value")
.matches(s -> s.getMin() >= minBase, "Min value");
}
/**
* Assert that at a summary has a matching mean value.
* @param summary summary data.
* @param expectedSampleCount sample count -which must match
* @param meanGreaterThan the mean must be greater than this value.
*/
protected void assertSummaryMean(
final DurationStatisticSummary summary,
final int expectedSampleCount,
final double meanGreaterThan) {
String description = "mean of " + summary;
assertThat(summary.getMean())
.describedAs(description)
.isNotNull();
assertThat(summary.getMean().getSamples())
.describedAs(description)
.isEqualTo(expectedSampleCount);
assertThat(summary.getMean().mean())
.describedAs(description)
.isGreaterThan(meanGreaterThan);
}
}

View File

@ -0,0 +1,311 @@
/*
* 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.statistics;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
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.fs.statistics.impl.SourceWrappedStatistics;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsTracked;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsUntracked;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.ENTRY_PATTERN;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.NULL_SOURCE;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
/**
* verify dynamic statistics are dynamic, except when you iterate through
* them, along with other tests of the class's behavior.
*/
public class TestDynamicIOStatistics extends AbstractHadoopTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestDynamicIOStatistics.class);
private static final String ALONG = "along";
private static final String AINT = "aint";
private static final String COUNT = "count";
private static final String EVAL = "eval";
/**
* The statistics.
*/
private IOStatistics statistics = emptyStatistics();
/**
* A source of these statistics.
*/
private IOStatisticsSource statsSource;
private final AtomicLong aLong = new AtomicLong();
private final AtomicInteger aInt = new AtomicInteger();
private final MutableCounterLong counter = new MutableCounterLong(
new Info("counter"), 0);
private long evalLong;
private static final String[] KEYS = new String[]{ALONG, AINT, COUNT, EVAL};
@Before
public void setUp() throws Exception {
statistics = dynamicIOStatistics()
.withAtomicLongCounter(ALONG, aLong)
.withAtomicIntegerCounter(AINT, aInt)
.withMutableCounter(COUNT, counter)
.withLongFunctionCounter(EVAL, x -> evalLong)
.build();
statsSource = new SourceWrappedStatistics(statistics);
}
/**
* The eval operation is foundational.
*/
@Test
public void testEval() throws Throwable {
verifyStatisticCounterValue(statistics, EVAL, 0);
evalLong = 10;
verifyStatisticCounterValue(statistics, EVAL, 10);
}
/**
* Atomic Long statistic.
*/
@Test
public void testAlong() throws Throwable {
verifyStatisticCounterValue(statistics, ALONG, 0);
aLong.addAndGet(1);
verifyStatisticCounterValue(statistics, ALONG, 1);
}
/**
* Atomic Int statistic.
*/
@Test
public void testAint() throws Throwable {
verifyStatisticCounterValue(statistics, AINT, 0);
aInt.addAndGet(1);
verifyStatisticCounterValue(statistics, AINT, 1);
}
/**
* Metrics2 counter.
*/
@Test
public void testCounter() throws Throwable {
verifyStatisticCounterValue(statistics, COUNT, 0);
counter.incr();
verifyStatisticCounterValue(statistics, COUNT, 1);
}
/**
* keys() returns all the keys.
*/
@Test
public void testKeys() throws Throwable {
Assertions.assertThat(statistics.counters().keySet())
.describedAs("statistic keys of %s", statistics)
.containsExactlyInAnyOrder(KEYS);
}
@Test
public void testIteratorHasAllKeys() throws Throwable {
// go through the statistics iterator and assert that it contains exactly
// the values.
assertThat(statistics.counters().keySet())
.containsExactlyInAnyOrder(KEYS);
}
/**
* Verify that the iterator is taken from
* a snapshot of the values.
*/
@Test
public void testIteratorIsSnapshot() throws Throwable {
// set the counters all to 1
incrementAllCounters();
// take the snapshot
final Iterator<Map.Entry<String, Long>> it =
statistics.counters().entrySet().iterator();
// increment the counters
incrementAllCounters();
// now assert that all the iterator values are of value 1
while (it.hasNext()) {
Map.Entry<String, Long> next = it.next();
assertThat(next.getValue())
.describedAs("Value of entry %s", next)
.isEqualTo(1);
}
}
@Test
public void testUnknownStatistic() throws Throwable {
assertStatisticCounterIsUntracked(statistics, "anything");
}
@Test
public void testStatisticsTrackedAssertion() throws Throwable {
// expect an exception to be raised when an assertion
// is made that an unknown statistic is tracked,.
assertThatThrownBy(() ->
assertStatisticCounterIsTracked(statistics, "anything"))
.isInstanceOf(AssertionError.class);
}
@Test
public void testStatisticsValueAssertion() throws Throwable {
// expect an exception to be raised when
// an assertion is made about the value of an unknown statistics
assertThatThrownBy(() ->
verifyStatisticCounterValue(statistics, "anything", 0))
.isInstanceOf(AssertionError.class);
}
/**
* Serialization round trip will preserve all the values.
*/
@Test
public void testSerDeser() throws Throwable {
incrementAllCounters();
IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(statistics);
incrementAllCounters();
IOStatistics deser = IOStatisticAssertions.statisticsJavaRoundTrip(stat);
assertThat(deser.counters().keySet())
.containsExactlyInAnyOrder(KEYS);
for (Map.Entry<String, Long> e : deser.counters().entrySet()) {
assertThat(e.getValue())
.describedAs("Value of entry %s", e)
.isEqualTo(1);
}
}
@Test
public void testStringification() throws Throwable {
assertThat(ioStatisticsToString(statistics))
.isNotBlank()
.contains(KEYS);
}
@Test
public void testDemandStringification() throws Throwable {
String counterPattern = ENTRY_PATTERN;
// this is not yet evaluated
Object demand = demandStringifyIOStatistics(statistics);
// nor is this.
Object demandSource = demandStringifyIOStatisticsSource(statsSource);
// show it evaluates
String formatted1 = String.format(counterPattern, ALONG, aLong.get());
assertThat(demand
.toString())
.contains(formatted1);
assertThat(demandSource
.toString())
.contains(formatted1);
// when the counters are incremented
incrementAllCounters();
incrementAllCounters();
// there are new values to expect
String formatted2 = String.format(counterPattern, ALONG, aLong.get());
assertThat(demand
.toString())
.doesNotContain(formatted1)
.contains(formatted2);
assertThat(demandSource
.toString())
.doesNotContain(formatted1)
.contains(formatted2);
}
@Test
public void testNullSourceStringification() throws Throwable {
assertThat(demandStringifyIOStatisticsSource((IOStatisticsSource) null)
.toString())
.isEqualTo(NULL_SOURCE);
}
@Test
public void testNullStatStringification() throws Throwable {
assertThat(demandStringifyIOStatistics((IOStatistics) null)
.toString())
.isEqualTo(NULL_SOURCE);
}
@Test
public void testStringLogging() throws Throwable {
LOG.info("Output {}", demandStringifyIOStatistics(statistics));
}
/**
* Increment all the counters from their current value.
*/
private void incrementAllCounters() {
aLong.incrementAndGet();
aInt.incrementAndGet();
evalLong += 1;
counter.incr();
}
/**
* Needed to provide a metrics info instance for the counter
* constructor.
*/
private static final class Info implements MetricsInfo {
private final String name;
private Info(final String name) {
this.name = name;
}
@Override
public String name() {
return name;
}
@Override
public String description() {
return name;
}
}
}

View File

@ -0,0 +1,110 @@
/*
* 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.statistics;
import org.junit.Test;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsTracked;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsUntracked;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
/**
* Test handling of the empty IO statistics class.
*/
public class TestEmptyIOStatistics extends AbstractHadoopTestBase {
private final IOStatistics empty = emptyStatistics();
@Test
public void testUnknownStatistic() throws Throwable {
assertStatisticCounterIsUntracked(empty, "anything");
}
@Test
public void testStatisticsTrackedAssertion() throws Throwable {
// expect an exception to be raised when an assertion
// is made that an unknown statistic is tracked,.
assertThatThrownBy(() ->
assertStatisticCounterIsTracked(empty, "anything"))
.isInstanceOf(AssertionError.class);
}
@Test
public void testStatisticsValueAssertion() throws Throwable {
// expect an exception to be raised when
// an assertion is made about the value of an unknown statistics
assertThatThrownBy(() ->
verifyStatisticCounterValue(empty, "anything", 0))
.isInstanceOf(AssertionError.class);
}
@Test
public void testEmptySnapshot() throws Throwable {
final IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(empty);
assertThat(stat.counters().keySet())
.describedAs("keys of snapshot")
.isEmpty();
IOStatistics deser = IOStatisticAssertions.statisticsJavaRoundTrip(stat);
assertThat(deser.counters().keySet())
.describedAs("keys of deserialized snapshot")
.isEmpty();
}
@Test
public void testStringification() throws Throwable {
assertThat(ioStatisticsToString(empty))
.isNotBlank();
}
@Test
public void testWrap() throws Throwable {
IOStatisticsSource statisticsSource = IOStatisticsBinding.wrap(empty);
assertThat(statisticsSource.getIOStatistics())
.isSameAs(empty);
}
@Test
public void testStringifyNullSource() throws Throwable {
assertThat(IOStatisticsLogging.ioStatisticsSourceToString(null))
.isEmpty();
}
@Test
public void testStringifyNullStats() throws Throwable {
assertThat(
IOStatisticsLogging.ioStatisticsSourceToString(
IOStatisticsBinding.wrap(null)))
.isEmpty();
}
@Test
public void testStringificationNull() throws Throwable {
assertThat(ioStatisticsToString(null))
.describedAs("Null statistics should stringify to \"\"")
.isEmpty();
}
}

View File

@ -0,0 +1,147 @@
/*
* 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.statistics;
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.fs.statistics.impl.IOStatisticsBinding;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.apache.hadoop.util.JsonSerialization;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.*;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
/**
* Test handling of the {@link IOStatisticsSnapshot} class.
*/
public class TestIOStatisticsSnapshot extends AbstractHadoopTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestIOStatisticsSnapshot.class);
/**
* Simple snapshot built up in test setup.
*/
private final IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot();
/** Saved to the snapshot as "mean01". */
private MeanStatistic mean0;
/** Saved to the snapshot as "mean1". */
private MeanStatistic mean1;
@Before
public void setup() throws Exception {
snapshot.counters().put("c1", 0L);
snapshot.gauges().put("g1", 1L);
snapshot.minimums().put("m1", -1L);
mean1 = new MeanStatistic(1, 1);
snapshot.meanStatistics().put("mean1",
mean1);
mean0 = new MeanStatistic(0, 1);
snapshot.meanStatistics().put("mean0",
mean0);
}
@Test
public void testTrackedValues() throws Throwable {
verifyStatisticCounterValue(snapshot, "c1", 0L);
verifyStatisticGaugeValue(snapshot, "g1", 1L);
verifyStatisticMinimumValue(snapshot, "m1", -1L);
verifyStatisticMeanValue(snapshot, "mean0",
new MeanStatistic(0, 1));
}
@Test
public void testStatisticsValueAssertion() throws Throwable {
// expect an exception to be raised when
// an assertion is made about the value of an unknown statistics
assertThatThrownBy(() ->
verifyStatisticCounterValue(snapshot, "anything", 0))
.isInstanceOf(AssertionError.class);
}
@Test
public void testStringification() throws Throwable {
assertThat(ioStatisticsToString(snapshot))
.isNotBlank();
}
@Test
public void testStringification2() throws Throwable {
String ss = snapshot.toString();
LOG.info("original {}", ss);
Assertions.assertThat(ss)
.describedAs("snapshot toString()")
.contains("c1=0")
.contains("g1=1");
}
@Test
public void testWrap() throws Throwable {
IOStatisticsSource statisticsSource = IOStatisticsBinding.wrap(snapshot);
assertThat(statisticsSource.getIOStatistics())
.isSameAs(snapshot);
}
@Test
public void testJsonRoundTrip() throws Throwable {
JsonSerialization<IOStatisticsSnapshot> serializer
= IOStatisticsSnapshot.serializer();
String json = serializer.toJson(snapshot);
LOG.info("serialized form\n{}", json);
IOStatisticsSnapshot deser = serializer.fromJson(json);
verifyDeserializedInstance(deser);
}
/**
* Verify the deserialized instance's data
* matches the expected values.
* @param deser deserialized vlaue.
*/
public void verifyDeserializedInstance(
final IOStatistics deser) {
LOG.info("deserialized {}", deser);
verifyStatisticCounterValue(deser, "c1", 0L);
verifyStatisticGaugeValue(deser, "g1", 1L);
verifyStatisticMinimumValue(deser, "m1", -1L);
verifyStatisticMeanValue(deser, "mean0",
new MeanStatistic(0, 1));
verifyStatisticMeanValue(deser, "mean1",
snapshot.meanStatistics().get("mean1"));
}
@Test
public void testJavaRoundTrip() throws Throwable {
verifyDeserializedInstance(
IOStatisticAssertions.statisticsJavaRoundTrip(
snapshot));
}
}

View File

@ -0,0 +1,177 @@
/*
* 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.statistics;
import org.assertj.core.api.Assertions;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.apache.hadoop.util.JsonSerialization;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMeanMatches;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticMaximumValue;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticMinimumValue;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
/**
* Test the IOStatisticStore implementation.
*/
public class TestIOStatisticsStore extends AbstractHadoopTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestIOStatisticsStore.class);
private static final String COUNT = "count";
private static final String GAUGE = "gauge";
private static final String MIN = "min";
private static final String MAX = "max";
private static final String MEAN = "mean";
public static final String UNKNOWN = "unknown";
private IOStatisticsStore stats;
@Before
public void setup() {
stats = iostatisticsStore()
.withCounters(COUNT)
.withGauges(GAUGE)
.withMinimums(MIN)
.withMaximums(MAX)
.withMeanStatistics(MEAN)
.build();
}
@After
public void teardown() {
LOG.info("stats {}", stats);
}
/**
* Gauges go up and down.
*/
@Test
public void testGauges() throws Throwable {
stats.setGauge(GAUGE, 1);
verifyStatisticGaugeValue(stats, GAUGE, 1);
stats.incrementGauge(GAUGE, 1);
verifyStatisticGaugeValue(stats, GAUGE, 2);
stats.setGauge(GAUGE, -1);
verifyStatisticGaugeValue(stats, GAUGE, -1);
Assertions.assertThat(stats.incrementGauge(GAUGE, -1))
.isEqualTo(-2);
verifyStatisticGaugeValue(stats, GAUGE, -2);
Assertions.assertThat(stats.getGaugeReference(GAUGE).get())
.isEqualTo(-2);
stats.setGauge(UNKNOWN, 1);
Assertions.assertThat(stats.incrementGauge(UNKNOWN, 1))
.isEqualTo(0);
}
@Test
public void testMinimums() throws Throwable {
stats.setMinimum(MIN, 100);
verifyStatisticMinimumValue(stats, MIN, 100);
stats.setMinimum(MIN, 100);
// will do nothing as it is higher
stats.addMinimumSample(MIN, 200);
verifyStatisticMinimumValue(stats, MIN, 100);
stats.addMinimumSample(MIN, 10);
verifyStatisticMinimumValue(stats, MIN, 10);
stats.setMinimum(UNKNOWN, 100);
stats.addMinimumSample(UNKNOWN, 200);
}
@Test
public void testMaximums() throws Throwable {
stats.setMaximum(MAX, 100);
verifyStatisticMaximumValue(stats, MAX, 100);
stats.setMaximum(MAX, 100);
stats.addMaximumSample(MAX, 200);
verifyStatisticMaximumValue(stats, MAX, 200);
stats.addMaximumSample(MAX, 10);
verifyStatisticMaximumValue(stats, MAX, 200);
stats.setMaximum(UNKNOWN, 100);
stats.addMaximumSample(UNKNOWN, 200);
}
@Test
public void testMeans() throws Throwable {
stats.setMeanStatistic(MEAN,
new MeanStatistic(1, 1));
assertThatStatisticMeanMatches(stats, MEAN, 1, 1)
.matches(p -> p.mean() == 1, "mean");
stats.addMeanStatisticSample(MEAN, 9);
assertThatStatisticMeanMatches(stats, MEAN, 2, 10)
.matches(p -> p.mean() == 5, "mean");
}
@Test
public void testRoundTrip() throws Throwable {
JsonSerialization<IOStatisticsSnapshot> serializer
= IOStatisticsSnapshot.serializer();
stats.incrementCounter(COUNT);
stats.setGauge(GAUGE, -1);
stats.addMaximumSample(MAX, 200);
stats.addMinimumSample(MIN, -100);
stats.addMeanStatisticSample(MEAN, 1);
stats.addMeanStatisticSample(MEAN, 9);
String json = serializer.toJson(snapshotIOStatistics(stats));
LOG.info("serialized form\n{}", json);
IOStatisticsSnapshot deser = serializer.fromJson(json);
LOG.info("deserialized {}", deser);
verifyStatisticCounterValue(deser, COUNT, 1L);
verifyStatisticGaugeValue(deser, GAUGE, -1);
verifyStatisticMaximumValue(deser, MAX, 200);
verifyStatisticMinimumValue(deser, MIN, -100);
assertThatStatisticMeanMatches(deser, MEAN, 2, 10)
.matches(p -> p.mean() == 5, "mean");
}
@Test
public void testUnknownCounter() throws Throwable {
Assertions.assertThat(stats.incrementCounter("unknown", -10))
.isEqualTo(0);
}
@Test
public void testNegativeCounterIncrementIgnored() throws Throwable {
Assertions.assertThat(stats.incrementCounter(COUNT, 2))
.isEqualTo(2);
Assertions.assertThat(stats.incrementCounter(COUNT, -10))
.isEqualTo(2);
}
}

View File

@ -0,0 +1,219 @@
/*
* 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.statistics;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.apache.hadoop.util.JsonSerialization;
/**
* Test the {@link MeanStatistic} class.
*/
public class TestMeanStatistic extends AbstractHadoopTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestMeanStatistic.class);
private static final int TEN = 10;
private static final double ZEROD = 0.0d;
private static final double TEND = 10.0d;
private final MeanStatistic empty = new MeanStatistic(0, 0);
private final MeanStatistic tenFromOne = new MeanStatistic(1, TEN);
private final MeanStatistic tenFromTen = new MeanStatistic(TEN, TEN);
@Test
public void testEmptiness() throws Throwable {
Assertions.assertThat(empty)
.matches(MeanStatistic::isEmpty, "is empty")
.isEqualTo(new MeanStatistic(0, TEN))
.isEqualTo(new MeanStatistic())
.isNotEqualTo(tenFromOne);
Assertions.assertThat(empty.mean())
.isEqualTo(ZEROD);
Assertions.assertThat(empty.toString())
.contains("0.0");
}
@Test
public void testTenFromOne() throws Throwable {
Assertions.assertThat(tenFromOne)
.matches(p -> !p.isEmpty(), "is not empty")
.isEqualTo(tenFromOne)
.isNotEqualTo(tenFromTen);
Assertions.assertThat(tenFromOne.mean())
.isEqualTo(TEND);
}
@Test
public void testNegativeSamplesAreEmpty() throws Throwable {
MeanStatistic stat = new MeanStatistic(-10, 1);
Assertions.assertThat(stat)
.describedAs("stat with negative samples")
.matches(MeanStatistic::isEmpty, "is empty")
.isEqualTo(empty)
.extracting(MeanStatistic::mean)
.isEqualTo(ZEROD);
Assertions.assertThat(stat.toString())
.contains("0.0");
}
@Test
public void testCopyNonEmpty() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
Assertions.assertThat(stat)
.describedAs("copy of " + tenFromOne)
.isEqualTo(tenFromOne)
.isNotSameAs(tenFromOne);
}
@Test
public void testCopyEmpty() throws Throwable {
MeanStatistic stat = empty.copy();
Assertions.assertThat(stat)
.describedAs("copy of " + empty)
.isEqualTo(empty)
.isNotSameAs(empty);
}
@Test
public void testDoubleSamples() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
Assertions.assertThat(stat.add(tenFromOne))
.isEqualTo(new MeanStatistic(2, 20))
.extracting(MeanStatistic::mean)
.isEqualTo(TEND);
}
@Test
public void testAddEmptyR() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
Assertions.assertThat(stat.add(empty))
.isEqualTo(tenFromOne);
}
@Test
public void testAddEmptyL() throws Throwable {
MeanStatistic stat = empty.copy();
Assertions.assertThat(stat.add(tenFromOne))
.isEqualTo(tenFromOne);
}
@Test
public void testAddEmptyLR() throws Throwable {
MeanStatistic stat = empty.copy();
Assertions.assertThat(stat.add(empty))
.isEqualTo(empty);
}
@Test
public void testAddSampleToEmpty() throws Throwable {
MeanStatistic stat = empty.copy();
stat.addSample(TEN);
Assertions.assertThat(stat)
.isEqualTo(tenFromOne);
}
@Test
public void testAddZeroValueSamples() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
for (int i = 0; i < 9; i++) {
stat.addSample(0);
}
Assertions.assertThat(stat)
.isEqualTo(tenFromTen);
}
@Test
public void testSetSamples() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
stat.setSamples(10);
Assertions.assertThat(stat)
.isEqualTo(tenFromTen);
}
@Test
public void testSetSums() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
stat.setSum(100);
stat.setSamples(20);
Assertions.assertThat(stat)
.isEqualTo(new MeanStatistic(20, 100))
.extracting(MeanStatistic::mean)
.isEqualTo(5.0d);
}
@Test
public void testSetNegativeSamplesMakesEmpty() throws Throwable {
MeanStatistic stat = tenFromOne.copy();
stat.setSamples(-3);
Assertions.assertThat(stat)
.isEqualTo(empty);
}
@Test
public void testJsonRoundTrip() throws Throwable {
JsonSerialization<MeanStatistic> serializer = serializer();
String json = serializer.toJson(tenFromTen);
LOG.info("serialized form\n{}", json);
Assertions.assertThat(json)
.describedAs("JSON form of %s", tenFromTen)
.doesNotContain("empty")
.doesNotContain("mean");
MeanStatistic deser = serializer.fromJson(json);
LOG.info("deserialized {}", deser);
Assertions.assertThat(deser)
.isEqualTo(tenFromTen);
}
/**
* negative sample counts in the json convert the stat to being empty.
*/
@Test
public void testHandleMaliciousStat() throws Throwable {
String json = "{\n"
+ " \"sum\" : 10,\n"
+ " \"samples\" : -10\n"
+ "}";
JsonSerialization<MeanStatistic> serializer = serializer();
MeanStatistic deser = serializer.fromJson(json);
LOG.info("deserialized {}", deser);
Assertions.assertThat(deser)
.isEqualTo(empty);
}
/**
* Get a JSON serializer.
* @return a serializer.
*/
public static JsonSerialization<MeanStatistic> serializer() {
return new JsonSerialization<>(MeanStatistic.class, true, true);
}
}

View File

@ -0,0 +1,469 @@
/*
* 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.functional;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.util.functional.RemoteIterators.*;
import static org.assertj.core.api.Assertions.assertThat;
/**
* Test for {@link RemoteIterators}.
*
*/
public class TestRemoteIterators extends AbstractHadoopTestBase {
private static final Logger LOG = LoggerFactory.getLogger(
TestRemoteIterators.class);
private static final String[] DATA = {"a", "b", "c"};
/** Counter for lambda-expressions. */
private int counter;
@Test
public void testIterateArray() throws Throwable {
verifyInvoked(remoteIteratorFromArray(DATA), DATA.length,
(s) -> LOG.info(s));
}
@Test
public void testIterateArrayMapped() throws Throwable {
verifyInvoked(
mappingRemoteIterator(
remoteIteratorFromArray(DATA),
(d) -> {
counter += d.length();
return d;
}),
DATA.length,
this::log);
assertCounterValue(3);
}
public void log(Object o) {
LOG.info("{}", o);
}
/**
* Singleton is iterated through once.
* The toString() call is passed through.
*/
@Test
public void testSingleton() throws Throwable {
StringBuffer result = new StringBuffer();
String name = "singleton";
RemoteIterator<String> it = remoteIteratorFromSingleton(name);
assertStringValueContains(it, "SingletonIterator");
assertStringValueContains(it, name);
verifyInvoked(
it,
1,
(s) -> result.append(s));
assertThat(result.toString())
.isEqualTo(name);
}
@Test
public void testSingletonNotClosed() throws Throwable {
CloseCounter closeCounter = new CloseCounter();
RemoteIterator<CloseCounter> it = remoteIteratorFromSingleton(closeCounter);
verifyInvoked(it, 1, this::log);
close(it);
closeCounter.assertCloseCount(0);
}
/**
* A null singleton is not an error.
*/
@Test
public void testNullSingleton() throws Throwable {
verifyInvoked(remoteIteratorFromSingleton(null), 0, this::log);
}
/**
* If you create a singleton iterator and it is an IOStatisticsSource,
* then that is the statistics which can be extracted from the
* iterator.
*/
@Test
public void testSingletonStats() throws Throwable {
IOStatsInstance singleton = new IOStatsInstance();
RemoteIterator<IOStatsInstance> it
= remoteIteratorFromSingleton(singleton);
extractStatistics(it);
}
/**
* The mapping remote iterator passes IOStatistics
* calls down.
*/
@Test
public void testMappedSingletonStats() throws Throwable {
IOStatsInstance singleton = new IOStatsInstance();
RemoteIterator<String> it
= mappingRemoteIterator(remoteIteratorFromSingleton(singleton),
Object::toString);
verifyInvoked(it, 1, this::log);
extractStatistics(it);
}
/**
* Close() calls are passed through.
*/
@Test
public void testClosePassthrough() throws Throwable {
CountdownRemoteIterator countdown = new CountdownRemoteIterator(0);
RemoteIterator<Integer> it = mappingRemoteIterator(
countdown,
i -> i);
verifyInvoked(it, 0, this::log);
// the foreach() operation called close()
countdown.assertCloseCount(1);
extractStatistics(countdown);
((Closeable)it).close();
countdown.assertCloseCount(1);
}
@Test
public void testMapping() throws Throwable {
CountdownRemoteIterator countdown = new CountdownRemoteIterator(100);
RemoteIterator<Integer> it = mappingRemoteIterator(
countdown,
i -> i);
verifyInvoked(it, 100, c -> counter++);
assertCounterValue(100);
extractStatistics(it);
assertStringValueContains(it, "CountdownRemoteIterator");
close(it);
countdown.assertCloseCount(1);
}
@Test
public void testFiltering() throws Throwable {
CountdownRemoteIterator countdown = new CountdownRemoteIterator(100);
// only even numbers are passed through
RemoteIterator<Integer> it = filteringRemoteIterator(
countdown,
i -> (i % 2) == 0);
verifyInvoked(it, 50, c -> counter++);
assertCounterValue(50);
extractStatistics(it);
close(it);
countdown.assertCloseCount(1);
}
/**
* A filter which accepts nothing results in
* an empty iteration.
*/
@Test
public void testFilterNoneAccepted() throws Throwable {
// nothing gets through
RemoteIterator<Integer> it = filteringRemoteIterator(
new CountdownRemoteIterator(100),
i -> false);
verifyInvoked(it, 0, c -> counter++);
assertCounterValue(0);
extractStatistics(it);
}
@Test
public void testFilterAllAccepted() throws Throwable {
// nothing gets through
RemoteIterator<Integer> it = filteringRemoteIterator(
new CountdownRemoteIterator(100),
i -> true);
verifyInvoked(it, 100, c -> counter++);
assertStringValueContains(it, "CountdownRemoteIterator");
}
@Test
public void testJavaIteratorSupport() throws Throwable {
CountdownIterator countdownIterator = new CountdownIterator(100);
RemoteIterator<Integer> it = remoteIteratorFromIterator(
countdownIterator);
verifyInvoked(it, 100, c -> counter++);
assertStringValueContains(it, "CountdownIterator");
extractStatistics(it);
close(it);
countdownIterator.assertCloseCount(1);
}
@Test
public void testJavaIterableSupport() throws Throwable {
CountdownIterable countdown = new CountdownIterable(100);
RemoteIterator<Integer> it = remoteIteratorFromIterable(
countdown);
verifyInvoked(it, 100, c -> counter++);
assertStringValueContains(it, "CountdownIterator");
extractStatistics(it);
// close the iterator
close(it);
countdown.assertCloseCount(0);
// and a new iterator can be crated
verifyInvoked(remoteIteratorFromIterable(countdown),
100, c -> counter++);
}
/**
* If a RemoteIterator is constructed from an iterable
* and that is to be closed, we close it.
*/
@Test
public void testJavaIterableClose() throws Throwable {
CountdownIterable countdown = new CountdownIterable(100);
RemoteIterator<Integer> it = closingRemoteIterator(
remoteIteratorFromIterable(countdown),
countdown);
verifyInvoked(it, 100, c -> counter++);
assertStringValueContains(it, "CountdownIterator");
extractStatistics(it);
// verify the iterator was self closed in hasNext()
countdown.assertCloseCount(1);
// explicitly close the iterator
close(it);
countdown.assertCloseCount(1);
// and a new iterator cannot be created
intercept(IllegalStateException.class, () ->
remoteIteratorFromIterable(countdown));
}
/**
* If a RemoteIterator is constructed from an iterable
* and that is to be closed, we close it.
*/
@SuppressWarnings("InfiniteLoopStatement")
@Test
public void testJavaIterableCloseInNextLoop() throws Throwable {
CountdownIterable countdown = new CountdownIterable(100);
RemoteIterator<Integer> it = closingRemoteIterator(
remoteIteratorFromIterable(countdown),
countdown);
try {
while(true) {
it.next();
}
} catch (NoSuchElementException expected) {
}
// verify the iterator was self closed in next()
countdown.assertCloseCount(1);
}
/**
* assert that the string value of an object contains the
* expected text.
* @param o object
* @param expected expected text
*/
protected void assertStringValueContains(
final Object o,
final String expected) {
assertThat(o.toString())
.describedAs("Object string value")
.contains(expected);
}
/**
* Assert that the counter field is at a specific value.
* @param expected counter
*/
protected void assertCounterValue(final int expected) {
assertThat(counter)
.describedAs("Counter value")
.isEqualTo(expected);
}
/**
* Verify that the iteration completes with a given size.
* @param it iterator
* @param <T> type.
* @param length expected size
* @param consumer consumer
*/
protected <T> void verifyInvoked(final RemoteIterator<T> it,
int length,
ConsumerRaisingIOE<T> consumer)
throws IOException {
assertThat(foreach(it, consumer))
.describedAs("Scan through iterator %s", it)
.isEqualTo(length);
}
/**
* Close an iterator if it is iterable.
* @param it iterator
* @param <T> type.
*/
private <T> void close(final RemoteIterator<T> it) throws IOException {
if (it instanceof Closeable) {
((Closeable) it).close();
}
}
/**
* Class whose close() call increments a counter.
*/
private static class CloseCounter extends
IOStatsInstance implements Closeable {
private int closeCount;
@Override
public void close() throws IOException {
closeCount++;
LOG.info("close ${}", closeCount);
}
public int getCloseCount() {
return closeCount;
}
public void reset() {
closeCount = 0;
}
public void assertCloseCount(int expected) {
assertThat(closeCount)
.describedAs("Close count")
.isEqualTo(expected);
}
}
/**
* Simple class to implement IOStatistics.
*/
private static class IOStatsInstance implements IOStatisticsSource {
private IOStatisticsSnapshot stats = new IOStatisticsSnapshot();
@Override
public IOStatistics getIOStatistics() {
return stats;
}
}
/**
* Iterator which counts down.
*/
private static final class CountdownRemoteIterator extends CloseCounter
implements RemoteIterator<Integer> {
private int limit;
private CountdownRemoteIterator(final int limit) {
this.limit = limit;
}
@Override
public boolean hasNext() throws IOException {
return limit > 0;
}
@Override
public Integer next() throws IOException {
return limit--;
}
@Override
public String toString() {
return "CountdownRemoteIterator{" +
"limit=" + limit +
'}';
}
}
/**
* Iterator which counts down.
*/
private static final class CountdownIterator extends CloseCounter
implements Iterator<Integer> {
private int limit;
private CountdownIterator(final int limit) {
this.limit = limit;
}
@Override
public boolean hasNext() {
return limit > 0;
}
@Override
public Integer next() {
if (!hasNext()) {
throw new NoSuchElementException("limit reached");
}
return limit--;
}
@Override
public String toString() {
return "CountdownIterator{" +
"limit=" + limit +
'}';
}
}
/**
* Iterable for countdown iterators.
* Once closed, calls to iterator() raise an exception.
*/
private static final class CountdownIterable extends CloseCounter
implements Iterable<Integer> {
private int limit;
private CountdownIterable(final int limit) {
this.limit = limit;
}
@Override
public Iterator<Integer> iterator() {
Preconditions.checkState(getCloseCount() == 0);
return new CountdownIterator(limit);
}
}
}