HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch. (ozawa)

(cherry picked from commit a6ed4894b5)

Conflicts:
	hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java
This commit is contained in:
Tsuyoshi Ozawa 2015-01-08 14:51:57 +09:00
parent 9eb7611e81
commit 132fd6ba58
14 changed files with 229 additions and 46 deletions

View File

@ -103,6 +103,9 @@ Release 2.7.0 - UNRELEASED
HADOOP-11390 Metrics 2 ganglia provider to include hostname in HADOOP-11390 Metrics 2 ganglia provider to include hostname in
unresolved address problems. (Varun Saxena via stevel) unresolved address problems. (Varun Saxena via stevel)
HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
(ozawa)
OPTIMIZATIONS OPTIMIZATIONS
HADOOP-11323. WritableComparator#compare keeps reference to byte array. HADOOP-11323. WritableComparator#compare keeps reference to byte array.

View File

@ -22,6 +22,7 @@ import java.lang.management.ManagementFactory;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -30,7 +31,6 @@ import org.apache.hadoop.conf.Configuration;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
@ -172,7 +172,7 @@ public class JvmPauseMonitor {
private class Monitor implements Runnable { private class Monitor implements Runnable {
@Override @Override
public void run() { public void run() {
Stopwatch sw = new Stopwatch(); StopWatch sw = new StopWatch();
Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes(); Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
while (shouldRun) { while (shouldRun) {
sw.reset().start(); sw.reset().start();
@ -181,7 +181,7 @@ public class JvmPauseMonitor {
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
return; return;
} }
long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS; long extraSleepTime = sw.now(TimeUnit.MILLISECONDS) - SLEEP_INTERVAL_MS;
Map<String, GcTimes> gcTimesAfterSleep = getGcTimes(); Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
if (extraSleepTime > warnThresholdMs) { if (extraSleepTime > warnThresholdMs) {

View File

@ -0,0 +1,108 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.util;
import java.io.Closeable;
import java.util.concurrent.TimeUnit;
/**
* A simplified StopWatch implementation which can measure times in nanoseconds.
*/
public class StopWatch implements Closeable {
private boolean isStarted;
private long startNanos;
private long currentElapsedNanos;
public StopWatch() {
}
/**
* The method is used to find out if the StopWatch is started.
* @return boolean If the StopWatch is started.
*/
public boolean isRunning() {
return isStarted;
}
/**
* Start to measure times and make the state of stopwatch running.
* @return this instance of StopWatch.
*/
public StopWatch start() {
if (isStarted) {
throw new IllegalStateException("StopWatch is already running");
}
isStarted = true;
startNanos = System.nanoTime();
return this;
}
/**
* Stop elapsed time and make the state of stopwatch stop.
* @return this instance of StopWatch.
*/
public StopWatch stop() {
if (!isStarted) {
throw new IllegalStateException("StopWatch is already stopped");
}
long now = System.nanoTime();
isStarted = false;
currentElapsedNanos += now - startNanos;
return this;
}
/**
* Reset elapsed time to zero and make the state of stopwatch stop.
* @return this instance of StopWatch.
*/
public StopWatch reset() {
currentElapsedNanos = 0;
isStarted = false;
return this;
}
/**
* @return current elapsed time in specified timeunit.
*/
public long now(TimeUnit timeUnit) {
return timeUnit.convert(now(), TimeUnit.NANOSECONDS);
}
/**
* @return current elapsed time in nanosecond.
*/
public long now() {
return isStarted ?
System.nanoTime() - startNanos + currentElapsedNanos :
currentElapsedNanos;
}
@Override
public String toString() {
return String.valueOf(now());
}
@Override
public void close() {
if (isStarted) {
stop();
}
}
}

View File

@ -21,12 +21,11 @@ import static org.junit.Assert.*;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.concurrent.TimeUnit;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import com.google.common.base.Stopwatch;
public class TestChunkedArrayList { public class TestChunkedArrayList {
@Test @Test
@ -71,24 +70,24 @@ public class TestChunkedArrayList {
System.gc(); System.gc();
{ {
ArrayList<String> arrayList = new ArrayList<String>(); ArrayList<String> arrayList = new ArrayList<String>();
Stopwatch sw = new Stopwatch(); StopWatch sw = new StopWatch();
sw.start(); sw.start();
for (int i = 0; i < numElems; i++) { for (int i = 0; i < numElems; i++) {
arrayList.add(obj); arrayList.add(obj);
} }
System.out.println(" ArrayList " + sw.elapsedMillis()); System.out.println(" ArrayList " + sw.now(TimeUnit.MILLISECONDS));
} }
// test ChunkedArrayList // test ChunkedArrayList
System.gc(); System.gc();
{ {
ChunkedArrayList<String> chunkedList = new ChunkedArrayList<String>(); ChunkedArrayList<String> chunkedList = new ChunkedArrayList<String>();
Stopwatch sw = new Stopwatch(); StopWatch sw = new StopWatch();
sw.start(); sw.start();
for (int i = 0; i < numElems; i++) { for (int i = 0; i < numElems; i++) {
chunkedList.add(obj); chunkedList.add(obj);
} }
System.out.println("ChunkedArrayList " + sw.elapsedMillis()); System.out.println("ChunkedArrayList " + sw.now(TimeUnit.MILLISECONDS));
} }
} }
} }

View File

@ -21,8 +21,6 @@ import java.nio.ByteBuffer;
import java.util.Random; import java.util.Random;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import com.google.common.base.Stopwatch;
import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.ChecksumException;
import org.junit.Test; import org.junit.Test;
@ -147,19 +145,19 @@ public class TestDataChecksum {
Harness h = new Harness(checksum, dataLength, true); Harness h = new Harness(checksum, dataLength, true);
for (int i = 0; i < NUM_RUNS; i++) { for (int i = 0; i < NUM_RUNS; i++) {
Stopwatch s = new Stopwatch().start(); StopWatch s = new StopWatch().start();
// calculate real checksum, make sure it passes // calculate real checksum, make sure it passes
checksum.calculateChunkedSums(h.dataBuf, h.checksumBuf); checksum.calculateChunkedSums(h.dataBuf, h.checksumBuf);
s.stop(); s.stop();
System.err.println("Calculate run #" + i + ": " + System.err.println("Calculate run #" + i + ": " +
s.elapsedTime(TimeUnit.MICROSECONDS) + "us"); s.now(TimeUnit.MICROSECONDS) + "us");
s = new Stopwatch().start(); s = new StopWatch().start();
// calculate real checksum, make sure it passes // calculate real checksum, make sure it passes
checksum.verifyChunkedSums(h.dataBuf, h.checksumBuf, "fake file", 0); checksum.verifyChunkedSums(h.dataBuf, h.checksumBuf, "fake file", 0);
s.stop(); s.stop();
System.err.println("Verify run #" + i + ": " + System.err.println("Verify run #" + i + ": " +
s.elapsedTime(TimeUnit.MICROSECONDS) + "us"); s.now(TimeUnit.MICROSECONDS) + "us");
} }
} }

View File

@ -0,0 +1,62 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.util;
import org.junit.Assert;
import org.junit.Test;
public class TestStopWatch {
@Test
public void testStartAndStop() throws Exception {
try (StopWatch sw = new StopWatch()) {
Assert.assertFalse(sw.isRunning());
sw.start();
Assert.assertTrue(sw.isRunning());
sw.stop();
Assert.assertFalse(sw.isRunning());
}
}
@Test
public void testStopInTryWithResource() throws Exception {
try (StopWatch sw = new StopWatch()) {
// make sure that no exception is thrown.
}
}
@Test
public void testExceptions() throws Exception {
StopWatch sw = new StopWatch();
try {
sw.stop();
} catch (Exception e) {
Assert.assertTrue("IllegalStateException is expected",
e instanceof IllegalStateException);
}
sw.reset();
sw.start();
try {
sw.start();
} catch (Exception e) {
Assert.assertTrue("IllegalStateException is expected",
e instanceof IllegalStateException);
}
}
}

View File

@ -222,6 +222,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7484. Make FSDirectory#addINode take existing INodes as its parameter. HDFS-7484. Make FSDirectory#addINode take existing INodes as its parameter.
(jing9) (jing9)
HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
(ozawa)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-7454. Reduce memory footprint for AclEntries in NameNode. HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

View File

@ -52,10 +52,10 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.StopWatch;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import com.google.common.net.InetAddresses; import com.google.common.net.InetAddresses;
import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.Futures;
@ -143,7 +143,7 @@ public class IPCLoggerChannel implements AsyncLogger {
/** /**
* Stopwatch which starts counting on each heartbeat that is sent * Stopwatch which starts counting on each heartbeat that is sent
*/ */
private final Stopwatch lastHeartbeatStopwatch = new Stopwatch(); private final StopWatch lastHeartbeatStopwatch = new StopWatch();
private static final long HEARTBEAT_INTERVAL_MILLIS = 1000; private static final long HEARTBEAT_INTERVAL_MILLIS = 1000;
@ -463,8 +463,8 @@ public class IPCLoggerChannel implements AsyncLogger {
* written. * written.
*/ */
private void heartbeatIfNecessary() throws IOException { private void heartbeatIfNecessary() throws IOException {
if (lastHeartbeatStopwatch.elapsedMillis() > HEARTBEAT_INTERVAL_MILLIS || if (lastHeartbeatStopwatch.now(TimeUnit.MILLISECONDS)
!lastHeartbeatStopwatch.isRunning()) { > HEARTBEAT_INTERVAL_MILLIS || !lastHeartbeatStopwatch.isRunning()) {
try { try {
getProxy().heartbeat(createReqInfo()); getProxy().heartbeat(createReqInfo());
} finally { } finally {

View File

@ -65,11 +65,11 @@ import org.apache.hadoop.security.UserGroupInformation;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Range; import com.google.common.collect.Range;
import com.google.common.collect.Ranges; import com.google.common.collect.Ranges;
import com.google.protobuf.TextFormat; import com.google.protobuf.TextFormat;
import org.apache.hadoop.util.StopWatch;
/** /**
* A JournalNode can manage journals for several clusters at once. * A JournalNode can manage journals for several clusters at once.
@ -374,15 +374,20 @@ public class Journal implements Closeable {
curSegment.writeRaw(records, 0, records.length); curSegment.writeRaw(records, 0, records.length);
curSegment.setReadyToFlush(); curSegment.setReadyToFlush();
Stopwatch sw = new Stopwatch(); StopWatch sw = new StopWatch();
sw.start(); sw.start();
curSegment.flush(shouldFsync); curSegment.flush(shouldFsync);
sw.stop(); sw.stop();
metrics.addSync(sw.elapsedTime(TimeUnit.MICROSECONDS)); long nanoSeconds = sw.now();
if (sw.elapsedTime(TimeUnit.MILLISECONDS) > WARN_SYNC_MILLIS_THRESHOLD) { metrics.addSync(
TimeUnit.MICROSECONDS.convert(nanoSeconds, TimeUnit.NANOSECONDS));
long milliSeconds = TimeUnit.MILLISECONDS.convert(
nanoSeconds, TimeUnit.NANOSECONDS);
if (milliSeconds > WARN_SYNC_MILLIS_THRESHOLD) {
LOG.warn("Sync of transaction range " + firstTxnId + "-" + lastTxnId + LOG.warn("Sync of transaction range " + firstTxnId + "-" + lastTxnId +
" took " + sw.elapsedTime(TimeUnit.MILLISECONDS) + "ms"); " took " + milliSeconds + "ms");
} }
if (isLagging) { if (isLagging) {

View File

@ -32,12 +32,11 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.metrics2.util.Quantile; import org.apache.hadoop.metrics2.util.Quantile;
import org.apache.hadoop.metrics2.util.SampleQuantiles; import org.apache.hadoop.metrics2.util.SampleQuantiles;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.junit.Test; import org.junit.Test;
import com.google.common.base.Stopwatch;
/** /**
* This class tests hflushing concurrently from many threads. * This class tests hflushing concurrently from many threads.
*/ */
@ -100,10 +99,10 @@ public class TestMultiThreadedHflush {
} }
private void doAWrite() throws IOException { private void doAWrite() throws IOException {
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
stm.write(toWrite); stm.write(toWrite);
stm.hflush(); stm.hflush();
long micros = sw.elapsedTime(TimeUnit.MICROSECONDS); long micros = sw.now(TimeUnit.MICROSECONDS);
quantiles.insert(micros); quantiles.insert(micros);
} }
} }
@ -276,12 +275,12 @@ public class TestMultiThreadedHflush {
int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
DFSConfigKeys.DFS_REPLICATION_DEFAULT); DFSConfigKeys.DFS_REPLICATION_DEFAULT);
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
test.doMultithreadedWrites(conf, p, numThreads, writeSize, numWrites, test.doMultithreadedWrites(conf, p, numThreads, writeSize, numWrites,
replication); replication);
sw.stop(); sw.stop();
System.out.println("Finished in " + sw.elapsedMillis() + "ms"); System.out.println("Finished in " + sw.now(TimeUnit.MILLISECONDS) + "ms");
System.out.println("Latency quantiles (in microseconds):\n" + System.out.println("Latency quantiles (in microseconds):\n" +
test.quantiles); test.quantiles);
return 0; return 0;

View File

@ -27,6 +27,7 @@ import java.io.File;
import java.net.HttpURLConnection; import java.net.HttpURLConnection;
import java.net.URL; import java.net.URL;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
@ -38,8 +39,6 @@ import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel; import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto; import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.server.Journal;
import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion; import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@ -48,12 +47,12 @@ import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.MetricsAsserts; import org.apache.hadoop.test.MetricsAsserts;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StopWatch;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Stopwatch;
import com.google.common.primitives.Bytes; import com.google.common.primitives.Bytes;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
@ -325,11 +324,11 @@ public class TestJournalNode {
ch.setEpoch(1); ch.setEpoch(1);
ch.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION).get(); ch.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION).get();
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
for (int i = 1; i < numEdits; i++) { for (int i = 1; i < numEdits; i++) {
ch.sendEdits(1L, i, 1, data).get(); ch.sendEdits(1L, i, 1, data).get();
} }
long time = sw.elapsedMillis(); long time = sw.now(TimeUnit.MILLISECONDS);
System.err.println("Wrote " + numEdits + " batches of " + editsSize + System.err.println("Wrote " + numEdits + " batches of " + editsSize +
" bytes in " + time + "ms"); " bytes in " + time + "ms");

View File

@ -23,6 +23,9 @@ Release 2.7.0 - UNRELEASED
MAPREDUCE-6046. Change the class name for logs in RMCommunicator MAPREDUCE-6046. Change the class name for logs in RMCommunicator
(Sahil Takiar via devaraj) (Sahil Takiar via devaraj)
HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
(ozawa)
OPTIMIZATIONS OPTIMIZATIONS
MAPREDUCE-6169. MergeQueue should release reference to the current item MAPREDUCE-6169. MergeQueue should release reference to the current item

View File

@ -28,6 +28,7 @@ import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -45,9 +46,9 @@ import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node; import org.apache.hadoop.net.Node;
import org.apache.hadoop.net.NodeBase; import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Stopwatch;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
/** /**
@ -223,7 +224,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS, org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS,
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.DEFAULT_LIST_STATUS_NUM_THREADS); org.apache.hadoop.mapreduce.lib.input.FileInputFormat.DEFAULT_LIST_STATUS_NUM_THREADS);
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
if (numThreads == 1) { if (numThreads == 1) {
List<FileStatus> locatedFiles = singleThreadedListStatus(job, dirs, inputFilter, recursive); List<FileStatus> locatedFiles = singleThreadedListStatus(job, dirs, inputFilter, recursive);
result = locatedFiles.toArray(new FileStatus[locatedFiles.size()]); result = locatedFiles.toArray(new FileStatus[locatedFiles.size()]);
@ -242,7 +243,8 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
sw.stop(); sw.stop();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Time taken to get FileStatuses: " + sw.elapsedMillis()); LOG.debug("Time taken to get FileStatuses: "
+ sw.now(TimeUnit.MILLISECONDS));
} }
LOG.info("Total input paths to process : " + result.length); LOG.info("Total input paths to process : " + result.length);
return result; return result;
@ -309,7 +311,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
* they're too big.*/ * they're too big.*/
public InputSplit[] getSplits(JobConf job, int numSplits) public InputSplit[] getSplits(JobConf job, int numSplits)
throws IOException { throws IOException {
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
FileStatus[] files = listStatus(job); FileStatus[] files = listStatus(job);
// Save the number of input files for metrics/loadgen // Save the number of input files for metrics/loadgen
@ -371,7 +373,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
sw.stop(); sw.stop();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Total # of splits generated by getSplits: " + splits.size() LOG.debug("Total # of splits generated by getSplits: " + splits.size()
+ ", TimeTaken: " + sw.elapsedMillis()); + ", TimeTaken: " + sw.now(TimeUnit.MILLISECONDS));
} }
return splits.toArray(new FileSplit[splits.size()]); return splits.toArray(new FileSplit[splits.size()]);
} }

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -43,9 +44,9 @@ import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.security.TokenCache; import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Stopwatch;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
/** /**
@ -259,7 +260,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
int numThreads = job.getConfiguration().getInt(LIST_STATUS_NUM_THREADS, int numThreads = job.getConfiguration().getInt(LIST_STATUS_NUM_THREADS,
DEFAULT_LIST_STATUS_NUM_THREADS); DEFAULT_LIST_STATUS_NUM_THREADS);
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
if (numThreads == 1) { if (numThreads == 1) {
result = singleThreadedListStatus(job, dirs, inputFilter, recursive); result = singleThreadedListStatus(job, dirs, inputFilter, recursive);
} else { } else {
@ -276,7 +277,8 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
sw.stop(); sw.stop();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Time taken to get FileStatuses: " + sw.elapsedMillis()); LOG.debug("Time taken to get FileStatuses: "
+ sw.now(TimeUnit.MILLISECONDS));
} }
LOG.info("Total input paths to process : " + result.size()); LOG.info("Total input paths to process : " + result.size());
return result; return result;
@ -376,7 +378,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
* @throws IOException * @throws IOException
*/ */
public List<InputSplit> getSplits(JobContext job) throws IOException { public List<InputSplit> getSplits(JobContext job) throws IOException {
Stopwatch sw = new Stopwatch().start(); StopWatch sw = new StopWatch().start();
long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job)); long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
long maxSize = getMaxSplitSize(job); long maxSize = getMaxSplitSize(job);
@ -427,7 +429,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
sw.stop(); sw.stop();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Total # of splits generated by getSplits: " + splits.size() LOG.debug("Total # of splits generated by getSplits: " + splits.size()
+ ", TimeTaken: " + sw.elapsedMillis()); + ", TimeTaken: " + sw.now(TimeUnit.MILLISECONDS));
} }
return splits; return splits;
} }