HADOOP-15552. Move logging APIs over to slf4j in hadoop-tools - Part2. Contributed by Ian Pickering.

This commit is contained in:
Akira Ajisaka 2018-08-16 00:31:59 +09:00
parent f2315f2e9c
commit 3e3963b035
No known key found for this signature in database
GPG Key ID: C1EDBB9CA400FD50
91 changed files with 282 additions and 280 deletions

View File

@ -69,7 +69,7 @@ public void testRenameNonexistentFile() throws Throwable {
} else {
// at least one FS only returns false here, if that is the case
// warn but continue
getLog().warn("Rename returned {} renaming a nonexistent file", renamed);
getLogger().warn("Rename returned {} renaming a nonexistent file", renamed);
assertFalse("Renaming a missing file returned true", renamed);
}
} catch (FileNotFoundException e) {
@ -118,7 +118,7 @@ public void testRenameFileOverExistingFile() throws Throwable {
if (renamed && !renameReturnsFalseOnRenameDestExists) {
//expected an exception
String destDirLS = generateAndLogErrorListing(srcFile, destFile);
getLog().error("dest dir {}", destDirLS);
getLogger().error("dest dir {}", destDirLS);
fail("expected rename(" + srcFile + ", " + destFile + " ) to fail," +
" but got success and destination of " + destDirLS);
}

View File

@ -132,7 +132,7 @@ public void testBlockReadZeroByteFile() throws Throwable {
@Test
public void testSeekReadClosedFile() throws Throwable {
instream = getFileSystem().open(smallSeekFile);
getLog().debug(
getLogger().debug(
"Stream is of type " + instream.getClass().getCanonicalName());
instream.close();
try {

View File

@ -110,7 +110,7 @@ public FileSystem getFileSystem() {
* Get the log of the base class.
* @return a logger
*/
public static Logger getLog() {
public static Logger getLogger() {
return LOG;
}
@ -281,7 +281,7 @@ protected void handleRelaxedException(String action,
* @param e exception raised.
*/
protected void handleExpectedException(Exception e) {
getLog().debug("expected :{}" ,e, e);
getLogger().debug("expected :{}" ,e, e);
}
/**
@ -366,7 +366,7 @@ protected boolean rename(Path src, Path dst) throws IOException {
protected String generateAndLogErrorListing(Path src, Path dst) throws
IOException {
FileSystem fs = getFileSystem();
getLog().error(
getLogger().error(
"src dir " + ContractTestUtils.ls(fs, src.getParent()));
String destDirLS = ContractTestUtils.ls(fs, dst.getParent());
if (fs.isDirectory(dst)) {

View File

@ -177,4 +177,4 @@ public Boolean get() {
}
}, SCAN_WAIT * 1000, 60000);
}
}
}

View File

@ -25,8 +25,8 @@
import java.util.concurrent.ExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
@ -40,7 +40,7 @@
* stream.
*/
public class AliyunOSSInputStream extends FSInputStream {
public static final Log LOG = LogFactory.getLog(AliyunOSSInputStream.class);
public static final Logger LOG = LoggerFactory.getLogger(AliyunOSSInputStream.class);
private final long downloadPartSize;
private AliyunOSSFileSystemStore store;
private final String key;

View File

@ -27,8 +27,8 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.output.FileWriterWithEncoding;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -70,7 +70,7 @@
* {@link HadoopArchiveLogsRunner}.
*/
public class HadoopArchiveLogs implements Tool {
private static final Log LOG = LogFactory.getLog(HadoopArchiveLogs.class);
private static final Logger LOG = LoggerFactory.getLogger(HadoopArchiveLogs.class);
private static final String HELP_OPTION = "help";
private static final String MAX_ELIGIBLE_APPS_OPTION = "maxEligibleApps";

View File

@ -25,8 +25,8 @@
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -47,8 +47,8 @@
* tool via the Distributed Shell. It's not meant to be run directly.
*/
public class HadoopArchiveLogsRunner implements Tool {
private static final Log LOG =
LogFactory.getLog(HadoopArchiveLogsRunner.class);
private static final Logger LOG =
LoggerFactory.getLogger(HadoopArchiveLogsRunner.class);
private static final String APP_ID_OPTION = "appId";
private static final String USER_OPTION = "user";

View File

@ -37,8 +37,8 @@
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.Parser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -82,7 +82,7 @@
*/
public class HadoopArchives implements Tool {
public static final int VERSION = 3;
private static final Log LOG = LogFactory.getLog(HadoopArchives.class);
private static final Logger LOG = LoggerFactory.getLogger(HadoopArchives.class);
private static final String NAME = "har";
private static final String ARCHIVE_NAME = "archiveName";

View File

@ -41,7 +41,7 @@ protected AbstractFSContract createContract(Configuration conf) {
@Override
public void teardown() throws Exception {
getLog().info("FS details {}", getFileSystem());
getLogger().info("FS details {}", getFileSystem());
super.teardown();
}

View File

@ -41,7 +41,7 @@ protected AbstractFSContract createContract(Configuration conf) {
@Override
public void teardown() throws Exception {
getLog().info("FS details {}", getFileSystem());
getLogger().info("FS details {}", getFileSystem());
super.teardown();
}

View File

@ -23,8 +23,8 @@
import java.util.Map.Entry;
import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Mapper;
import org.apache.hadoop.mapred.Reducer;
@ -36,7 +36,7 @@
*/
public abstract class JobBase implements Mapper, Reducer {
public static final Log LOG = LogFactory.getLog("datajoin.job");
public static final Logger LOG = LoggerFactory.getLogger("datajoin.job");
private SortedMap<Object, Long> longCounters = null;

View File

@ -27,8 +27,8 @@
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.tools.util.DistCpUtils;
import org.apache.hadoop.security.Credentials;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.reflect.Constructor;
@ -48,7 +48,7 @@
public abstract class CopyListing extends Configured {
private Credentials credentials;
static final Log LOG = LogFactory.getLog(DistCp.class);
static final Logger LOG = LoggerFactory.getLogger(DistCp.class);
/**
* Build listing function creates the input listing that distcp uses to
* perform the copy.

View File

@ -22,8 +22,8 @@
import java.util.Random;
import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -64,7 +64,7 @@ public class DistCp extends Configured implements Tool {
*/
static final int SHUTDOWN_HOOK_PRIORITY = 30;
static final Log LOG = LogFactory.getLog(DistCp.class);
static final Logger LOG = LoggerFactory.getLogger(DistCp.class);
@VisibleForTesting
DistCpContext context;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -35,7 +35,7 @@
* listing-file by "globbing" all specified source paths (wild-cards and all.)
*/
public class GlobbedCopyListing extends CopyListing {
private static final Log LOG = LogFactory.getLog(GlobbedCopyListing.class);
private static final Logger LOG = LoggerFactory.getLogger(GlobbedCopyListing.class);
private final CopyListing simpleListing;
/**

View File

@ -29,8 +29,8 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import com.google.common.base.Preconditions;
@ -41,7 +41,7 @@
*/
public class OptionsParser {
static final Log LOG = LogFactory.getLog(OptionsParser.class);
static final Logger LOG = LoggerFactory.getLogger(OptionsParser.class);
private static final Options cliOptions = new Options();

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
@ -43,7 +43,7 @@
*/
public class RegexCopyFilter extends CopyFilter {
private static final Log LOG = LogFactory.getLog(RegexCopyFilter.class);
private static final Logger LOG = LoggerFactory.getLogger(RegexCopyFilter.class);
private File filtersFile;
private List<Pattern> filters;
@ -77,7 +77,7 @@ public void initialize() {
LOG.error("An error occurred while attempting to read from " +
filtersFile);
} finally {
IOUtils.cleanup(LOG, reader);
IOUtils.cleanupWithLogger(LOG, reader);
}
}

View File

@ -20,8 +20,8 @@
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
@ -60,7 +60,7 @@
* Note: The SimpleCopyListing doesn't handle wild-cards in the input-paths.
*/
public class SimpleCopyListing extends CopyListing {
private static final Log LOG = LogFactory.getLog(SimpleCopyListing.class);
private static final Logger LOG = LoggerFactory.getLogger(SimpleCopyListing.class);
public static final int DEFAULT_FILE_STATUS_SIZE = 1000;
public static final boolean DEFAULT_RANDOMIZE_FILE_LISTING = true;
@ -309,7 +309,7 @@ protected void doBuildListingWithSnapshotDiff(
fileListWriter.close();
fileListWriter = null;
} finally {
IOUtils.cleanup(LOG, fileListWriter);
IOUtils.cleanupWithLogger(LOG, fileListWriter);
}
}
@ -402,7 +402,7 @@ protected void doBuildListing(SequenceFile.Writer fileListWriter,
LOG.info("Build file listing completed.");
fileListWriter = null;
} finally {
IOUtils.cleanup(LOG, fileListWriter);
IOUtils.cleanupWithLogger(LOG, fileListWriter);
}
}

View File

@ -23,8 +23,8 @@
import java.util.EnumSet;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
@ -74,7 +74,7 @@ static enum FileAction {
OVERWRITE, // Overwrite the whole file
}
private static Log LOG = LogFactory.getLog(CopyMapper.class);
private static Logger LOG = LoggerFactory.getLogger(CopyMapper.class);
private Configuration conf;

View File

@ -23,8 +23,8 @@
import java.io.OutputStream;
import java.util.EnumSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
@ -53,7 +53,7 @@
*/
public class RetriableFileCopyCommand extends RetriableCommand {
private static Log LOG = LogFactory.getLog(RetriableFileCopyCommand.class);
private static Logger LOG = LoggerFactory.getLogger(RetriableFileCopyCommand.class);
private boolean skipCrc = false;
private FileAction action;
@ -297,7 +297,7 @@ long copyBytes(CopyListingFileStatus source2, long sourceOffset,
outStream.close();
outStream = null;
} finally {
IOUtils.cleanup(LOG, outStream, inStream);
IOUtils.cleanupWithLogger(LOG, outStream, inStream);
}
return totalBytesRead;
}

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.mapred;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.IOUtils;
@ -50,8 +50,8 @@
*/
public class UniformSizeInputFormat
extends InputFormat<Text, CopyListingFileStatus> {
private static final Log LOG
= LogFactory.getLog(UniformSizeInputFormat.class);
private static final Logger LOG
= LoggerFactory.getLogger(UniformSizeInputFormat.class);
/**
* Implementation of InputFormat::getSplits(). Returns a list of InputSplits,

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.tools.mapred.lib;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
@ -42,7 +42,7 @@
* consumed.
*/
class DynamicInputChunk<K, V> {
private static Log LOG = LogFactory.getLog(DynamicInputChunk.class);
private static Logger LOG = LoggerFactory.getLogger(DynamicInputChunk.class);
private Path chunkFilePath;
private SequenceFileRecordReader<K, V> reader;
private SequenceFile.Writer writer;
@ -78,7 +78,7 @@ public void write(Text key, CopyListingFileStatus value) throws IOException {
* Closes streams opened to the chunk-file.
*/
public void close() {
IOUtils.cleanup(LOG, reader, writer);
IOUtils.cleanupWithLogger(LOG, reader, writer);
}
/**

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.mapred.lib;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.slf4j.LoggerFactory;
import org.slf4j.Logger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
@ -34,7 +34,7 @@
*/
class DynamicInputChunkContext<K, V> {
private static Log LOG = LogFactory.getLog(DynamicInputChunkContext.class);
private static Logger LOG = LoggerFactory.getLogger(DynamicInputChunkContext.class);
private Configuration configuration;
private Path chunkRootPath = null;
private String chunkFilePrefix;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.mapred.lib;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapreduce.*;
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
import org.apache.hadoop.tools.DistCpConstants;
@ -49,7 +49,7 @@
* performance characteristics.
*/
public class DynamicInputFormat<K, V> extends InputFormat<K, V> {
private static final Log LOG = LogFactory.getLog(DynamicInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(DynamicInputFormat.class);
private static final String CONF_LABEL_LISTING_SPLIT_RATIO
= "mapred.listing.split.ratio";

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.mapred.lib;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.tools.util.DistCpUtils;
import org.apache.hadoop.tools.DistCpConstants;
import org.apache.hadoop.mapreduce.*;
@ -37,7 +37,7 @@
* transparently.
*/
public class DynamicRecordReader<K, V> extends RecordReader<K, V> {
private static final Log LOG = LogFactory.getLog(DynamicRecordReader.class);
private static final Logger LOG = LoggerFactory.getLogger(DynamicRecordReader.class);
private TaskAttemptContext taskAttemptContext;
private Configuration configuration;
private DynamicInputChunk<K, V> chunk;

View File

@ -20,8 +20,8 @@
import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileChecksum;
@ -56,7 +56,7 @@
*/
public class DistCpUtils {
private static final Log LOG = LogFactory.getLog(DistCpUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(DistCpUtils.class);
/**
* Retrieves size of the file at the specified path.

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -34,7 +34,7 @@
* WorkReport{@literal <R>} to the outputQueue.
*/
public class ProducerConsumer<T, R> {
private Log LOG = LogFactory.getLog(ProducerConsumer.class);
private Logger LOG = LoggerFactory.getLogger(ProducerConsumer.class);
private LinkedBlockingQueue<WorkRequest<T>> inputQueue;
private LinkedBlockingQueue<WorkReport<R>> outputQueue;
private ExecutorService executor;

View File

@ -19,8 +19,8 @@
package org.apache.hadoop.tools.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
import org.apache.hadoop.io.retry.RetryPolicies;
@ -35,7 +35,7 @@
*/
public abstract class RetriableCommand {
private static Log LOG = LogFactory.getLog(RetriableCommand.class);
private static Logger LOG = LoggerFactory.getLogger(RetriableCommand.class);
private static final long DELAY_MILLISECONDS = 500;
private static final int MAX_RETRIES = 3;

View File

@ -20,8 +20,8 @@
import static org.mockito.Mockito.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.conf.Configuration;
@ -51,7 +51,7 @@
@RunWith(value = Parameterized.class)
public class TestCopyListing extends SimpleCopyListing {
private static final Log LOG = LogFactory.getLog(TestCopyListing.class);
private static final Logger LOG = LoggerFactory.getLogger(TestCopyListing.class);
private static final Credentials CREDENTIALS = new Credentials();

View File

@ -31,8 +31,8 @@
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
@ -57,8 +57,8 @@
*/
public class TestDistCpSystem {
private static final Log LOG =
LogFactory.getLog(TestDistCpSystem.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestDistCpSystem.class);
@Rule
public Timeout globalTimeout = new Timeout(30000);

View File

@ -18,9 +18,9 @@
package org.apache.hadoop.tools;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.apache.hadoop.fs.viewfs.*;
import org.apache.commons.logging.LogFactory;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -37,7 +37,7 @@
import java.net.URISyntaxException;
public class TestDistCpViewFs {
private static final Log LOG = LogFactory.getLog(TestDistCpViewFs.class);
private static final Logger LOG = LoggerFactory.getLogger(TestDistCpViewFs.class);
private static FileSystem fs;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -37,7 +37,7 @@
public class TestExternalCall {
private static final Log LOG = LogFactory.getLog(TestExternalCall.class);
private static final Logger LOG = LoggerFactory.getLogger(TestExternalCall.class);
private static FileSystem fs;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -40,7 +40,7 @@
import java.util.Map;
public class TestFileBasedCopyListing {
private static final Log LOG = LogFactory.getLog(TestFileBasedCopyListing.class);
private static final Logger LOG = LoggerFactory.getLogger(TestFileBasedCopyListing.class);
private static final Credentials CREDENTIALS = new Credentials();

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -43,7 +43,7 @@
@RunWith(value = Parameterized.class)
public class TestIntegration {
private static final Log LOG = LogFactory.getLog(TestIntegration.class);
private static final Logger LOG = LoggerFactory.getLogger(TestIntegration.class);
private static FileSystem fs;

View File

@ -523,7 +523,7 @@ private void largeFiles(FileSystem srcFS, Path srcDir, FileSystem dstFS,
int fileSizeKb = conf.getInt(SCALE_TEST_DISTCP_FILE_SIZE_KB,
DEFAULT_DISTCP_SIZE_KB);
int fileSizeMb = fileSizeKb / 1024;
getLog().info("{} with file size {}", testName.getMethodName(), fileSizeMb);
getLogger().info("{} with file size {}", testName.getMethodName(), fileSizeMb);
byte[] data1 = dataset((fileSizeMb + 1) * MB, 33, 43);
createFile(srcFS, largeFile1, true, data1);
byte[] data2 = dataset((fileSizeMb + 2) * MB, 43, 53);

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.mapred;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -47,7 +47,7 @@
import static org.apache.hadoop.tools.util.TestDistCpUtils.*;
public class TestCopyCommitter {
private static final Log LOG = LogFactory.getLog(TestCopyCommitter.class);
private static final Logger LOG = LoggerFactory.getLogger(TestCopyCommitter.class);
private static final Random rand = new Random();

View File

@ -27,8 +27,8 @@
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -62,7 +62,7 @@
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
public class TestCopyMapper {
private static final Log LOG = LogFactory.getLog(TestCopyMapper.class);
private static final Logger LOG = LoggerFactory.getLogger(TestCopyMapper.class);
private static List<Path> pathList = new ArrayList<Path>();
private static int nFiles = 0;
private static final int DEFAULT_FILE_SIZE = 1024;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.mapred;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapreduce.*;
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
import org.apache.hadoop.mapreduce.task.JobContextImpl;
@ -32,7 +32,7 @@
import java.io.IOException;
public class TestCopyOutputFormat {
private static final Log LOG = LogFactory.getLog(TestCopyOutputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(TestCopyOutputFormat.class);
@Test
public void testSetCommitDirectory() {

View File

@ -21,8 +21,8 @@
import org.apache.hadoop.tools.DistCpConstants;
import org.apache.hadoop.tools.DistCpContext;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -46,7 +46,7 @@
import java.util.List;
public class TestDynamicInputFormat {
private static final Log LOG = LogFactory.getLog(TestDynamicInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(TestDynamicInputFormat.class);
private static MiniDFSCluster cluster;
private static final int N_FILES = 1000;
private static final int NUM_SPLITS = 7;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -50,7 +50,7 @@
import static org.junit.Assert.assertTrue;
public class TestDistCpUtils {
private static final Log LOG = LogFactory.getLog(TestDistCpUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(TestDistCpUtils.class);
private static final Configuration config = new Configuration();
private static MiniDFSCluster cluster;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.tools.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.IOUtils;
import org.junit.Assert;
import org.junit.Test;
@ -27,7 +27,7 @@
import java.io.*;
public class TestThrottledInputStream {
private static final Log LOG = LogFactory.getLog(TestThrottledInputStream.class);
private static final Logger LOG = LoggerFactory.getLogger(TestThrottledInputStream.class);
private static final int BUFF_SIZE = 1024;
private enum CB {ONE_C, BUFFER, BUFF_OFFSET}
@ -89,7 +89,7 @@ private long copyAndAssert(File tmpFile, File outFile,
copyByteByByte(in, out);
}
LOG.info(in);
LOG.info("{}", in);
bandwidth = in.getBytesPerSec();
Assert.assertEquals(in.getTotalBytesRead(), tmpFile.length());
Assert.assertTrue(in.getBytesPerSec() > maxBandwidth / (factor * 1.2));

View File

@ -27,8 +27,8 @@
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -40,7 +40,7 @@
* An abstract class for distributed tool for file related operations.
*/
abstract class DistTool implements org.apache.hadoop.util.Tool {
protected static final Log LOG = LogFactory.getLog(DistTool.class);
protected static final Logger LOG = LoggerFactory.getLogger(DistTool.class);
protected JobConf jobconf;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.lang3.time.FastDateFormat;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
@ -40,7 +40,7 @@
* addresses are also recorded in the summary.
*/
class ClusterSummarizer implements StatListener<ClusterStats> {
static final Log LOG = LogFactory.getLog(ClusterSummarizer.class);
static final Logger LOG = LoggerFactory.getLogger(ClusterSummarizer.class);
private int numBlacklistedTrackers;
private int numActiveTrackers;

View File

@ -25,8 +25,8 @@
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -58,7 +58,7 @@
* This is a utility class for all the compression related modules.
*/
class CompressionEmulationUtil {
static final Log LOG = LogFactory.getLog(CompressionEmulationUtil.class);
static final Logger LOG = LoggerFactory.getLogger(CompressionEmulationUtil.class);
/**
* Enable compression usage in GridMix runs.

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -83,8 +83,8 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
class DistributedCacheEmulator {
private static final Log LOG =
LogFactory.getLog(DistributedCacheEmulator.class);
private static final Logger LOG =
LoggerFactory.getLogger(DistributedCacheEmulator.class);
static final long AVG_BYTES_PER_MAP = 128 * 1024 * 1024L;// 128MB

View File

@ -22,14 +22,14 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Echos the UGI offered.
*/
public class EchoUserResolver implements UserResolver {
public static final Log LOG = LogFactory.getLog(Gridmix.class);
public static final Logger LOG = LoggerFactory.getLogger(Gridmix.class);
public EchoUserResolver() {
LOG.info(" Current user resolver is EchoUserResolver ");

View File

@ -20,8 +20,8 @@
import java.io.IOException;
import org.apache.commons.lang3.time.FastDateFormat;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -47,7 +47,7 @@
* </ul>
*/
class ExecutionSummarizer implements StatListener<JobStats> {
static final Log LOG = LogFactory.getLog(ExecutionSummarizer.class);
static final Logger LOG = LoggerFactory.getLogger(ExecutionSummarizer.class);
private static final FastDateFormat UTIL = FastDateFormat.getInstance();
private int numJobsInInputTrace;

View File

@ -37,8 +37,8 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapred.gridmix.RandomAlgorithms.Selector;
/**
@ -47,7 +47,7 @@
*/
class FilePool {
public static final Log LOG = LogFactory.getLog(FilePool.class);
public static final Logger LOG = LoggerFactory.getLogger(FilePool.class);
/**
* The minimum file size added to the pool. Default 128MiB.

View File

@ -49,8 +49,8 @@
import org.apache.hadoop.tools.rumen.JobStory;
import static org.apache.hadoop.tools.rumen.datatypes.util.MapReduceJobPropertiesParser.extractMaxHeapOpts;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Synthetic job generated from a trace description.
@ -59,7 +59,7 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
// Gridmix job name format is GRIDMIX<6 digit sequence number>
public static final String JOB_NAME_PREFIX = "GRIDMIX";
public static final Log LOG = LogFactory.getLog(GridmixJob.class);
public static final Logger LOG = LoggerFactory.getLogger(GridmixJob.class);
private static final ThreadLocal<Formatter> nameFormat =
new ThreadLocal<Formatter>() {

View File

@ -33,15 +33,15 @@
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Given a {@link #FilePool}, obtain a set of files capable of satisfying
* a full set of splits, then iterate over each source to fill the request.
*/
class InputStriper {
public static final Log LOG = LogFactory.getLog(InputStriper.class);
public static final Logger LOG = LoggerFactory.getLogger(InputStriper.class);
int idx;
long currentStart;
FileStatus current;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
@ -52,7 +52,7 @@
*/
abstract class JobFactory<T> implements Gridmix.Component<Void>,StatListener<T> {
public static final Log LOG = LogFactory.getLog(JobFactory.class);
public static final Logger LOG = LoggerFactory.getLogger(JobFactory.class);
protected final Path scratch;
protected final float rateFactor;

View File

@ -29,8 +29,8 @@
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
import org.apache.hadoop.mapreduce.Job;
@ -54,7 +54,7 @@
*/
class JobMonitor implements Gridmix.Component<JobStats> {
public static final Log LOG = LogFactory.getLog(JobMonitor.class);
public static final Logger LOG = LoggerFactory.getLogger(JobMonitor.class);
private final Queue<JobStats> mJobs;
private ExecutorService executor;

View File

@ -26,8 +26,8 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
/**
@ -39,7 +39,7 @@
*/
class JobSubmitter implements Gridmix.Component<GridmixJob> {
public static final Log LOG = LogFactory.getLog(JobSubmitter.class);
public static final Logger LOG = LoggerFactory.getLogger(JobSubmitter.class);
private final Semaphore sem;
private final Statistics statistics;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
@ -54,7 +54,7 @@
*/
class LoadJob extends GridmixJob {
public static final Log LOG = LogFactory.getLog(LoadJob.class);
public static final Logger LOG = LoggerFactory.getLogger(LoadJob.class);
public LoadJob(final Configuration conf, long submissionMillis,
final JobStory jobdesc, Path outRoot, UserGroupInformation ugi,

View File

@ -22,15 +22,15 @@
import java.util.Random;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
/**
* A random text generator. The words are simply sequences of alphabets.
*/
class RandomTextDataGenerator {
static final Log LOG = LogFactory.getLog(RandomTextDataGenerator.class);
static final Logger LOG = LoggerFactory.getLogger(RandomTextDataGenerator.class);
/**
* Configuration key for random text data generator's list size.

View File

@ -23,15 +23,15 @@
import org.apache.hadoop.tools.rumen.JobStory;
import org.apache.hadoop.tools.rumen.JobStoryProducer;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
class ReplayJobFactory extends JobFactory<Statistics.ClusterStats> {
public static final Log LOG = LogFactory.getLog(ReplayJobFactory.class);
public static final Logger LOG = LoggerFactory.getLogger(ReplayJobFactory.class);
/**
* Creating a new instance does not start the thread.

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -34,7 +34,7 @@
import java.util.List;
public class RoundRobinUserResolver implements UserResolver {
public static final Log LOG = LogFactory.getLog(RoundRobinUserResolver.class);
public static final Logger LOG = LoggerFactory.getLogger(RoundRobinUserResolver.class);
private int uidx = 0;
private List<UserGroupInformation> users = Collections.emptyList();

View File

@ -24,8 +24,8 @@
import org.apache.hadoop.tools.rumen.JobStoryProducer;
import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
@ -33,7 +33,7 @@
public class SerialJobFactory extends JobFactory<JobStats> {
public static final Log LOG = LogFactory.getLog(SerialJobFactory.class);
public static final Logger LOG = LoggerFactory.getLogger(SerialJobFactory.class);
private final Condition jobCompleted = lock.newCondition();
/**

View File

@ -50,12 +50,12 @@
import org.apache.hadoop.tools.rumen.ReduceTaskAttemptInfo;
import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.util.Time;
public class SleepJob extends GridmixJob {
public static final Log LOG = LogFactory.getLog(SleepJob.class);
public static final Logger LOG = LoggerFactory.getLogger(SleepJob.class);
private static final ThreadLocal <Random> rand =
new ThreadLocal <Random> () {
@Override protected Random initialValue() {

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.ClusterStatus;
import org.apache.hadoop.mapred.JobClient;
@ -50,7 +50,7 @@
* notified either on every job completion event or some fixed time interval.
*/
public class Statistics implements Component<Statistics.JobStats> {
public static final Log LOG = LogFactory.getLog(Statistics.class);
public static final Logger LOG = LoggerFactory.getLogger(Statistics.class);
private final StatCollector statistics = new StatCollector();
private JobClient cluster;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
@ -38,7 +38,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
public static final Log LOG = LogFactory.getLog(StressJobFactory.class);
public static final Logger LOG = LoggerFactory.getLogger(StressJobFactory.class);
private final LoadStatus loadStatus = new LoadStatus();
/**

View File

@ -21,14 +21,14 @@
import java.net.URI;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Resolves all UGIs to the submitting user.
*/
public class SubmitterUserResolver implements UserResolver {
public static final Log LOG = LogFactory.getLog(SubmitterUserResolver.class);
public static final Logger LOG = LoggerFactory.getLogger(SubmitterUserResolver.class);
private UserGroupInformation ugi = null;

View File

@ -33,8 +33,8 @@
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileStatus;
@ -57,7 +57,7 @@
import org.apache.hadoop.util.ToolRunner;
public class CommonJobTest {
public static final Log LOG = LogFactory.getLog(Gridmix.class);
public static final Logger LOG = LoggerFactory.getLogger(Gridmix.class);
protected static int NJOBS = 2;
protected static final long GENDATA = 1; // in megabytes

View File

@ -40,12 +40,12 @@
import java.util.concurrent.TimeUnit;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DebugJobProducer implements JobStoryProducer {
public static final Log LOG = LogFactory.getLog(DebugJobProducer.class);
public static final Logger LOG = LoggerFactory.getLogger(DebugJobProducer.class);
final ArrayList<JobStory> submitted;
private final Configuration conf;
private final AtomicInteger numJobs;

View File

@ -16,8 +16,8 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.permission.FsPermission;
@ -35,7 +35,7 @@
* This is a test class.
*/
public class GridmixTestUtils {
private static final Log LOG = LogFactory.getLog(GridmixTestUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(GridmixTestUtils.class);
static final Path DEST = new Path("/gridmix");
static FileSystem dfs = null;
static MiniDFSCluster dfsCluster = null;

View File

@ -28,8 +28,8 @@
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
@ -40,7 +40,7 @@
public class TestFilePool {
static final Log LOG = LogFactory.getLog(TestFileQueue.class);
static final Logger LOG = LoggerFactory.getLogger(TestFileQueue.class);
static final int NFILES = 26;
static final Path base = getBaseDir();

View File

@ -26,8 +26,8 @@
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -36,7 +36,7 @@
public class TestFileQueue {
static final Log LOG = LogFactory.getLog(TestFileQueue.class);
static final Logger LOG = LoggerFactory.getLogger(TestFileQueue.class);
static final int NFILES = 4;
static final int BLOCK = 256;
static final Path[] paths = new Path[NFILES];

View File

@ -30,8 +30,8 @@
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.CustomOutputCommitter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
@ -85,7 +85,7 @@
import static org.junit.Assert.*;
public class TestGridMixClasses {
private static final Log LOG = LogFactory.getLog(TestGridMixClasses.class);
private static final Logger LOG = LoggerFactory.getLogger(TestGridMixClasses.class);
/*
* simple test LoadSplit (getters,copy, write, read...)

View File

@ -23,8 +23,8 @@
import org.junit.Test;
import static org.junit.Assert.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
@ -32,7 +32,7 @@
import org.apache.hadoop.io.WritableUtils;
public class TestGridmixRecord {
private static final Log LOG = LogFactory.getLog(TestGridmixRecord.class);
private static final Logger LOG = LoggerFactory.getLogger(TestGridmixRecord.class);
static void lengthTest(GridmixRecord x, GridmixRecord y, int min,
int max) throws Exception {

View File

@ -22,14 +22,14 @@
import org.junit.Test;
import static org.junit.Assert.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataOutputBuffer;
public class TestRecordFactory {
private static final Log LOG = LogFactory.getLog(TestRecordFactory.class);
private static final Logger LOG = LoggerFactory.getLogger(TestRecordFactory.class);
public static void testFactory(long targetBytes, long targetRecs)
throws Exception {

View File

@ -27,8 +27,8 @@
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DeskewedJobTraceReader implements Closeable {
// underlying engine
@ -57,8 +57,8 @@ public class DeskewedJobTraceReader implements Closeable {
private final PriorityQueue<LoggedJob> skewBuffer;
static final private Log LOG =
LogFactory.getLog(DeskewedJobTraceReader.class);
static final private Logger LOG =
LoggerFactory.getLogger(DeskewedJobTraceReader.class);
static private class JobComparator implements Comparator<LoggedJob>,
Serializable {

View File

@ -30,8 +30,8 @@
import java.util.Random;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@ -52,7 +52,7 @@ public class Folder extends Configured implements Tool {
private int skewBufferLength = 0;
private long startsAfter = -1;
static final private Log LOG = LogFactory.getLog(Folder.class);
static final private Logger LOG = LoggerFactory.getLogger(Folder.class);
private DeskewedJobTraceReader reader = null;
private Outputter<LoggedJob> outGen = null;

View File

@ -36,8 +36,8 @@
import java.util.regex.Pattern;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
@ -139,7 +139,7 @@ public class HadoopLogsAnalyzer extends Configured implements Tool {
private Histogram successfulNthReducerAttempts;
private Histogram mapperLocality;
static final private Log LOG = LogFactory.getLog(HadoopLogsAnalyzer.class);
static final private Logger LOG = LoggerFactory.getLogger(HadoopLogsAnalyzer.class);
private int[] attemptTimesPercentiles;

View File

@ -22,13 +22,13 @@
import java.util.List;
import java.util.Queue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
abstract class HistoryEventEmitter {
static final private Log LOG = LogFactory.getLog(HistoryEventEmitter.class);
static final private Logger LOG = LoggerFactory.getLogger(HistoryEventEmitter.class);
abstract List<SingleEventEmitter> nonFinalSEEs();

View File

@ -26,8 +26,8 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapred.TaskStatus;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskType;
@ -69,7 +69,7 @@ public class JobBuilder {
private static final long BYTES_IN_MEG =
StringUtils.TraditionalBinaryPrefix.string2long("1m");
static final private Log LOG = LogFactory.getLog(JobBuilder.class);
static final private Logger LOG = LoggerFactory.getLogger(JobBuilder.class);
private String jobID;

View File

@ -25,8 +25,8 @@
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.security.authorize.AccessControlList;
@ -37,7 +37,7 @@
*/
public class ParsedJob extends LoggedJob {
private static final Log LOG = LogFactory.getLog(ParsedJob.class);
private static final Logger LOG = LoggerFactory.getLogger(ParsedJob.class);
private Map<String, Long> totalCountersMap = new HashMap<String, Long>();
private Map<String, Long> mapCountersMap = new HashMap<String, Long>();

View File

@ -22,8 +22,8 @@
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapreduce.jobhistory.JhCounters;
/**
@ -33,7 +33,7 @@
*/
public class ParsedTask extends LoggedTask {
private static final Log LOG = LogFactory.getLog(ParsedTask.class);
private static final Logger LOG = LoggerFactory.getLogger(ParsedTask.class);
private String diagnosticInfo;
private String failedDueToAttempt;

View File

@ -21,8 +21,8 @@
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.mapreduce.jobhistory.JhCounters;
/**
@ -32,7 +32,7 @@
*/
public class ParsedTaskAttempt extends LoggedTaskAttempt {
private static final Log LOG = LogFactory.getLog(ParsedTaskAttempt.class);
private static final Logger LOG = LoggerFactory.getLogger(ParsedTaskAttempt.class);
private String diagnosticInfo;
private String trackerName;

View File

@ -21,8 +21,8 @@
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The purpose of this class is to generate new random seeds from a master
@ -42,7 +42,7 @@
* http://www.iro.umontreal.ca/~lecuyer/myftp/streams00/
*/
public class RandomSeedGenerator {
private static Log LOG = LogFactory.getLog(RandomSeedGenerator.class);
private static Logger LOG = LoggerFactory.getLogger(RandomSeedGenerator.class);
private static final Charset UTF_8 = Charset.forName("UTF-8");
/** MD5 algorithm instance, one for each thread. */

View File

@ -30,8 +30,8 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
@ -49,7 +49,7 @@
* The main driver of the Rumen Parser.
*/
public class TraceBuilder extends Configured implements Tool {
static final private Log LOG = LogFactory.getLog(TraceBuilder.class);
static final private Logger LOG = LoggerFactory.getLogger(TraceBuilder.class);
static final int RUN_METHOD_FAILED_EXIT_CODE = 3;
@ -310,6 +310,6 @@ void processJobHistory(JobHistoryParser parser, JobBuilder jobBuilder)
}
void finish() {
IOUtils.cleanup(LOG, traceWriter, topologyWriter);
IOUtils.cleanupWithLogger(LOG, traceWriter, topologyWriter);
}
}

View File

@ -23,8 +23,8 @@
import java.util.Random;
import java.util.HashMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TaskStatus.State;
@ -49,7 +49,7 @@
*/
@SuppressWarnings("deprecation")
public class ZombieJob implements JobStory {
static final Log LOG = LogFactory.getLog(ZombieJob.class);
static final Logger LOG = LoggerFactory.getLogger(ZombieJob.class);
private final LoggedJob job;
private Map<TaskID, LoggedTask> loggedTaskMap;
private Map<TaskAttemptID, LoggedTaskAttempt> loggedTaskAttemptMap;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.yarn.sls.resourcemanager;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -42,7 +42,7 @@
public class MockAMLauncher extends ApplicationMasterLauncher
implements EventHandler<AMLauncherEvent> {
private static final Log LOG = LogFactory.getLog(
private static final Logger LOG = LoggerFactory.getLogger(
MockAMLauncher.class);
Map<String, AMSimulator> amMap;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.yarn.sls.synthetic;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.math3.random.JDKRandomGenerator;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
@ -51,7 +51,7 @@
public class SynthJob implements JobStory {
@SuppressWarnings("StaticVariableName")
private static Log LOG = LogFactory.getLog(SynthJob.class);
private static Logger LOG = LoggerFactory.getLogger(SynthJob.class);
private static final long MIN_MEMORY = 1024;
private static final long MIN_VCORES = 1;

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.yarn.sls.synthetic;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.math3.distribution.AbstractRealDistribution;
import org.apache.commons.math3.random.JDKRandomGenerator;
import org.apache.hadoop.conf.Configuration;
@ -53,7 +53,7 @@
public class SynthTraceJobProducer implements JobStoryProducer {
@SuppressWarnings("StaticVariableName")
private static final Log LOG = LogFactory.getLog(SynthTraceJobProducer.class);
private static final Logger LOG = LoggerFactory.getLogger(SynthTraceJobProducer.class);
private final Configuration conf;
private final AtomicInteger numJobs;

View File

@ -24,7 +24,8 @@
import java.util.ArrayList;
import java.util.Properties;
import org.apache.commons.logging.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
@ -44,7 +45,7 @@
*/
public abstract class PipeMapRed {
protected static final Log LOG = LogFactory.getLog(PipeMapRed.class.getName());
protected static final Logger LOG = LoggerFactory.getLogger(PipeMapRed.class.getName());
/**
* Returns the Configuration.
@ -397,7 +398,7 @@ public void run() {
}
} catch (Throwable th) {
outerrThreadsThrowable = th;
LOG.warn(th);
LOG.warn("{}", th);
} finally {
try {
if (clientIn_ != null) {
@ -405,7 +406,7 @@ public void run() {
clientIn_ = null;
}
} catch (IOException io) {
LOG.info(io);
LOG.info("{}", io);
}
}
}
@ -466,7 +467,7 @@ public void run() {
}
} catch (Throwable th) {
outerrThreadsThrowable = th;
LOG.warn(th);
LOG.warn("{}", th);
try {
if (lineReader != null) {
lineReader.close();
@ -476,7 +477,7 @@ public void run() {
clientErr_ = null;
}
} catch (IOException io) {
LOG.info(io);
LOG.info("{}", io);
}
}
}
@ -531,13 +532,13 @@ public void mapRedFinished() {
clientOut_.flush();
clientOut_.close();
} catch (IOException io) {
LOG.warn(io);
LOG.warn("{}", io);
}
}
try {
waitOutputThreads();
} catch (IOException io) {
LOG.warn(io);
LOG.warn("{}", io);
}
if (sim != null) sim.destroy();
LOG.info("mapRedFinished");

View File

@ -30,7 +30,8 @@
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.commons.logging.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Shared functionality for hadoopStreaming formats.
@ -40,7 +41,7 @@
*/
public abstract class StreamBaseRecordReader implements RecordReader<Text, Text> {
protected static final Log LOG = LogFactory.getLog(StreamBaseRecordReader.class.getName());
protected static final Logger LOG = LoggerFactory.getLogger(StreamBaseRecordReader.class.getName());
// custom JobConf properties for this class are prefixed with this namespace
final static String CONF_NS = "stream.recordreader.";

View File

@ -36,8 +36,8 @@
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.mapreduce.MRConfig;
@ -79,7 +79,7 @@
*/
public class StreamJob implements Tool {
protected static final Log LOG = LogFactory.getLog(StreamJob.class.getName());
protected static final Logger LOG = LoggerFactory.getLogger(StreamJob.class.getName());
final static String REDUCE_NONE = "NONE";
/** -----------Streaming CLI Implementation **/

View File

@ -20,8 +20,8 @@
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -40,8 +40,8 @@
*/
public abstract class StreamBaseRecordReader extends RecordReader<Text, Text> {
protected static final Log LOG = LogFactory
.getLog(StreamBaseRecordReader.class.getName());
protected static final Logger LOG = LoggerFactory
.getLogger(StreamBaseRecordReader.class.getName());
// custom JobConf properties for this class are prefixed with this namespace
final static String CONF_NS = "stream.recordreader.";

View File

@ -25,8 +25,8 @@
import java.util.zip.ZipEntry;
import java.util.zip.ZipOutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -42,7 +42,7 @@
*/
public class TestMultipleArchiveFiles extends TestStreaming
{
private static final Log LOG = LogFactory.getLog(TestMultipleArchiveFiles.class);
private static final Logger LOG = LoggerFactory.getLogger(TestMultipleArchiveFiles.class);
private StreamJob job;
private String INPUT_DIR = "multiple-archive-files/";

View File

@ -23,8 +23,8 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.junit.Before;
import org.junit.Test;
@ -38,7 +38,7 @@
*/
public class TestStreamXmlMultipleRecords extends TestStreaming
{
private static final Log LOG = LogFactory.getLog(
private static final Logger LOG = LoggerFactory.getLogger(
TestStreamXmlMultipleRecords.class);
private boolean hasPerl = false;

View File

@ -31,8 +31,8 @@
import java.util.Properties;
import java.util.StringTokenizer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.ClusterMapReduceTestCase;
@ -51,8 +51,8 @@
public class TestStreamingBadRecords extends ClusterMapReduceTestCase
{
private static final Log LOG =
LogFactory.getLog(TestStreamingBadRecords.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestStreamingBadRecords.class);
private static final List<String> MAPPER_BAD_RECORDS =
Arrays.asList("hey022","hey023","hey099");

View File

@ -26,13 +26,13 @@
import java.util.Iterator;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
class UtilTest {
private static final Log LOG = LogFactory.getLog(UtilTest.class);
private static final Logger LOG = LoggerFactory.getLogger(UtilTest.class);
/**
* Utility routine to recurisvely delete a directory.