Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
19 commits
Select commit Hold shift + click to select a range
2655a61
HADOOP-17461. Thread-level IOStatistics in S3A
mehakmeet May 24, 2022
471379f
HADOOP-17461. IOStatisticsContext in S3ABlockOutputStream
mehakmeet May 26, 2022
9f5631b
HADOOP-17461. steve's review comments
mehakmeet Jun 21, 2022
6cb9a81
HADOOP-17461. changing the property key
mehakmeet Jun 22, 2022
bcb4c25
HADOOP-18242. checkstyle + merge conflicts
mehakmeet Jun 23, 2022
232d6b9
HADOOP-17461. resetting stats + review comments
Jul 11, 2022
82bc205
HADOOP-17461. review comments + test fixes
Jul 12, 2022
c2dcad1
HADOOP-17461. changes required for spark wiring.
Jul 13, 2022
7344b4e
HADOOP-17461. IOStatisticsContext committer integration
steveloughran Jul 14, 2022
bf84de6
HADOOP-17461. IOStatisticsContext committer integration
steveloughran Jul 15, 2022
f54ad18
HADOOP-17461. javadocs and checkstyles
Jul 19, 2022
6f7e72e
HADOOP-17461. moving IOStatisticsContext in a different package
Jul 20, 2022
7b3c335
HADOOP-17461. making committer collection of context iostats optional.
steveloughran Jul 20, 2022
359206d
HADOOP-17461. TaskPool automatically propagates context
steveloughran Jul 21, 2022
81c0480
HADOOP-17461. adding stream capability and enabling iocontext for test
Jul 25, 2022
cdc9915
HADOOP-17461. review comments
Jul 25, 2022
de12dbe
HADOOP-17461. RawLocal streams capabilties and checkstyle
mehakmeet Jul 26, 2022
2ebe82e
Revert "HADOOP-17461. RawLocal streams capabilties and checkstyle"
mehakmeet Jul 26, 2022
0db1a76
HADOOP-17461. RawLocalFileSystem streams are IOStatisticsContext aware
steveloughran Jul 25, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -471,4 +471,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