HADOOP-17670. S3AFS and ABFS to log IOStats at DEBUG mode or optionally at INFO level in close() (#2963)

When the S3A and ABFS filesystems are closed,
their IOStatistics are logged at debug in the log:

org.apache.hadoop.fs.statistics.IOStatisticsLogging

Set `fs.iostatistics.logging.level` to `info` for the statistics
to be logged at info. (also: `warn` or `error` for even higher
log levels).

Contributed by: Mehakmeet Singh

Change-Id: I56d44ad89fc1c0dd4baf701681834e7fd96c544f
This commit is contained in:
Mehakmeet Singh 2021-05-24 17:32:11 +05:30 committed by Steve Loughran
parent 745a2f5ec3
commit a786847b8f
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
6 changed files with 87 additions and 7 deletions

View File

@ -435,4 +435,28 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
"hadoop.metrics.jvm.use-thread-mxbean";
public static final boolean HADOOP_METRICS_JVM_USE_THREAD_MXBEAN_DEFAULT =
false;
/** logging level for IOStatistics (debug or info). */
public static final String IOSTATISTICS_LOGGING_LEVEL
= "fs.iostatistics.logging.level";
/** DEBUG logging level for IOStatistics logging. */
public static final String IOSTATISTICS_LOGGING_LEVEL_DEBUG
= "debug";
/** WARN logging level for IOStatistics logging. */
public static final String IOSTATISTICS_LOGGING_LEVEL_WARN
= "warn";
/** ERROR logging level for IOStatistics logging. */
public static final String IOSTATISTICS_LOGGING_LEVEL_ERROR
= "error";
/** INFO logging level for IOStatistics logging. */
public static final String IOSTATISTICS_LOGGING_LEVEL_INFO
= "info";
/** Default value for IOStatistics logging level. */
public static final String IOSTATISTICS_LOGGING_LEVEL_DEFAULT
= IOSTATISTICS_LOGGING_LEVEL_DEBUG;
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.fs.statistics;
import javax.annotation.Nullable;
import java.util.Locale;
import java.util.Map;
import java.util.TreeMap;
import java.util.function.Predicate;
@ -30,6 +31,9 @@ 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.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_ERROR;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_WARN;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
@ -249,6 +253,33 @@ public final class IOStatisticsLogging {
logIOStatisticsAtDebug(LOG, message, source);
}
/**
* A method to log IOStatistics from a source at different levels.
*
* @param log Logger for logging.
* @param level LOG level.
* @param source Source to LOG.
*/
public static void logIOStatisticsAtLevel(Logger log, String level,
Object source) {
IOStatistics stats = retrieveIOStatistics(source);
if (stats != null) {
switch (level.toLowerCase(Locale.US)) {
case IOSTATISTICS_LOGGING_LEVEL_INFO:
LOG.info("IOStatistics: {}", ioStatisticsToPrettyString(stats));
break;
case IOSTATISTICS_LOGGING_LEVEL_ERROR:
LOG.error("IOStatistics: {}", ioStatisticsToPrettyString(stats));
break;
case IOSTATISTICS_LOGGING_LEVEL_WARN:
LOG.warn("IOStatistics: {}", ioStatisticsToPrettyString(stats));
break;
default:
logIOStatisticsAtDebug(log, "IOStatistics: {}", source);
}
}
}
/**
* On demand stringifier.
* <p>

View File

@ -177,6 +177,8 @@ import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT;
import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
import static org.apache.hadoop.fs.s3a.Constants.*;
@ -199,6 +201,7 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory;
@ -3537,6 +3540,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
}
isClosed = true;
LOG.debug("Filesystem {} is closed", uri);
if (getConf() != null) {
String iostatisticsLoggingLevel =
getConf().getTrimmed(IOSTATISTICS_LOGGING_LEVEL,
IOSTATISTICS_LOGGING_LEVEL_DEFAULT);
logIOStatisticsAtLevel(LOG, iostatisticsLoggingLevel, getIOStatistics());
}
try {
super.close();
} finally {

View File

@ -41,6 +41,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.slf4j.Logger;
@ -84,7 +85,6 @@ import org.apache.hadoop.fs.permission.FsAction;
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.io.IOUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.UserGroupInformation;
@ -93,9 +93,11 @@ import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.LambdaUtils;
import org.apache.hadoop.util.Progressable;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT;
import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
/**
* A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
@ -166,8 +168,6 @@ public class AzureBlobFileSystem extends FileSystem
sb.append("uri=").append(uri);
sb.append(", user='").append(abfsStore.getUser()).append('\'');
sb.append(", primaryUserGroup='").append(abfsStore.getPrimaryGroup()).append('\'');
sb.append(", \nIOStatistics: {").append(ioStatisticsToString(getIOStatistics()));
sb.append("}");
sb.append('}');
return sb.toString();
}
@ -490,6 +490,12 @@ public class AzureBlobFileSystem extends FileSystem
// does all the delete-on-exit calls, and may be slow.
super.close();
LOG.debug("AzureBlobFileSystem.close");
if (getConf() != null) {
String iostatisticsLoggingLevel =
getConf().getTrimmed(IOSTATISTICS_LOGGING_LEVEL,
IOSTATISTICS_LOGGING_LEVEL_DEFAULT);
logIOStatisticsAtLevel(LOG, iostatisticsLoggingLevel, getIOStatistics());
}
IOUtils.cleanupWithLogger(LOG, abfsStore, delegationTokenManager);
this.isClosed = true;
if (LOG.isDebugEnabled()) {

View File

@ -330,9 +330,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
threadExecutor.shutdownNow();
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Closing AbfsOutputStream ", toString());
}
LOG.debug("Closing AbfsOutputStream : {}", this);
}
private synchronized void flushInternal(boolean isClose) throws IOException {

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.fs.azurebfs;
import java.io.IOException;
import java.util.Map;
import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.fs.Path;
@ -28,6 +29,9 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsCounters;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.statistics.IOStatistics;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO;
/**
* Tests AzureBlobFileSystem Statistics.
*/
@ -38,6 +42,14 @@ public class ITestAbfsStatistics extends AbstractAbfsIntegrationTest {
public ITestAbfsStatistics() throws Exception {
}
@Before
public void setUp() throws Exception {
super.setup();
// Setting IOStats to INFO level, to see the IOStats after close().
getFileSystem().getConf().set(IOSTATISTICS_LOGGING_LEVEL,
IOSTATISTICS_LOGGING_LEVEL_INFO);
}
/**
* Testing the initial value of statistics.
*/