diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 65ec560950..f829dd5cdc 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -74,6 +74,7 @@ import java.util.StringTokenizer; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; import java.util.jar.Attributes; import java.util.jar.Manifest; @@ -2294,7 +2295,7 @@ public void addLocalDriverClazz(String driverClazz) { } Driver[] scanDrivers(boolean knownOnly) throws IOException { - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); ServiceLoader sqlDrivers = ServiceLoader.load(Driver.class); @@ -2303,8 +2304,11 @@ public void addLocalDriverClazz(String driverClazz) { for (Driver driver : sqlDrivers) { driverClasses.add(driver); } + + final long estimatedTime = System.nanoTime() - startTime; info("scan complete in " - + (System.currentTimeMillis() - start) + "ms"); + + TimeUnit.NANOSECONDS.toMillis(estimatedTime) + "ms"); + return driverClasses.toArray(new Driver[0]); } diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java index 8f47323700..73215213ab 100644 --- a/beeline/src/java/org/apache/hive/beeline/Commands.java +++ b/beeline/src/java/org/apache/hive/beeline/Commands.java @@ -52,6 +52,7 @@ import java.util.Properties; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.hive.common.cli.ShellCmdExecutor; import org.apache.hadoop.hive.conf.HiveConf; @@ -448,12 +449,12 @@ public boolean commit(String line) throws SQLException { return false; } try { - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); beeLine.getDatabaseConnection().getConnection().commit(); - long end = System.currentTimeMillis(); + final long estimatedTime = System.nanoTime() - startTime; beeLine.showWarnings(); beeLine.info(beeLine.loc("commit-complete") - + " " + beeLine.locElapsedTime(end - start)); + + " " + beeLine.locElapsedTime(TimeUnit.NANOSECONDS.toMillis(estimatedTime))); return true; } catch (Exception e) { return beeLine.error(e); @@ -469,12 +470,12 @@ public boolean rollback(String line) throws SQLException { return false; } try { - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); beeLine.getDatabaseConnection().getConnection().rollback(); - long end = System.currentTimeMillis(); + final long estimatedTime = System.nanoTime() - startTime; beeLine.showWarnings(); beeLine.info(beeLine.loc("rollback-complete") - + " " + beeLine.locElapsedTime(end - start)); + + " " + beeLine.locElapsedTime(TimeUnit.NANOSECONDS.toMillis(estimatedTime))); return true; } catch (Exception e) { return beeLine.error(e); @@ -978,7 +979,7 @@ private boolean executeInternal(String sql, boolean call) { Thread logThread = null; try { - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); if (call) { stmnt = beeLine.getDatabaseConnection().getConnection().prepareCall(sql); @@ -1024,10 +1025,12 @@ private boolean executeInternal(String sql, boolean call) { ResultSet rs = stmnt.getResultSet(); try { int count = beeLine.print(rs); - long end = System.currentTimeMillis(); + final long estimatedTime = System.nanoTime() - startTime; if (showReport()) { - beeLine.output(beeLine.loc("rows-selected", count) + " " + beeLine.locElapsedTime(end - start), + beeLine.output( + beeLine.loc("rows-selected", count) + " " + + beeLine.locElapsedTime(TimeUnit.NANOSECONDS.toMillis(estimatedTime)), true, beeLine.getErrorStream()); } } finally { @@ -1044,10 +1047,12 @@ private boolean executeInternal(String sql, boolean call) { } } else { int count = stmnt.getUpdateCount(); - long end = System.currentTimeMillis(); + final long estimatedTime = System.nanoTime() - startTime; if (showReport()) { - beeLine.output(beeLine.loc("rows-affected", count) + " " + beeLine.locElapsedTime(end - start), + beeLine.output( + beeLine.loc("rows-affected", count) + " " + + beeLine.locElapsedTime(TimeUnit.NANOSECONDS.toMillis(estimatedTime)), true, beeLine.getErrorStream()); } } diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java index c6f8946247..7020173a1c 100644 --- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java +++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java @@ -36,6 +36,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -238,7 +239,7 @@ CommandProcessorResponse processLocalCmd(String cmd, CommandProcessor proc, CliS if (proc instanceof IDriver) { IDriver qp = (IDriver) proc; PrintStream out = ss.out; - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); if (ss.getIsVerbose()) { out.println(cmd); } @@ -254,8 +255,7 @@ CommandProcessorResponse processLocalCmd(String cmd, CommandProcessor proc, CliS } // query has run capture the time - long end = System.currentTimeMillis(); - double timeTaken = (end - start) / 1000.0; + final long estimatedTime = System.nanoTime() - startTime; ArrayList res = new ArrayList(); @@ -293,7 +293,8 @@ CommandProcessorResponse processLocalCmd(String cmd, CommandProcessor proc, CliS } console.printInfo( - "Time taken: " + timeTaken + " seconds" + (counter == 0 ? "" : ", Fetched: " + counter + " row(s)")); + "Time taken: " + TimeUnit.NANOSECONDS.toSeconds(estimatedTime) + " seconds" + + (counter == 0 ? "" : ", Fetched: " + counter + " row(s)")); } } else { String firstToken = tokenizeCmd(cmd.trim())[0]; diff --git a/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java b/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java index f266778a7d..0c268c556e 100644 --- a/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java +++ b/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java @@ -28,6 +28,7 @@ import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; import java.nio.charset.CodingErrorAction; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -75,7 +76,7 @@ public int run(String[] args) throws Exception { long start = 0l; long length = -1l; int recordCount = 0; - long startT = System.currentTimeMillis(); + final long startTime = System.nanoTime(); boolean verbose = false; boolean columnSizes = false; boolean pretty = false; @@ -189,12 +190,12 @@ public int run(String[] args) throws Exception { printRecord(value, buf); recordCount++; if (verbose && (recordCount % RECORD_PRINT_INTERVAL) == 0) { - long now = System.currentTimeMillis(); + final long estimatedTime = System.nanoTime() - startTime; System.err.println("Read " + recordCount/1024 + "k records"); System.err.println("Read " + ((recordReader.getPos() / (1024L*1024L))) + "MB"); - System.err.printf("Input scan rate %.2f MB/s\n", - (recordReader.getPos() * 1.0 / (now - startT)) / 1024.0); + System.err.printf("Input scan rate %.2f MB/s\n", (((double) recordReader.getPos() / (1024.0 * 1024.0)) + / (double) TimeUnit.NANOSECONDS.toSeconds(estimatedTime))); } if (buf.length() > STRING_BUFFER_FLUSH_SIZE) { System.out.print(buf.toString()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index 9ad4e71482..53d3a96742 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -31,6 +31,7 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import com.google.common.collect.Lists; @@ -438,7 +439,7 @@ public int createDynamicBucket(int bucketNum) { protected transient boolean bDynParts; private transient SubStructObjectInspector subSetOI; private transient int timeOut; // JT timeout in msec. - private transient long lastProgressReport = System.currentTimeMillis(); + private transient long lastProgressReport = System.nanoTime(); protected transient boolean autoDelete = false; protected transient JobConf jc; @@ -875,9 +876,9 @@ private void createDpDir(final Path dpPath) throws IOException { */ protected boolean updateProgress() { if (reporter != null && - (System.currentTimeMillis() - lastProgressReport) > timeOut) { + (System.nanoTime() - lastProgressReport) > TimeUnit.MILLISECONDS.toNanos(timeOut)) { reporter.progress(); - lastProgressReport = System.currentTimeMillis(); + lastProgressReport = System.nanoTime(); return true; } else { return false; @@ -1270,7 +1271,7 @@ public void closeOp(boolean abort) throws HiveException { } } - lastProgressReport = System.currentTimeMillis(); + lastProgressReport = System.nanoTime(); if (!abort) { // If serializer is ThriftJDBCBinarySerDe, then it buffers rows to a certain limit (hive.server2.thrift.resultset.max.fetch.size) // and serializes the whole batch when the buffer is full. The serialize returns null if the buffer is not full diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index 695d08bbe2..7566e58173 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -76,6 +76,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; /** * MoveTask implementation. @@ -549,7 +550,7 @@ private DataContainer handleDynParts(Hive db, Table table, LoadTableDesc tbd, List> dps = Utilities.getFullDPSpecs(conf, dpCtx); console.printInfo(System.getProperty("line.separator")); - long startTime = System.currentTimeMillis(); + final long loadPartitionsStartTime = System.nanoTime(); // load the list of DP partitions and return the list of partition specs // TODO: In a follow-up to HIVE-1361, we should refactor loadDynamicPartitions // to use Utilities.getFullDPSpecs() to get the list of full partSpecs. @@ -578,8 +579,9 @@ private DataContainer handleDynParts(Hive db, Table table, LoadTableDesc tbd, pushFeed(FeedType.DYNAMIC_PARTITIONS, dp.values()); } + final long estimatedLoadPartitionTime = System.nanoTime() - loadPartitionsStartTime; String loadTime = "\t Time taken to load dynamic partitions: " + - (System.currentTimeMillis() - startTime)/1000.0 + " seconds"; + TimeUnit.NANOSECONDS.toSeconds(estimatedLoadPartitionTime) + " seconds"; console.printInfo(loadTime); LOG.info(loadTime); @@ -588,7 +590,7 @@ private DataContainer handleDynParts(Hive db, Table table, LoadTableDesc tbd, " To turn off this error, set hive.error.on.empty.partition=false."); } - startTime = System.currentTimeMillis(); + final long getPartitionsStartTime = System.nanoTime(); // for each partition spec, get the partition // and put it to WriteEntity for post-exec hook for(Map.Entry, Partition> entry : dp.entrySet()) { @@ -626,8 +628,10 @@ private DataContainer handleDynParts(Hive db, Table table, LoadTableDesc tbd, } LOG.info("Loading partition " + entry.getKey()); } + + final long estimatedGetPartitionTime = System.nanoTime() - getPartitionsStartTime; console.printInfo("\t Time taken for adding to write entity : " + - (System.currentTimeMillis() - startTime)/1000.0 + " seconds"); + TimeUnit.NANOSECONDS.toSeconds(estimatedGetPartitionTime) + " seconds"); dc = null; // reset data container to prevent it being added again. return dc; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java index 3210ca5cf8..0899a8f76d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java @@ -677,9 +677,9 @@ public void processLine(Writable line) throws HiveException { // Report progress for each stderr line, but no more frequently than once // per minute. - long now = System.currentTimeMillis(); + final long now = System.nanoTime(); // reporter is a member variable of the Operator class. - if (now - lastReportTime > 60 * 1000 && reporter != null) { + if (now - lastReportTime > TimeUnit.MINUTES.toNanos(1L) && reporter != null) { if (LOG.isInfoEnabled()) { LOG.info("ErrorStreamProcessor calling reporter.progress()"); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java index dbf75b426d..66f0ae0925 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.hive.common.MemoryEstimate; @@ -800,7 +801,7 @@ private void expandAndRehash() { } private void expandAndRehashImpl(long capacity) { - long expandTime = System.currentTimeMillis(); + final long startTime = System.nanoTime(); final long[] oldRefs = refs; validateCapacity(capacity); long[] newRefs = new long[(int)capacity]; @@ -831,7 +832,7 @@ private void expandAndRehashImpl(long capacity) { this.largestNumberOfSteps = maxSteps; this.hashBitCount = newHashBitCount; this.resizeThreshold = (int)(capacity * loadFactor); - metricExpandsMs += (System.currentTimeMillis() - expandTime); + metricExpandsMs += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime); ++metricExpands; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java index 32cb38ae90..84492a16dc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java @@ -249,7 +249,7 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive Long lastReplId;// get list of events matching dbPattern & tblPattern // go through each event, and dump out each event to a event-level dump dir inside dumproot String validTxnList = null; - long waitUntilTime = 0; + long waitUntilTime = 0L; long bootDumpBeginReplId = -1; List tableList = work.replScope.includeAllTables() ? null : new ArrayList<>(); @@ -265,7 +265,7 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive work.dbNameOrPattern); long timeoutInMs = HiveConf.getTimeVar(conf, HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT, TimeUnit.MILLISECONDS); - waitUntilTime = System.currentTimeMillis() + timeoutInMs; + waitUntilTime = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutInMs); } // TODO : instead of simply restricting by message format, we should eventually @@ -324,7 +324,6 @@ private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive if (shouldExamineTablesToDump() || (tableList != null)) { // If required wait more for any transactions open at the time of starting the ACID bootstrap. if (needBootstrapAcidTablesDuringIncrementalDump()) { - assert (waitUntilTime > 0); validTxnList = getValidTxnListForReplDump(hiveDb, waitUntilTime); } @@ -624,7 +623,7 @@ String getValidTxnListForReplDump(Hive hiveDb, long waitUntilTime) throws HiveEx // of time to see if all open txns < current txn is getting aborted/committed. If not, then // we forcefully abort those txns just like AcidHouseKeeperService. ValidTxnList validTxnList = getTxnMgr().getValidTxns(); - while (System.currentTimeMillis() < waitUntilTime) { + while ((System.nanoTime() - waitUntilTime) > 0) { // If there are no txns which are open for the given ValidTxnList snapshot, then just return it. if (getOpenTxns(validTxnList).isEmpty()) { return validTxnList.toString(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java index 8becef1cd3..c1780766b0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java @@ -858,11 +858,11 @@ private String getSha(final Path localFile) throws IOException, IllegalArgumentE FSDataInputStream is = null; try { is = localFs.open(localFile); - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); sha256 = DigestUtils.sha256Hex(is); - long end = System.currentTimeMillis(); + final long estimatedTime = System.nanoTime() - startTime; LOG.info("Computed sha: {} for file: {} of length: {} in {} ms", sha256, localFile, - LlapUtil.humanReadableByteCount(fileStatus.getLen()), end - start); + LlapUtil.humanReadableByteCount(fileStatus.getLen()), TimeUnit.NANOSECONDS.toMillis(estimatedTime)); shaCache.put(key, sha256); } finally { if (is != null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java index 8b2890b496..33b74d3401 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java @@ -2006,7 +2006,7 @@ private static void scheduleSplits(ETLSplitStrategy splitStrategy, Context conte @Override public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { - long start = System.currentTimeMillis(); + final long startTime = System.nanoTime(); LOG.info("getSplits started"); Configuration conf = job; if (HiveConf.getBoolVar(job, HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED)) { @@ -2015,8 +2015,9 @@ private static void scheduleSplits(ETLSplitStrategy splitStrategy, Context conte } List result = generateSplitsInfo(conf, new Context(conf, numSplits, createExternalCaches())); - long end = System.currentTimeMillis(); - LOG.info("getSplits finished (#splits: {}). duration: {} ms", result.size(), (end - start)); + final long estimatedTime = System.nanoTime() - startTime; + LOG.info("getSplits finished (#splits: {}). duration: {} ms", result.size(), + TimeUnit.NANOSECONDS.toMillis(estimatedTime)); return result.toArray(new InputSplit[result.size()]); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java index e131e3d6af..bef1206e99 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java @@ -22,6 +22,7 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; import org.apache.hadoop.hive.ql.exec.FilterOperator; @@ -89,7 +90,7 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { index = new Index(); } - long sTime = System.currentTimeMillis(); + final long startTime = System.nanoTime(); // Create the lineage context LineageCtx lCtx = new LineageCtx(pctx, index); @@ -127,7 +128,11 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { topNodes.addAll(pctx.getTopOps().values()); ogw.startWalking(topNodes, null); - LOG.debug("Time taken for lineage transform={}", (System.currentTimeMillis() - sTime)); + if (LOG.isDebugEnabled()) { + final long estimatedTime = System.nanoTime() - startTime; + LOG.debug("Time taken for lineage transform={}ms", TimeUnit.NANOSECONDS.toMillis(estimatedTime)); + } + return pctx; } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java index e96a7ba289..a3ffa55be0 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java @@ -56,10 +56,12 @@ public void run() { TxnStore.MutexAPI.LockHandle handle = null; try { handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.CompactionHistory.name()); - long startTime = System.currentTimeMillis(); + final long startTime = System.nanoTime(); txnHandler.purgeCompactionHistory(); - LOG.debug("History reaper reaper ran for " + (System.currentTimeMillis() - startTime)/1000 + - "seconds."); + if (LOG.isDebugEnabled()) { + final long estimatedTime = System.nanoTime() - startTime; + LOG.debug("History reaper reaper ran for {} seconds", TimeUnit.NANOSECONDS.toSeconds(estimatedTime)); + } } catch(Throwable t) { LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t); } finally {