Skip to content

Commit

Permalink
HADOOP-17461. Collect thread-level IOStatistics. (#4352)
Browse files Browse the repository at this point in the history
This adds a thread-level collector of IOStatistics, IOStatisticsContext,
which can be:
* Retrieved for a thread and cached for access from other
  threads.
* reset() to record new statistics.
* Queried for live statistics through the
  IOStatisticsSource.getIOStatistics() method.
* Queries for a statistics aggregator for use in instrumented
  classes.
* Asked to create a serializable copy in snapshot()

The goal is to make it possible for applications with multiple
threads performing different work items simultaneously
to be able to collect statistics on the individual threads,
and so generate aggregate reports on the total work performed
for a specific job, query or similar unit of work.

Some changes in IOStatistics-gathering classes are needed for 
this feature
* Caching the active context's aggregator in the object's
  constructor
* Updating it in close()

Slightly more work is needed in multithreaded code,
such as the S3A committers, which collect statistics across
all threads used in task and job commit operations.

Currently the IOStatisticsContext-aware classes are:
* The S3A input stream, output stream and list iterators.
* RawLocalFileSystem's input and output streams.
* The S3A committers.
* The TaskPool class in hadoop-common, which propagates
  the active context into scheduled worker threads.

Collection of statistics in the IOStatisticsContext
is disabled process-wide by default until the feature 
is considered stable.

To enable the collection, set the option
fs.thread.level.iostatistics.enabled
to "true" in core-site.xml;
	
Contributed by Mehakmeet Singh and Steve Loughran
  • Loading branch information
mehakmeet committed Jul 27, 2022
1 parent 0c12873 commit 6483021
Show file tree
Hide file tree
Showing 26 changed files with 1,459 additions and 80 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -472,4 +472,18 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
* default hadoop temp dir on local system: {@value}.
*/
public static final String HADOOP_TMP_DIR = "hadoop.tmp.dir";

/**
* Thread-level IOStats Support.
* {@value}
*/
public static final String THREAD_LEVEL_IOSTATISTICS_ENABLED =
"fs.thread.level.iostatistics.enabled";

/**
* Default value for Thread-level IOStats Support is true.
*/
public static final boolean THREAD_LEVEL_IOSTATISTICS_ENABLED_DEFAULT =
true;

}
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
import org.apache.hadoop.fs.statistics.IOStatisticsContext;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.BufferedIOStatisticsOutputStream;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
Expand Down Expand Up @@ -156,11 +158,19 @@ class LocalFSFileInputStream extends FSInputStream implements
/** Reference to the bytes read counter for slightly faster counting. */
private final AtomicLong bytesRead;

/**
* Thread level IOStatistics aggregator to update in close().
*/
private final IOStatisticsAggregator
ioStatisticsAggregator;

public LocalFSFileInputStream(Path f) throws IOException {
name = pathToFile(f);
fis = new FileInputStream(name);
bytesRead = ioStatistics.getCounterReference(
STREAM_READ_BYTES);
ioStatisticsAggregator =
IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator();
}

@Override
Expand Down Expand Up @@ -193,9 +203,13 @@ public boolean seekToNewSource(long targetPos) throws IOException {

@Override
public void close() throws IOException {
fis.close();
if (asyncChannel != null) {
asyncChannel.close();
try {
fis.close();
if (asyncChannel != null) {
asyncChannel.close();
}
} finally {
ioStatisticsAggregator.aggregate(ioStatistics);
}
}

Expand Down Expand Up @@ -278,6 +292,7 @@ public boolean hasCapability(String capability) {
// new capabilities.
switch (capability.toLowerCase(Locale.ENGLISH)) {
case StreamCapabilities.IOSTATISTICS:
case StreamCapabilities.IOSTATISTICS_CONTEXT:
case StreamCapabilities.VECTOREDIO:
return true;
default:
Expand Down Expand Up @@ -407,9 +422,19 @@ final class LocalFSFileOutputStream extends OutputStream implements
STREAM_WRITE_EXCEPTIONS)
.build();

/**
* Thread level IOStatistics aggregator to update in close().
*/
private final IOStatisticsAggregator
ioStatisticsAggregator;

private LocalFSFileOutputStream(Path f, boolean append,
FsPermission permission) throws IOException {
File file = pathToFile(f);
// store the aggregator before attempting any IO.
ioStatisticsAggregator =
IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator();

if (!append && permission == null) {
permission = FsPermission.getFileDefault();
}
Expand All @@ -436,10 +461,17 @@ private LocalFSFileOutputStream(Path f, boolean append,
}

/*
* Just forward to the fos
* Close the fos; update the IOStatisticsContext.
*/
@Override
public void close() throws IOException { fos.close(); }
public void close() throws IOException {
try {
fos.close();
} finally {
ioStatisticsAggregator.aggregate(ioStatistics);
}
}

@Override
public void flush() throws IOException { fos.flush(); }
@Override
Expand Down Expand Up @@ -485,6 +517,7 @@ public boolean hasCapability(String capability) {
// new capabilities.
switch (capability.toLowerCase(Locale.ENGLISH)) {
case StreamCapabilities.IOSTATISTICS:
case StreamCapabilities.IOSTATISTICS_CONTEXT:
return true;
default:
return StoreImplementationUtils.isProbeForSyncable(capability);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,12 @@ public interface StreamCapabilities {
*/
String ABORTABLE_STREAM = CommonPathCapabilities.ABORTABLE_STREAM;

/**
* Streams that support IOStatistics context and capture thread-level
* IOStatistics.
*/
String IOSTATISTICS_CONTEXT = "fs.capability.iocontext.supported";

/**
* Capabilities that a stream can support and be queried for.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hadoop.fs.impl;

import java.lang.ref.WeakReference;
import java.util.function.Consumer;
import java.util.function.Function;
import javax.annotation.Nullable;
Expand Down Expand Up @@ -48,7 +49,17 @@ public long currentThreadId() {
}

public V setForCurrentThread(V newVal) {
return put(currentThreadId(), newVal);
long id = currentThreadId();

// if the same object is already in the map, just return it.
WeakReference<V> ref = lookup(id);
// Reference value could be set to null. Thus, ref.get() could return
// null. Should be handled accordingly while using the returned value.
if (ref != null && ref.get() == newVal) {
return ref.get();
}

return put(id, newVal);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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.fs.statistics.impl.IOStatisticsContextIntegration;

/**
* An interface defined to capture thread-level IOStatistics by using per
* thread context.
* <p>
* The aggregator should be collected in their constructor by statistics-generating
* classes to obtain the aggregator to update <i>across all threads</i>.
* <p>
* The {@link #snapshot()} call creates a snapshot of the statistics;
* <p>
* The {@link #reset()} call resets the statistics in the context so
* that later snapshots will get the incremental data.
*/
public interface IOStatisticsContext extends IOStatisticsSource {

/**
* Get the IOStatisticsAggregator for the context.
*
* @return return the aggregator for the context.
*/
IOStatisticsAggregator getAggregator();

/**
* Capture the snapshot of the context's IOStatistics.
*
* @return IOStatisticsSnapshot for the context.
*/
IOStatisticsSnapshot snapshot();

/**
* Get a unique ID for this context, for logging
* purposes.
*
* @return an ID unique for all contexts in this process.
*/
long getID();

/**
* Reset the context's IOStatistics.
*/
void reset();

/**
* Get the context's IOStatisticsContext.
*
* @return instance of IOStatisticsContext for the context.
*/
static IOStatisticsContext getCurrentIOStatisticsContext() {
return IOStatisticsContextIntegration.getCurrentIOStatisticsContext();
}

/**
* Set the IOStatisticsContext for the current thread.
* @param statisticsContext IOStatistics context instance for the
* current thread. If null, the context is reset.
*/
static void setThreadIOStatisticsContext(
IOStatisticsContext statisticsContext) {
IOStatisticsContextIntegration.setThreadIOStatisticsContext(
statisticsContext);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.IOStatisticsAggregator;
import org.apache.hadoop.fs.statistics.IOStatisticsContext;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;

/**
* Empty IOStatistics context which serves no-op for all the operations and
* returns an empty Snapshot if asked.
*
*/
final class EmptyIOStatisticsContextImpl implements IOStatisticsContext {

private static final IOStatisticsContext EMPTY_CONTEXT = new EmptyIOStatisticsContextImpl();

private EmptyIOStatisticsContextImpl() {
}

/**
* Create a new empty snapshot.
* A new one is always created for isolation.
*
* @return a statistics snapshot
*/
@Override
public IOStatisticsSnapshot snapshot() {
return new IOStatisticsSnapshot();
}

@Override
public IOStatisticsAggregator getAggregator() {
return EmptyIOStatisticsStore.getInstance();
}

@Override
public IOStatistics getIOStatistics() {
return EmptyIOStatistics.getInstance();
}

@Override
public void reset() {}

/**
* The ID is always 0.
* As the real context implementation counter starts at 1,
* we are guaranteed to have unique IDs even between them and
* the empty context.
* @return 0
*/
@Override
public long getID() {
return 0;
}

/**
* Get the single instance.
* @return an instance.
*/
static IOStatisticsContext getInstance() {
return EMPTY_CONTEXT;
}
}
Loading

0 comments on commit 6483021

Please sign in to comment.