diff --git a/lucene/tools/forbiddenApis/solr.txt b/lucene/tools/forbiddenApis/solr.txt index 56d368829ad..38cf6676168 100644 --- a/lucene/tools/forbiddenApis/solr.txt +++ b/lucene/tools/forbiddenApis/solr.txt @@ -30,4 +30,7 @@ java.util.concurrent.ThreadPoolExecutor#(int,int,long,java.util.concurrent @defaultMessage Use slf4j classes instead org.apache.log4j.** -java.util.logging.** \ No newline at end of file +java.util.logging.** + +@defaultMessage Use RTimer/TimeOut/System.nanoTime for time comparisons, and `new Date()` output/debugging/stats of timestamps. If for some miscellaneous reason, you absolutely need to use this, use a SuppressForbidden. +java.lang.System#currentTimeMillis() diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index d4f48af91ca..6028719a3a6 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -87,6 +87,10 @@ Jetty 9.2.11.v20150529 Upgrading from Solr 5.3 ----------------------- +* SOLR-7859: The following APIs are now deprecated: + - SolrCore.getStartTime: Use SolrCore.getStartTimeStamp instead. + - SolrIndexSearcher.getOpenTime: Use SolrIndexSearcher.getOpenTimeStamp instead. + Detailed Change List ---------------------- @@ -108,6 +112,9 @@ New Features Bug Fixes ---------------------- +* SOLR-7859: Fix usage of currentTimeMillis instead of nanoTime in multiple places, + whitelist valid uses of currentTimeMillis (Ramkumar Aiyengar) + Optimizations ---------------------- @@ -345,7 +352,7 @@ Bug Fixes * SOLR-7866: VersionInfo caused an unhandled NPE when trying to determine the max value for the version field. (Timothy Potter) -* SOLR-7666 (and linked tickets): Many fixes to AngularJS Admin UI bringing it close to feature +* SOLR-7666 (and linked tickets): Many fixes to AngularJS Admin UI bringing it close to feature parity with existing UI. (Upayavira) Optimizations diff --git a/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java b/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java index df762505658..2fa2725c46e 100644 --- a/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java +++ b/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java @@ -19,6 +19,7 @@ package org.apache.solr.handler.dataimport; import com.sun.mail.imap.IMAPMessage; import org.apache.solr.handler.dataimport.config.ConfigNameConstants; +import org.apache.solr.util.RTimer; import org.apache.tika.Tika; import org.apache.tika.metadata.Metadata; import org.slf4j.Logger; @@ -623,8 +624,8 @@ public class MailEntityProcessor extends EntityProcessorBase { // envelopes; unless you're using gmail server-side filter, which is // fast LOG.info("Searching folder " + folder.getName() + " for messages"); - long searchAtMs = System.currentTimeMillis(); - + final RTimer searchTimer = new RTimer(); + // If using GMail, speed up the envelope processing by doing a // server-side // search for messages occurring on or after the fetch date (at @@ -652,9 +653,8 @@ public class MailEntityProcessor extends EntityProcessorBase { totalInFolder = messagesInCurBatch.length; folder.fetch(messagesInCurBatch, fp); current = 0; - long tookMs = (System.currentTimeMillis() - searchAtMs); LOG.info("Total messages : " + totalInFolder); - LOG.info("Search criteria applied. Batching disabled. Took " + tookMs + " (ms)"); + LOG.info("Search criteria applied. Batching disabled. Took {} (ms)", searchTimer.getTime()); } else { totalInFolder = folder.getMessageCount(); LOG.info("Total messages : " + totalInFolder); diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java index 23734751006..60f0811074f 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java @@ -27,6 +27,7 @@ import java.util.Map; import org.apache.commons.io.FileUtils; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.util.NamedList; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.SolrCore; import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.SolrQueryRequest; @@ -165,6 +166,7 @@ public abstract class AbstractDataImportHandlerTestCase extends return result; } + @SuppressForbidden(reason = "Needs currentTimeMillis to set modified time for a file") public static File createFile(File tmpdir, String name, byte[] content, boolean changeModifiedTime) throws IOException { File file = new File(tmpdir.getAbsolutePath() + File.separator + name); diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java index e4eff3cb2bd..9bad4446f53 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java @@ -1,6 +1,7 @@ package org.apache.solr.handler.dataimport; import junit.framework.Assert; +import org.apache.solr.common.util.SuppressForbidden; import org.junit.After; import org.junit.Before; @@ -445,6 +446,7 @@ public abstract class AbstractSqlEntityProcessorTestCase extends return nameArr[0]; } + @SuppressForbidden(reason = "Needs currentTimeMillis to set change time for SQL query") public IntChanges modifySomePeople() throws Exception { underlyingDataModified = true; int numberToChange = random().nextInt(people.length + 1); @@ -519,7 +521,8 @@ public abstract class AbstractSqlEntityProcessorTestCase extends c.addedKeys = addSet.toArray(new Integer[addSet.size()]); return c; } - + + @SuppressForbidden(reason = "Needs currentTimeMillis to set change time for SQL query") public String[] modifySomeCountries() throws Exception { underlyingDataModified = true; int numberToChange = random().nextInt(countries.length + 1); @@ -700,6 +703,8 @@ public abstract class AbstractSqlEntityProcessorTestCase extends log.debug(config); return config; } + + @SuppressForbidden(reason = "Needs currentTimeMillis to set change time for SQL query") @Override protected void populateData(Connection conn) throws Exception { Statement s = null; diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java index 15517d17b60..aa37168a735 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java @@ -18,6 +18,7 @@ package org.apache.solr.handler.dataimport; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; +import org.apache.solr.common.util.SuppressForbidden; import org.junit.Ignore; import org.junit.Test; @@ -133,6 +134,7 @@ public class TestFileListEntityProcessor extends AbstractDataImportHandlerTestCa return fList; } + @SuppressForbidden(reason = "Needs currentTimeMillis to set last modified time") @Test public void testNTOT() throws IOException { File tmpdir = createTempDir().toFile(); diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSimplePropertiesWriter.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSimplePropertiesWriter.java index eb26b492670..5bef61ad773 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSimplePropertiesWriter.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSimplePropertiesWriter.java @@ -11,6 +11,7 @@ import java.util.HashMap; import java.util.Locale; import java.util.Map; +import org.apache.solr.common.util.SuppressForbidden; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -46,7 +47,8 @@ public class TestSimplePropertiesWriter extends AbstractDIHJdbcTestCase { fileLocation = createTempDir().toFile().getAbsolutePath(); fileName = "the.properties"; } - + + @SuppressForbidden(reason = "Needs currentTimeMillis to construct date stamps") @Test public void testSimplePropertiesWriter() throws Exception { diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java index df2e31aafe9..01681862aad 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java @@ -28,6 +28,7 @@ import java.util.Map; import org.apache.solr.cloud.AbstractZkTestCase; import org.apache.solr.cloud.ZkTestServer; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.CoreContainer; import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.SolrQueryRequest; @@ -86,6 +87,7 @@ public class TestZKPropertiesWriter extends AbstractDataImportHandlerTestCase { cc = null; } + @SuppressForbidden(reason = "Needs currentTimeMillis to construct date stamps") @Test public void testZKPropertiesWriter() throws Exception { // test using ZooKeeper diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java index 721706c9785..bd804e02d1f 100644 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java +++ b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java @@ -40,6 +40,7 @@ import org.apache.lucene.misc.IndexMergeTool; import org.apache.lucene.store.Directory; import org.apache.lucene.store.NoLockFactory; import org.apache.solr.store.hdfs.HdfsDirectory; +import org.apache.solr.util.RTimer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,38 +134,37 @@ public class TreeMergeOutputFormat extends FileOutputFormat context.setStatus("Logically merging " + shards.size() + " shards into one shard"); LOG.info("Logically merging " + shards.size() + " shards into one shard: " + workDir); - long start = System.nanoTime(); + RTimer timer = new RTimer(); writer.addIndexes(indexes); // TODO: avoid intermediate copying of files into dst directory; rename the files into the dir instead (cp -> rename) // This can improve performance and turns this phase into a true "logical" merge, completing in constant time. // See https://issues.apache.org/jira/browse/LUCENE-4746 - + + timer.stop(); if (LOG.isDebugEnabled()) { - context.getCounter(SolrCounters.class.getName(), SolrCounters.LOGICAL_TREE_MERGE_TIME.toString()).increment(System.currentTimeMillis() - start); + context.getCounter(SolrCounters.class.getName(), SolrCounters.LOGICAL_TREE_MERGE_TIME.toString()).increment((long) timer.getTime()); } - float secs = (System.nanoTime() - start) / (float)(10^9); - LOG.info("Logical merge took {} secs", secs); + LOG.info("Logical merge took {}ms", timer.getTime()); int maxSegments = context.getConfiguration().getInt(TreeMergeMapper.MAX_SEGMENTS_ON_TREE_MERGE, Integer.MAX_VALUE); context.setStatus("Optimizing Solr: forcing mtree merge down to " + maxSegments + " segments"); LOG.info("Optimizing Solr: forcing tree merge down to {} segments", maxSegments); - start = System.nanoTime(); + timer = new RTimer(); if (maxSegments < Integer.MAX_VALUE) { writer.forceMerge(maxSegments); // TODO: consider perf enhancement for no-deletes merges: bulk-copy the postings data // see http://lucene.472066.n3.nabble.com/Experience-with-large-merge-factors-tp1637832p1647046.html } + timer.stop(); if (LOG.isDebugEnabled()) { - context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_TREE_MERGE_TIME.toString()).increment(System.currentTimeMillis() - start); + context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_TREE_MERGE_TIME.toString()).increment((long) timer.getTime()); } - secs = (System.nanoTime() - start) / (float)(10^9); - LOG.info("Optimizing Solr: done forcing tree merge down to {} segments in {} secs", maxSegments, secs); - - start = System.nanoTime(); + LOG.info("Optimizing Solr: done forcing tree merge down to {} segments in {}ms", maxSegments, timer.getTime()); + + timer = new RTimer(); LOG.info("Optimizing Solr: Closing index writer"); writer.close(); - secs = (System.nanoTime() - start) / (float)(10^9); - LOG.info("Optimizing Solr: Done closing index writer in {} secs", secs); + LOG.info("Optimizing Solr: Done closing index writer in {}ms", timer.getTime()); context.setStatus("Done"); } finally { heartBeater.cancelHeartBeat(); diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java index 342a9f6bf91..063381dec28 100644 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java +++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileSystem; @@ -75,6 +76,7 @@ import org.apache.solr.hadoop.hack.MiniMRClientClusterFactory; import org.apache.solr.morphlines.solr.AbstractSolrMorphlineTestBase; import org.apache.solr.util.BadHdfsThreadsFilter; import org.apache.solr.util.BadMrClusterThreadsFilter; +import org.apache.solr.util.TimeOut; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -638,11 +640,10 @@ public class MorphlineGoLiveMiniMRTest extends AbstractFullDistribZkTestBase { request.setPath("/admin/collections"); cloudClient.request(request); - - long timeout = System.currentTimeMillis() + 10000; + final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS); while (cloudClient.getZkStateReader().getClusterState().hasCollection(replicatedCollection)) { - if (System.currentTimeMillis() > timeout) { - throw new AssertionError("Timeout waiting to see removed collection leave clusterstate"); + if (timeout.hasTimedOut()) { + throw new AssertionError("Timeout waiting to see removed collection leave clusterstate"); } Thread.sleep(200); diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java index b399b7a9552..cf872abdb1c 100644 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java +++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java @@ -18,7 +18,6 @@ package org.apache.solr.hadoop.hack; import java.io.File; import java.io.IOException; -import java.util.Random; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,7 +29,6 @@ import org.apache.hadoop.mapred.MapTaskCompletionEventsUpdate; import org.apache.hadoop.mapred.TaskCompletionEvent; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.lucene.util.LuceneTestCase; @@ -176,14 +174,6 @@ public class MiniMRCluster { int numTaskTrackers, String namenode, int numDir, String[] racks, String[] hosts, UserGroupInformation ugi, JobConf conf, int numTrackerToExclude) throws Exception { - this(jobTrackerPort, taskTrackerPort, numTaskTrackers, namenode, numDir, - racks, hosts, ugi, conf, numTrackerToExclude, new Clock()); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir, String[] racks, - String[] hosts, UserGroupInformation ugi, JobConf conf, - int numTrackerToExclude, Clock clock) throws Exception { if (conf == null) conf = new JobConf(); FileSystem.setDefaultUri(conf, namenode); String identifier = this.getClass().getSimpleName() + "_" @@ -273,11 +263,4 @@ public class MiniMRCluster { LOG.error(e); } } - - static class Clock { - long getTime() { - return System.currentTimeMillis(); - } - } - } diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java index d02726657e4..6931f87e2ce 100644 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java +++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java @@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; @@ -102,7 +101,7 @@ public class MiniYARNCluster extends CompositeService { // symlink as the test working directory. String targetPath = targetWorkDir.getAbsolutePath(); File link = new File(System.getProperty("java.io.tmpdir"), - String.valueOf(System.currentTimeMillis())); + String.valueOf(System.nanoTime())); String linkPath = link.getAbsolutePath(); try { diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java b/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java index fea083292d5..b5db71bd8be 100644 --- a/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java +++ b/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java @@ -11,6 +11,7 @@ import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction; import org.apache.solr.core.CoreContainer; +import org.apache.solr.util.RTimer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,7 +74,7 @@ public class LeaderInitiatedRecoveryThread extends Thread { } public void run() { - long startMs = System.currentTimeMillis(); + RTimer timer = new RTimer(); try { sendRecoveryCommandWithRetry(); } catch (Exception exc) { @@ -84,8 +85,7 @@ public class LeaderInitiatedRecoveryThread extends Thread { throw new SolrException(ErrorCode.SERVER_ERROR, exc); } } - long diffMs = (System.currentTimeMillis() - startMs); - log.info(getName()+" completed successfully after running for "+Math.round(diffMs/1000L)+" secs"); + log.info("{} completed successfully after running for {}ms", getName(), timer.getTime()); } protected void sendRecoveryCommandWithRetry() throws Exception { diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java index 6f6af431256..400e2dbdcd1 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java @@ -80,6 +80,8 @@ import org.apache.solr.handler.component.ShardHandlerFactory; import org.apache.solr.handler.component.ShardRequest; import org.apache.solr.handler.component.ShardResponse; import org.apache.solr.update.SolrIndexSplitter; +import org.apache.solr.util.RTimer; +import org.apache.solr.util.TimeOut; import org.apache.solr.util.stats.Snapshot; import org.apache.solr.util.stats.Timer; import org.apache.zookeeper.CreateMode; @@ -722,9 +724,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler } private boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException { - long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS); + TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS); boolean deleted = false; - while (System.nanoTime() < waitUntil) { + while (! timeout.hasTimedOut()) { Thread.sleep(100); DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName); if(docCollection != null) { @@ -775,10 +777,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler Utils.toJSON(m)); // wait for a while until we don't see the collection - long now = System.nanoTime(); - long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); boolean removed = false; - while (System.nanoTime() < timeout) { + while (! timeout.hasTimedOut()) { Thread.sleep(100); removed = !zkStateReader.getClusterState().hasCollection(collection); if (removed) { @@ -815,11 +816,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler throws KeeperException, InterruptedException { final String collectionName = message.getStr(COLLECTION_PROP); - // wait for a while until the state format changes - long now = System.nanoTime(); - long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); boolean firstLoop = true; - while (System.nanoTime() < timeout) { + // wait for a while until the state format changes + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); + while (! timeout.hasTimedOut()) { DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName); if (collection == null) { throw new SolrException(ErrorCode.BAD_REQUEST, "Collection: " + collectionName + " not found"); @@ -875,11 +875,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler private void checkForAlias(String name, String value) { - long now = System.nanoTime(); - long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); boolean success = false; Aliases aliases = null; - while (System.nanoTime() < timeout) { + while (! timeout.hasTimedOut()) { aliases = zkStateReader.getAliases(); String collections = aliases.getCollectionAlias(name); if (collections != null && collections.equals(value)) { @@ -894,11 +893,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler private void checkForAliasAbsence(String name) { - long now = System.nanoTime(); - long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); boolean success = false; Aliases aliases = null; - while (System.nanoTime() < timeout) { + while (! timeout.hasTimedOut()) { aliases = zkStateReader.getAliases(); String collections = aliases.getCollectionAlias(name); if (collections == null) { @@ -959,9 +957,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler Overseer.getInQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message)); // wait for a while until we see the shard - long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); boolean created = false; - while (System.nanoTime() < waitUntil) { + while (! timeout.hasTimedOut()) { Thread.sleep(100); created = zkStateReader.getClusterState().getCollection(collectionName).getSlice(sliceName) != null; if (created) break; @@ -1460,7 +1458,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler private void waitForNewShard(String collectionName, String sliceName) throws KeeperException, InterruptedException { log.info("Waiting for slice {} of collection {} to be available", sliceName, collectionName); - long startTime = System.currentTimeMillis(); + RTimer timer = new RTimer(); int retryCount = 320; while (retryCount-- > 0) { DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName); @@ -1470,8 +1468,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler } Slice slice = collection.getSlice(sliceName); if (slice != null) { - log.info("Waited for {} seconds for slice {} of collection {} to be available", - (System.currentTimeMillis() - startTime) / 1000, sliceName, collectionName); + log.info("Waited for {}ms for slice {} of collection {} to be available", + timer.getTime(), sliceName, collectionName); return; } Thread.sleep(1000); @@ -1479,7 +1477,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler } throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find new slice " + sliceName + " in collection " + collectionName - + " even after waiting for " + (System.currentTimeMillis() - startTime) / 1000 + " seconds" + + " even after waiting for " + timer.getTime() + "ms" ); } @@ -1541,10 +1539,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler Overseer.getInQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m)); // wait for a while until we don't see the shard - long now = System.nanoTime(); - long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); boolean removed = false; - while (System.nanoTime() < timeout) { + while (! timeout.hasTimedOut()) { Thread.sleep(100); removed = zkStateReader.getClusterState().getSlice(collection, sliceId) == null; if (removed) { @@ -1672,17 +1669,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler "routeKey", SolrIndexSplitter.getRouteKey(splitKey) + "!", "range", splitRange.toString(), "targetCollection", targetCollection.getName(), - // TODO: look at using nanoTime here? - "expireAt", String.valueOf(System.currentTimeMillis() + timeout)); + "expireAt", RoutingRule.makeExpiryAt(timeout)); log.info("Adding routing rule: " + m); Overseer.getInQueue(zkStateReader.getZkClient()).offer( Utils.toJSON(m)); // wait for a while until we see the new rule log.info("Waiting to see routing rule updated in clusterstate"); - long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS); + TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS); boolean added = false; - while (System.nanoTime() < waitUntil) { + while (! waitUntil.hasTimedOut()) { Thread.sleep(100); Map rules = zkStateReader.getClusterState().getSlice(sourceCollection.getName(), sourceSlice.getName()).getRoutingRules(); if (rules != null) { @@ -2006,9 +2002,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler Overseer.getInQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message)); // wait for a while until we don't see the collection - long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS); boolean created = false; - while (System.nanoTime() < waitUntil) { + while (! waitUntil.hasTimedOut()) { Thread.sleep(100); created = zkStateReader.getClusterState().getCollections().contains(collectionName); if(created) break; @@ -2142,7 +2138,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler private Map waitToSeeReplicasInState(String collectionName, Collection coreNames) throws InterruptedException { Map result = new HashMap<>(); - long endTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS); + TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); while (true) { DocCollection coll = zkStateReader.getClusterState().getCollection(collectionName); for (String coreName : coreNames) { @@ -2160,7 +2156,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler if (result.size() == coreNames.size()) { return result; } - if (System.nanoTime() > endTime) { + if (timeout.hasTimedOut()) { throw new SolrException(ErrorCode.SERVER_ERROR, "Timed out waiting to see all replicas: " + coreNames + " in cluster state."); } diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java index 6802ae15f0a..efa28b06fda 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCore.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java @@ -53,6 +53,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; @@ -178,7 +179,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable { private final UpdateHandler updateHandler; private final SolrCoreState solrCoreState; - private final long startTime; + private final Date startTime = new Date(); + private final long startNanoTime = System.nanoTime(); private final RequestHandlers reqHandlers; private final PluginBag searchComponents = new PluginBag<>(SearchComponent.class, this); private final PluginBag updateProcessors = new PluginBag<>(UpdateRequestProcessorFactory.class, this); @@ -194,7 +196,18 @@ public final class SolrCore implements SolrInfoMBean, Closeable { private final ReentrantLock ruleExpiryLock; - public long getStartTime() { return startTime; } + public Date getStartTimeStamp() { return startTime; } + + @Deprecated + public long getStartTime() { return startTime.getTime(); } + + public long getStartNanoTime() { + return startNanoTime; + } + + public long getUptimeMs() { + return TimeUnit.MILLISECONDS.convert(System.nanoTime() - startNanoTime, TimeUnit.NANOSECONDS); + } private final RestManager restManager; @@ -674,7 +687,6 @@ public final class SolrCore implements SolrInfoMBean, Closeable { this.ulogDir = null; this.solrConfig = null; this.configSetProperties = null; - this.startTime = System.currentTimeMillis(); this.maxWarmingSearchers = 2; // we don't have a config yet, just pick a number. this.slowQueryThresholdMillis = -1; this.resourceLoader = null; @@ -741,7 +753,6 @@ public final class SolrCore implements SolrInfoMBean, Closeable { this.schema = initSchema(config, schema); - this.startTime = System.currentTimeMillis(); this.maxWarmingSearchers = config.maxWarmingSearchers; this.slowQueryThresholdMillis = config.slowQueryThresholdMillis; @@ -2431,7 +2442,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable { public NamedList getStatistics() { NamedList lst = new SimpleOrderedMap<>(); lst.add("coreName", name==null ? "(null)" : name); - lst.add("startTime", new Date(startTime)); + lst.add("startTime", startTime); lst.add("refCount", getOpenCount()); lst.add("instanceDir", resourceLoader.getInstanceDir()); lst.add("indexDir", getIndexDir()); diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java index c8c06827aad..3b64b163b91 100644 --- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java +++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java @@ -98,7 +98,7 @@ class CdcrReplicatorState { errorCounters.put(error, 0l); } errorCounters.put(error, errorCounters.get(error) + 1); - errorsQueue.add(new ErrorQueueEntry(error, System.currentTimeMillis())); + errorsQueue.add(new ErrorQueueEntry(error, new Date())); consecutiveErrors++; } @@ -133,7 +133,7 @@ class CdcrReplicatorState { Iterator it = errorsQueue.iterator(); while (it.hasNext()) { ErrorQueueEntry entry = it.next(); - lastErrors.add(new String[]{DateFormatUtil.formatExternal(new Date(entry.timestamp)), entry.type.toLower()}); + lastErrors.add(new String[]{DateFormatUtil.formatExternal(entry.timestamp), entry.type.toLower()}); } } return lastErrors; @@ -240,9 +240,9 @@ class CdcrReplicatorState { private class ErrorQueueEntry { private ErrorType type; - private long timestamp; + private Date timestamp; - private ErrorQueueEntry(ErrorType type, long timestamp) { + private ErrorQueueEntry(ErrorType type, Date timestamp) { this.type = type; this.timestamp = timestamp; } diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java index a0a812dd0c5..1cb2ffaa41a 100644 --- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java +++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java @@ -72,6 +72,7 @@ import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.FastInputStream; import org.apache.solr.common.util.NamedList; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.DirectoryFactory; import org.apache.solr.core.DirectoryFactory.DirContext; import org.apache.solr.core.IndexDeletionPolicyWrapper; @@ -87,6 +88,7 @@ import org.apache.solr.util.DefaultSolrThreadFactory; import org.apache.solr.util.FileUtils; import org.apache.solr.util.PropertiesInputStream; import org.apache.solr.util.PropertiesOutputStream; +import org.apache.solr.util.RTimer; import org.apache.solr.util.RefCounted; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -132,7 +134,8 @@ public class IndexFetcher { final ReplicationHandler replicationHandler; - private volatile long replicationStartTime; + private volatile Date replicationStartTimeStamp; + private RTimer replicationTimer; private final SolrCore solrCore; @@ -275,7 +278,7 @@ public class IndexFetcher { boolean cleanupDone = false; boolean successfulInstall = false; - replicationStartTime = System.currentTimeMillis(); + markReplicationStart(); Directory tmpIndexDir = null; String tmpIndex; Directory indexDir = null; @@ -339,6 +342,7 @@ public class IndexFetcher { return true; } + // TODO: Should we be comparing timestamps (across machines) here? if (!forceReplication && IndexDeletionPolicyWrapper.getCommitTimestamp(commit) == latestVersion) { //master and slave are already in sync just return LOG.info("Slave in sync with master."); @@ -434,9 +438,7 @@ public class IndexFetcher { if (tlogFilesToDownload != null) { downloadTlogFiles(timestamp, latestGeneration); } - LOG.info("Total time taken for download : " - + ((System.currentTimeMillis() - replicationStartTime) / 1000) - + " secs"); + LOG.info("Total time taken for download: {} secs", getReplicationTimeElapsed()); Collection> modifiedConfFiles = getModifiedConfFiles(confFilesToDownload); if (!modifiedConfFiles.isEmpty()) { reloadCore = true; @@ -515,7 +517,7 @@ public class IndexFetcher { successfulInstall = fetchLatestIndex(true, reloadCore); } - replicationStartTime = 0; + markReplicationStop(); return successfulInstall; } catch (ReplicationHandlerException e) { LOG.error("User aborted Replication"); @@ -548,7 +550,7 @@ public class IndexFetcher { core.getUpdateHandler().getSolrCoreState().setLastReplicateIndexSuccess(successfulInstall); filesToDownload = filesDownloaded = confFilesDownloaded = confFilesToDownload = null; - replicationStartTime = 0; + markReplicationStop(); dirFileFetcher = null; localFileFetcher = null; if (fsyncService != null && !fsyncService.isShutdown()) fsyncService @@ -610,6 +612,7 @@ public class IndexFetcher { * restarts. * @throws IOException on IO error */ + @SuppressForbidden(reason = "Need currentTimeMillis for debugging/stats") private void logReplicationTimeAndConfFiles(Collection> modifiedConfFiles, boolean successfulInstall) throws IOException { List confFiles = new ArrayList<>(); if (modifiedConfFiles != null && !modifiedConfFiles.isEmpty()) @@ -618,7 +621,7 @@ public class IndexFetcher { Properties props = replicationHandler.loadReplicationProperties(); long replicationTime = System.currentTimeMillis(); - long replicationTimeTaken = (replicationTime - getReplicationStartTime()) / 1000; + long replicationTimeTaken = getReplicationTimeElapsed(); Directory dir = null; try { dir = solrCore.getDirectoryFactory().get(solrCore.getDataDir(), DirContext.META_DATA, solrCore.getSolrConfig().indexConfig.lockType); @@ -1219,14 +1222,25 @@ public class IndexFetcher { stop = true; } - long getReplicationStartTime() { - return replicationStartTime; + @SuppressForbidden(reason = "Need currentTimeMillis for debugging/stats") + private void markReplicationStart() { + replicationTimer = new RTimer(); + replicationStartTimeStamp = new Date(); + } + + private void markReplicationStop() { + replicationStartTimeStamp = null; + replicationTimer = null; + } + + Date getReplicationStartTimeStamp() { + return replicationStartTimeStamp; } long getReplicationTimeElapsed() { long timeElapsed = 0; - if (getReplicationStartTime() > 0) - timeElapsed = TimeUnit.SECONDS.convert(System.currentTimeMillis() - getReplicationStartTime(), TimeUnit.MILLISECONDS); + if (replicationStartTimeStamp != null) + timeElapsed = TimeUnit.SECONDS.convert((long) replicationTimer.getTime(), TimeUnit.MILLISECONDS); return timeElapsed; } diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java index ac736fa7543..86d82b5a965 100644 --- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java @@ -36,6 +36,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -71,6 +72,7 @@ import org.apache.solr.common.util.FastOutputStream; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.common.util.StrUtils; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.CloseHook; import org.apache.solr.core.DirectoryFactory; import org.apache.solr.core.DirectoryFactory.DirContext; @@ -182,7 +184,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw private final Map confFileInfoCache = new HashMap<>(); - private Integer reserveCommitDuration = readInterval("00:00:10"); + private Integer reserveCommitDuration = readIntervalMs("00:00:10"); volatile IndexCommit indexCommitPoint; @@ -190,8 +192,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw private AtomicBoolean replicationEnabled = new AtomicBoolean(true); - private Integer pollInterval; - + private Long pollIntervalNs; private String pollIntervalStr; /** @@ -693,10 +694,15 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw return pollDisabled.get(); } - Long getNextScheduledExecTime() { - Long nextTime = null; + @SuppressForbidden(reason = "Need currentTimeMillis, to output next execution time in replication details") + private void markScheduledExecutionStart() { + executorStartTime = System.currentTimeMillis(); + } + + private Date getNextScheduledExecTime() { + Date nextTime = null; if (executorStartTime > 0) - nextTime = executorStartTime + pollInterval; + nextTime = new Date(executorStartTime + TimeUnit.MILLISECONDS.convert(pollIntervalNs, TimeUnit.NANOSECONDS)); return nextTime; } @@ -843,8 +849,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw if (getPollInterval() != null) { slave.add(POLL_INTERVAL, getPollInterval()); } - if (getNextScheduledExecTime() != null && !isPollingDisabled()) { - slave.add(NEXT_EXECUTION_AT, new Date(getNextScheduledExecTime()).toString()); + Date nextScheduled = getNextScheduledExecTime(); + if (nextScheduled != null && !isPollingDisabled()) { + slave.add(NEXT_EXECUTION_AT, nextScheduled.toString()); } else if (isPollingDisabled()) { slave.add(NEXT_EXECUTION_AT, "Polling disabled"); } @@ -915,8 +922,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw long estimatedTimeRemaining = 0; - if (fetcher.getReplicationStartTime() > 0) { - slave.add("replicationStartTime", new Date(fetcher.getReplicationStartTime()).toString()); + Date replicationStartTimeStamp = fetcher.getReplicationStartTimeStamp(); + if (replicationStartTimeStamp != null) { + slave.add("replicationStartTime", replicationStartTimeStamp.toString()); } long elapsed = fetcher.getReplicationTimeElapsed(); slave.add("timeElapsed", String.valueOf(elapsed) + "s"); @@ -1030,8 +1038,8 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw private void setupPolling(String intervalStr) { pollIntervalStr = intervalStr; - pollInterval = readInterval(pollIntervalStr); - if (pollInterval == null || pollInterval <= 0) { + pollIntervalNs = readIntervalNs(pollIntervalStr); + if (pollIntervalNs == null || pollIntervalNs <= 0) { LOG.info(" No value set for 'pollInterval'. Timer Task not started."); return; } @@ -1045,7 +1053,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw } try { LOG.debug("Polling for index modifications"); - executorStartTime = System.currentTimeMillis(); + markScheduledExecutionStart(); doFetch(null, false); } catch (Exception e) { LOG.error("Exception in fetching index", e); @@ -1054,9 +1062,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw }; executorService = Executors.newSingleThreadScheduledExecutor( new DefaultSolrThreadFactory("indexFetcher")); - long initialDelay = pollInterval - (System.currentTimeMillis() % pollInterval); - executorService.scheduleAtFixedRate(task, initialDelay, pollInterval, TimeUnit.MILLISECONDS); - LOG.info("Poll Scheduled at an interval of " + pollInterval + "ms"); + // Randomize initial delay, with a minimum of 1ms + long initialDelayNs = new Random().nextLong() % pollIntervalNs + + TimeUnit.NANOSECONDS.convert(1, TimeUnit.MILLISECONDS); + executorService.scheduleAtFixedRate(task, initialDelayNs, pollIntervalNs, TimeUnit.NANOSECONDS); + LOG.info("Poll scheduled at an interval of {}ms", + TimeUnit.MILLISECONDS.convert(pollIntervalNs, TimeUnit.NANOSECONDS)); } @Override @@ -1178,7 +1189,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw } String reserve = (String) master.get(RESERVE); if (reserve != null && !reserve.trim().equals("")) { - reserveCommitDuration = readInterval(reserve); + reserveCommitDuration = readIntervalMs(reserve); } LOG.info("Commits will be reserved for " + reserveCommitDuration); isMaster = true; @@ -1534,36 +1545,34 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw } - static Integer readInterval(String interval) { + private static Integer readIntervalMs(String interval) { + return (int) TimeUnit.MILLISECONDS.convert(readIntervalNs(interval), TimeUnit.NANOSECONDS); + } + + private static Long readIntervalNs(String interval) { if (interval == null) return null; int result = 0; - if (interval != null) { - Matcher m = INTERVAL_PATTERN.matcher(interval.trim()); - if (m.find()) { - String hr = m.group(1); - String min = m.group(2); - String sec = m.group(3); - result = 0; - try { - if (sec != null && sec.length() > 0) - result += Integer.parseInt(sec); - if (min != null && min.length() > 0) - result += (60 * Integer.parseInt(min)); - if (hr != null && hr.length() > 0) - result += (60 * 60 * Integer.parseInt(hr)); - result *= 1000; - } catch (NumberFormatException e) { - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, - INTERVAL_ERR_MSG); - } - } else { - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, - INTERVAL_ERR_MSG); + Matcher m = INTERVAL_PATTERN.matcher(interval.trim()); + if (m.find()) { + String hr = m.group(1); + String min = m.group(2); + String sec = m.group(3); + result = 0; + try { + if (sec != null && sec.length() > 0) + result += Integer.parseInt(sec); + if (min != null && min.length() > 0) + result += (60 * Integer.parseInt(min)); + if (hr != null && hr.length() > 0) + result += (60 * 60 * Integer.parseInt(hr)); + return TimeUnit.NANOSECONDS.convert(result, TimeUnit.SECONDS); + } catch (NumberFormatException e) { + throw new SolrException(ErrorCode.SERVER_ERROR, INTERVAL_ERR_MSG); } - + } else { + throw new SolrException(ErrorCode.SERVER_ERROR, INTERVAL_ERR_MSG); } - return result; } private static final String LOCATION = "location"; diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java index 7c142ef03db..543e6805bbf 100644 --- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java +++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java @@ -21,6 +21,7 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.PluginInfo; import org.apache.solr.core.PluginBag; import org.apache.solr.core.SolrCore; @@ -56,9 +57,16 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo private final AtomicLong numErrors = new AtomicLong(); private final AtomicLong numTimeouts = new AtomicLong(); private final Timer requestTimes = new Timer(); - private final long handlerStart = System.currentTimeMillis(); + + private final long handlerStart; + private PluginInfo pluginInfo; + @SuppressForbidden(reason = "Need currentTimeMillis, used only for stats output") + public RequestHandlerBase() { + handlerStart = System.currentTimeMillis(); + } + /** * Initializes the {@link org.apache.solr.request.SolrRequestHandler} by creating three {@link org.apache.solr.common.params.SolrParams} named. * diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java index a508467023f..322a92b1ec7 100644 --- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java @@ -72,6 +72,7 @@ import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.schema.SchemaManager; import org.apache.solr.util.CommandOperation; import org.apache.solr.util.DefaultSolrThreadFactory; +import org.apache.solr.util.RTimer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -644,7 +645,7 @@ public class SolrConfigHandler extends RequestHandlerBase { String prop, int expectedVersion, int maxWaitSecs) { - long startMs = System.currentTimeMillis(); + final RTimer timer = new RTimer(); // get a list of active replica cores to query for the schema zk version (skipping this core of course) List concurrentTasks = new ArrayList<>(); @@ -702,10 +703,8 @@ public class SolrConfigHandler extends RequestHandlerBase { ExecutorUtil.shutdownNowAndAwaitTermination(parallelExecutor); } - long diffMs = (System.currentTimeMillis() - startMs); - log.info(formatString( - "Took {0} secs to set the property {1} to be of version {2} for collection {3}", - Math.round(diffMs / 1000d), prop, expectedVersion, collection)); + log.info("Took {}ms to set the property {} to be of version {} for collection {}", + timer.getTime(), prop, expectedVersion, collection); } public static List getActiveReplicaCoreUrls(ZkController zkController, @@ -754,13 +753,13 @@ public class SolrConfigHandler extends RequestHandlerBase { @Override public Boolean call() throws Exception { - long startTime = System.currentTimeMillis(); + final RTimer timer = new RTimer(); int attempts = 0; try (HttpSolrClient solr = new HttpSolrClient(coreUrl)) { // eventually, this loop will get killed by the ExecutorService's timeout while (true) { try { - long timeElapsed = (System.currentTimeMillis() - startTime) / 1000; + long timeElapsed = (long) timer.getTime() / 1000; if (timeElapsed >= maxWait) { return false; } diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java index 17525b35ab9..16730852c3a 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Date; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; @@ -31,7 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import com.google.common.collect.ImmutableMap; @@ -1165,8 +1163,8 @@ public class CoreAdminHandler extends RequestHandlerBase { info.add("dataDir", normalizePath(core.getDataDir())); info.add("config", core.getConfigResource()); info.add("schema", core.getSchemaResource()); - info.add("startTime", new Date(core.getStartTime())); - info.add("uptime", System.currentTimeMillis() - core.getStartTime()); + info.add("startTime", core.getStartTimeStamp()); + info.add("uptime", core.getUptimeMs()); if (isIndexInfoNeeded) { RefCounted searcher = core.getSearcher(); try { diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java index 506cd470558..fd2fda357e2 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java @@ -130,7 +130,7 @@ public class SystemInfoHandler extends RequestHandlerBase info.add( "now", new Date() ); // Start Time - info.add( "start", new Date(core.getStartTime()) ); + info.add( "start", core.getStartTimeStamp() ); // Solr Home SimpleOrderedMap dirs = new SimpleOrderedMap<>(); diff --git a/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java b/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java index e17cb2d122c..48fea5dcd07 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java @@ -36,6 +36,7 @@ import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.search.DocList; import org.apache.solr.search.QueryParsing; @@ -145,7 +146,8 @@ public class DebugComponent extends SearchComponent rb.rsp.addToLog(CommonParams.REQUEST_ID, rid); //to see it in the logs of the landing core } - + + @SuppressForbidden(reason = "Need currentTimeMillis, only used for naming") private String generateRid(ResponseBuilder rb) { String hostName = rb.req.getCore().getCoreDescriptor().getCoreContainer().getHostName(); return hostName + "-" + rb.req.getCore().getName() + "-" + System.currentTimeMillis() + "-" + ridCounter.getAndIncrement(); diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java index b6798700749..58bf8da1eea 100644 --- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java +++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java @@ -17,6 +17,7 @@ package org.apache.solr.request; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.util.RTimer; @@ -54,12 +55,15 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl protected Map json; private final RTimer requestTimer; + protected final long startTime; + @SuppressForbidden(reason = "Need currentTimeMillis to get start time for request (to be used for stats/debugging)") public SolrQueryRequestBase(SolrCore core, SolrParams params, RTimer requestTimer) { this.core = core; this.schema = null == core ? null : core.getLatestSchema(); this.params = this.origParams = params; this.requestTimer = requestTimer; + this.startTime = System.currentTimeMillis(); } public SolrQueryRequestBase(SolrCore core, SolrParams params) { @@ -88,7 +92,7 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl this.params = params; } - protected final long startTime=System.currentTimeMillis(); + // Get the start time of this request in milliseconds @Override public long getStartTime() { diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java index 8a0ea630da2..a7e06e047f4 100644 --- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java +++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java @@ -47,6 +47,7 @@ import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.rest.schema.FieldTypeXmlAdapter; import org.apache.solr.util.DefaultSolrThreadFactory; import org.apache.solr.util.FileUtils; +import org.apache.solr.util.RTimer; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -212,7 +213,7 @@ public final class ManagedIndexSchema extends IndexSchema { public static void waitForSchemaZkVersionAgreement(String collection, String localCoreNodeName, int schemaZkVersion, ZkController zkController, int maxWaitSecs) { - long startMs = System.currentTimeMillis(); + RTimer timer = new RTimer(); // get a list of active replica cores to query for the schema zk version (skipping this core of course) List concurrentTasks = new ArrayList<>(); @@ -271,9 +272,8 @@ public final class ManagedIndexSchema extends IndexSchema { parallelExecutor.shutdownNow(); } - long diffMs = (System.currentTimeMillis() - startMs); - log.info("Took "+Math.round(diffMs/1000d)+" secs for "+concurrentTasks.size()+ - " replicas to apply schema update version "+schemaZkVersion+" for collection "+collection); + log.info("Took {}ms for {} replicas to apply schema update version {} for collection {}", + timer.getTime(), concurrentTasks.size(), schemaZkVersion, collection); } protected static List getActiveReplicaCoreUrls(ZkController zkController, String collection, String localCoreNodeName) { diff --git a/solr/core/src/java/org/apache/solr/schema/OpenExchangeRatesOrgProvider.java b/solr/core/src/java/org/apache/solr/schema/OpenExchangeRatesOrgProvider.java index e6927cefef4..219702f1abb 100644 --- a/solr/core/src/java/org/apache/solr/schema/OpenExchangeRatesOrgProvider.java +++ b/solr/core/src/java/org/apache/solr/schema/OpenExchangeRatesOrgProvider.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; +import org.apache.solr.common.util.SuppressForbidden; import org.noggit.JSONParser; import org.apache.lucene.analysis.util.ResourceLoader; import org.apache.solr.common.SolrException; @@ -84,12 +85,9 @@ public class OpenExchangeRatesOrgProvider implements ExchangeRateProvider { if (sourceCurrencyCode == null || targetCurrencyCode == null) { throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot get exchange rate; currency was null."); } - - if ((rates.getTimestamp() + refreshIntervalSeconds)*1000 < System.currentTimeMillis()) { - log.debug("Refresh interval has expired. Refreshing exchange rates."); - reload(); - } - + + reloadIfExpired(); + Double source = (Double) rates.getRates().get(sourceCurrencyCode); Double target = (Double) rates.getRates().get(targetCurrencyCode); @@ -102,6 +100,14 @@ public class OpenExchangeRatesOrgProvider implements ExchangeRateProvider { return target / source; } + @SuppressForbidden(reason = "Need currentTimeMillis, for comparison with stamp in an external file") + private void reloadIfExpired() { + if ((rates.getTimestamp() + refreshIntervalSeconds)*1000 < System.currentTimeMillis()) { + log.debug("Refresh interval has expired. Refreshing exchange rates."); + reload(); + } + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java index 8f3ffcb22ba..76c4b58ef4d 100644 --- a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java @@ -61,6 +61,7 @@ import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.schema.TrieField; import org.apache.solr.search.join.ScoreJoinQParserPlugin; +import org.apache.solr.util.RTimer; import org.apache.solr.util.RefCounted; public class JoinQParserPlugin extends QParserPlugin { @@ -138,7 +139,7 @@ public class JoinQParserPlugin extends QParserPlugin { QParser parser = QParser.getParser(v, "lucene", otherReq); fromQuery = parser.getQuery(); fromHolder = fromCore.getRegisteredSearcher(); - if (fromHolder != null) fromCoreOpenTime = fromHolder.get().getOpenTime(); + if (fromHolder != null) fromCoreOpenTime = fromHolder.get().getOpenNanoTime(); } finally { otherReq.close(); fromCore.close(); @@ -290,13 +291,13 @@ class JoinQuery extends Query { public Scorer scorer(LeafReaderContext context) throws IOException { if (filter == null) { boolean debug = rb != null && rb.isDebug(); - long start = debug ? System.currentTimeMillis() : 0; + RTimer timer = (debug ? new RTimer() : null); resultSet = getDocSet(); - long end = debug ? System.currentTimeMillis() : 0; + if (timer != null) timer.stop(); if (debug) { SimpleOrderedMap dbg = new SimpleOrderedMap<>(); - dbg.add("time", (end-start)); + dbg.add("time", (long) timer.getTime()); dbg.add("fromSetSize", fromSetSize); // the input dbg.add("toSetSize", resultSet.size()); // the output diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java index ef820d23ad6..dc057121b94 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -114,8 +114,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn private boolean debug = log.isDebugEnabled(); private final String name; - private long openTime = System.currentTimeMillis(); - private long registerTime = 0; + private final Date openTime = new Date(); + private final long openNanoTime = System.nanoTime(); + private Date registerTime; private long warmupTime = 0; private final DirectoryReader reader; private final boolean closeReader; @@ -394,7 +395,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn cache.setState(SolrCache.State.LIVE); core.getInfoRegistry().put(cache.name(), cache); } - registerTime=System.currentTimeMillis(); + registerTime = new Date(); } /** @@ -2240,13 +2241,23 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn */ public Object cacheInsert(String cacheName, Object key, Object val) { SolrCache cache = cacheMap.get(cacheName); - return cache==null ? null : cache.put(key,val); + return cache==null ? null : cache.put(key, val); } - public long getOpenTime() { + public Date getOpenTimeStamp() { return openTime; } + // public but primarily for test case usage + public long getOpenNanoTime() { + return openNanoTime; + } + + @Deprecated + public long getOpenTime() { + return openTime.getTime(); + } + @Override public Explanation explain(Query query, int doc) throws IOException { return super.explain(QueryUtils.makeQueryable(query), doc); @@ -2297,8 +2308,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn lst.add("reader", reader.toString()); lst.add("readerDir", reader.directory()); lst.add("indexVersion", reader.getVersion()); - lst.add("openedAt", new Date(openTime)); - if (registerTime!=0) lst.add("registeredAt", new Date(registerTime)); + lst.add("openedAt", openTime); + if (registerTime!=null) lst.add("registeredAt", registerTime); lst.add("warmupTime", warmupTime); return lst; } diff --git a/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java index 72a95b2e0dc..3762728d2a5 100644 --- a/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java @@ -251,7 +251,7 @@ public class ScoreJoinQParserPlugin extends QParserPlugin { fromHolder = fromCore.getRegisteredSearcher(); if (fromHolder != null) { - fromCoreOpenTime = fromHolder.get().getOpenTime(); + fromCoreOpenTime = fromHolder.get().getOpenNanoTime(); } return new OtherCoreJoinQuery(fromQuery, fromField, fromIndex, fromCoreOpenTime, scoreMode, toField); diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java index 4389e7f7d81..cc9e0a63661 100644 --- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java +++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java @@ -46,6 +46,7 @@ import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.Base64; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.StrUtils; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.common.util.Utils; import org.apache.solr.core.CoreContainer; import org.apache.solr.handler.RequestHandlerBase; @@ -86,6 +87,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt } + @SuppressForbidden(reason = "Needs currentTimeMillis to compare against time in header") @Override public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception { @@ -248,7 +250,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt } - + @SuppressForbidden(reason = "Needs currentTimeMillis to set current time in header") void setHeader(HttpRequest httpRequest) { SolrRequestInfo reqInfo = getRequestInfo(); String usr = null; diff --git a/solr/core/src/java/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java b/solr/core/src/java/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java index dc976b2545d..7785f1aa757 100644 --- a/solr/core/src/java/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java +++ b/solr/core/src/java/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java @@ -27,6 +27,7 @@ import java.util.List; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.IndexDeletionPolicyWrapper; import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrConfig; @@ -153,7 +154,7 @@ public final class HttpCacheHeaderUtil { lastMod = LastModFrom.DIRLASTMOD == lastModFrom ? IndexDeletionPolicyWrapper.getCommitTimestamp(searcher.getIndexReader().getIndexCommit()) - : searcher.getOpenTime(); + : searcher.getOpenTimeStamp().getTime(); } catch (IOException e) { // we're pretty freaking screwed if this happens throw new SolrException(ErrorCode.SERVER_ERROR, e); @@ -164,6 +165,11 @@ public final class HttpCacheHeaderUtil { return lastMod - (lastMod % 1000L); } + @SuppressForbidden(reason = "Need currentTimeMillis to send out cache control headers externally") + private static long timeNowForHeader() { + return System.currentTimeMillis(); + } + /** * Set the Cache-Control HTTP header (and Expires if needed) * based on the SolrConfig. @@ -183,8 +189,7 @@ public final class HttpCacheHeaderUtil { } Long maxAge = conf.getHttpCachingConfig().getMaxAge(); if (null != maxAge) { - resp.setDateHeader("Expires", System.currentTimeMillis() - + (maxAge * 1000L)); + resp.setDateHeader("Expires", timeNowForHeader() + (maxAge * 1000L)); } return; @@ -330,11 +335,12 @@ public final class HttpCacheHeaderUtil { // As long as no time machines get invented this is safe resp.setHeader("Expires", "Sat, 01 Jan 2000 01:00:00 GMT"); + long timeNowForHeader = timeNowForHeader(); // We signal "just modified" just in case some broken // proxy cache does not follow the above headers - resp.setDateHeader("Last-Modified", System.currentTimeMillis()); + resp.setDateHeader("Last-Modified", timeNowForHeader); // We override the ETag with something different - resp.setHeader("ETag", '"'+Long.toHexString(System.currentTimeMillis())+'"'); + resp.setHeader("ETag", '"'+Long.toHexString(timeNowForHeader)+'"'); } } diff --git a/solr/core/src/java/org/apache/solr/store/blockcache/BlockCacheLocation.java b/solr/core/src/java/org/apache/solr/store/blockcache/BlockCacheLocation.java index d2a124dda8c..c80ff5bbc09 100644 --- a/solr/core/src/java/org/apache/solr/store/blockcache/BlockCacheLocation.java +++ b/solr/core/src/java/org/apache/solr/store/blockcache/BlockCacheLocation.java @@ -19,6 +19,8 @@ package org.apache.solr.store.blockcache; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.solr.common.util.SuppressForbidden; + /** * @lucene.experimental */ @@ -26,10 +28,14 @@ public class BlockCacheLocation { private int block; private int bankId; - private long lastAccess = System.currentTimeMillis(); - private long accesses; + private long lastAccess; + private long accesses = -1; private AtomicBoolean removed = new AtomicBoolean(false); - + + public BlockCacheLocation() { + touch(); + } + public void setBlock(int block) { this.block = block; } @@ -45,12 +51,13 @@ public class BlockCacheLocation { public int getBankId() { return bankId; } - + + @SuppressForbidden(reason = "Need currentTimeMillis, only used by unused getLastAccess") public void touch() { lastAccess = System.currentTimeMillis(); accesses++; } - + public long getLastAccess() { return lastAccess; } diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java index a2bdef29f18..a57edab7bfe 100644 --- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java +++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java @@ -35,6 +35,7 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.core.SolrConfig.UpdateHandlerInfo; import org.apache.solr.core.SolrCore; import org.apache.solr.request.LocalSolrQueryRequest; @@ -142,7 +143,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState } private void deleteAll() throws IOException { - SolrCore.log.info(core.getLogId()+"REMOVING ALL DOCUMENTS FROM INDEX"); + SolrCore.log.info(core.getLogId() + "REMOVING ALL DOCUMENTS FROM INDEX"); RefCounted iw = solrCoreState.getIndexWriter(core); try { iw.get().deleteAll(); @@ -493,6 +494,15 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState return rc; } + @SuppressForbidden(reason = "Need currentTimeMillis, commit time should be used only for debugging purposes, " + + " but currently suspiciously used for replication as well") + private void setCommitData(IndexWriter iw) { + final Map commitData = new HashMap<>(); + commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, + String.valueOf(System.currentTimeMillis())); + iw.setCommitData(commitData); + } + public void prepareCommit(CommitUpdateCommand cmd) throws IOException { boolean error=true; @@ -501,10 +511,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState log.info("start "+cmd); RefCounted iw = solrCoreState.getIndexWriter(core); try { - final Map commitData = new HashMap<>(); - commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, - String.valueOf(System.currentTimeMillis())); - iw.get().setCommitData(commitData); + setCommitData(iw.get()); iw.get().prepareCommit(); } finally { iw.decref(); @@ -579,10 +586,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState // SolrCore.verbose("writer.commit() start writer=",writer); if (writer.hasUncommittedChanges()) { - final Map commitData = new HashMap<>(); - commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, - String.valueOf(System.currentTimeMillis())); - writer.setCommitData(commitData); + setCommitData(writer); writer.commit(); } else { log.info("No uncommitted changes. Skipping IW.commit."); @@ -767,9 +771,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState } // todo: refactor this shared code (or figure out why a real CommitUpdateCommand can't be used) - final Map commitData = new HashMap<>(); - commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, String.valueOf(System.currentTimeMillis())); - writer.setCommitData(commitData); + setCommitData(writer); writer.commit(); synchronized (solrCoreState.getUpdateLock()) { diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java index 7028d458578..0d0876f302f 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java @@ -58,6 +58,7 @@ import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.apache.solr.update.processor.UpdateRequestProcessor; import org.apache.solr.update.processor.UpdateRequestProcessorChain; import org.apache.solr.util.DefaultSolrThreadFactory; +import org.apache.solr.util.RTimer; import org.apache.solr.util.RefCounted; import org.apache.solr.util.plugin.PluginInfoInitialized; import org.slf4j.Logger; @@ -1571,7 +1572,7 @@ public class UpdateLog implements PluginInfoInitialized { */ protected Long seedBucketsWithHighestVersion(SolrIndexSearcher newSearcher, VersionInfo versions) { Long highestVersion = null; - long startMs = System.currentTimeMillis(); + final RTimer timer = new RTimer(); RecentUpdates recentUpdates = null; try { @@ -1596,9 +1597,8 @@ public class UpdateLog implements PluginInfoInitialized { recentUpdates.close(); } - long tookMs = (System.currentTimeMillis() - startMs); - log.info("Took {} ms to seed version buckets with highest version {}", - tookMs, String.valueOf(highestVersion)); + log.info("Took {}ms to seed version buckets with highest version {}", + timer.getTime(), String.valueOf(highestVersion)); return highestVersion; } diff --git a/solr/core/src/java/org/apache/solr/update/VersionInfo.java b/solr/core/src/java/org/apache/solr/update/VersionInfo.java index 54ced7807ac..487bfce5880 100644 --- a/solr/core/src/java/org/apache/solr/update/VersionInfo.java +++ b/solr/core/src/java/org/apache/solr/update/VersionInfo.java @@ -35,6 +35,7 @@ import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.NumericUtils; import org.apache.solr.common.SolrException; +import org.apache.solr.common.util.SuppressForbidden; import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.SolrIndexSearcher; @@ -150,15 +151,14 @@ public class VersionInfo { // Time-based lamport clock. Good for introducing some reality into clocks (to the degree // that times are somewhat synchronized in the cluster). // Good if we want to relax some constraints to scale down to where only one node may be - // up at a time. Possibly harder to detect missing messages (because versions are not contiguous. - long vclock; - long time; + // up at a time. Possibly harder to detect missing messages (because versions are not contiguous). + private long vclock; private final Object clockSync = new Object(); - + @SuppressForbidden(reason = "need currentTimeMillis just for getting realistic version stamps, does not assume monotonicity") public long getNewClock() { synchronized (clockSync) { - time = System.currentTimeMillis(); + long time = System.currentTimeMillis(); long result = time << 20; if (result <= vclock) { result = vclock + 1; diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java index 92db6c61954..1ef1a86a885 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java +++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java @@ -524,8 +524,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { if (routeKey != null) { RoutingRule rule = routingRules.get(routeKey + "!"); if (rule != null) { - // TODO: look at using nanoTime - if (rule.getExpireAt() >= System.currentTimeMillis()) { + if (! rule.isExpired()) { List ranges = rule.getRouteRanges(); if (ranges != null && !ranges.isEmpty()) { int hash = compositeIdRouter.sliceHash(id, doc, null, coll); diff --git a/solr/core/src/java/org/apache/solr/util/SimplePostTool.java b/solr/core/src/java/org/apache/solr/util/SimplePostTool.java index 2e79e071d1a..646f5918d9b 100644 --- a/solr/core/src/java/org/apache/solr/util/SimplePostTool.java +++ b/solr/core/src/java/org/apache/solr/util/SimplePostTool.java @@ -60,6 +60,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.TimeZone; +import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; import java.util.zip.GZIPInputStream; @@ -172,7 +173,7 @@ public class SimplePostTool { * This method delegates to the correct mode method. */ public void execute() { - final long startTime = System.currentTimeMillis(); + final RTimer timer = new RTimer(); if (DATA_MODE_FILES.equals(mode) && args.length > 0) { doFilesMode(); } else if(DATA_MODE_ARGS.equals(mode) && args.length > 0) { @@ -188,8 +189,7 @@ public class SimplePostTool { if (commit) commit(); if (optimize) optimize(); - final long endTime = System.currentTimeMillis(); - displayTiming(endTime - startTime); + displayTiming((long) timer.getTime()); } /** diff --git a/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java b/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java index e90eb2abf47..91407bdca44 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java +++ b/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java @@ -51,8 +51,11 @@ public class SolrLogLayout extends Layout { public static interface TG { public String getTag(); } - - long startTime = System.currentTimeMillis(); + + @SuppressForbidden(reason = "Need currentTimeMillis to compare against log event timestamp. " + + "This is inaccurate but unavoidable due to interface limitations, in any case this is just for logging.") + final long startTime = System.currentTimeMillis(); + long lastTime = startTime; Map methodAlias = new HashMap<>(); diff --git a/solr/core/src/java/org/apache/solr/util/TimeOut.java b/solr/core/src/java/org/apache/solr/util/TimeOut.java new file mode 100644 index 00000000000..30bef10b2c9 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/util/TimeOut.java @@ -0,0 +1,37 @@ +package org.apache.solr.util; + +/* + * 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. + */ + +import java.util.concurrent.TimeUnit; + +public class TimeOut { + + private final long timeoutAt; + + public TimeOut(long interval, TimeUnit unit) { + this.timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(interval, unit); + } + + public boolean hasTimedOut() { + return System.nanoTime() > timeoutAt; + } + + public long timeLeft(TimeUnit unit) { + return unit.convert(timeoutAt - System.nanoTime(), TimeUnit.NANOSECONDS); + } +} diff --git a/solr/core/src/java/org/apache/solr/util/stats/Clock.java b/solr/core/src/java/org/apache/solr/util/stats/Clock.java index 28dbf347322..0fef68cf934 100644 --- a/solr/core/src/java/org/apache/solr/util/stats/Clock.java +++ b/solr/core/src/java/org/apache/solr/util/stats/Clock.java @@ -24,6 +24,8 @@ package org.apache.solr.util.stats; import java.lang.management.ManagementFactory; import java.lang.management.ThreadMXBean; +import org.apache.solr.common.util.SuppressForbidden; + /** * An abstraction for how time passes. It is passed to {@link Timer} to track timing. */ @@ -40,6 +42,7 @@ public abstract class Clock { * * @return time in milliseconds */ + @SuppressForbidden(reason = "Need currentTimeMillis, API used by ExponentiallyDecayingSample for suspect reasons") public long getTime() { return System.currentTimeMillis(); } diff --git a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java index 87ab2f93a3c..c91f5de2a6b 100644 --- a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java +++ b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java @@ -19,6 +19,9 @@ package org.apache.solr.util.xslt; import java.io.IOException; import java.io.InputStream; +import java.util.concurrent.TimeUnit; + +import org.apache.solr.util.TimeOut; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.io.IOUtils; @@ -45,8 +48,8 @@ import org.apache.solr.core.SolrConfig; public class TransformerProvider { private String lastFilename; private Templates lastTemplates = null; - private long cacheExpires = 0; - + private TimeOut cacheExpiresTimeout; + private static final Logger log = LoggerFactory.getLogger(TransformerProvider.class.getName()); private static final XMLErrorLogger xmllog = new XMLErrorLogger(log); @@ -69,7 +72,8 @@ public class TransformerProvider { public synchronized Transformer getTransformer(SolrConfig solrConfig, String filename,int cacheLifetimeSeconds) throws IOException { // For now, the Templates are blindly reloaded once cacheExpires is over. // It'd be better to check the file modification time to reload only if needed. - if(lastTemplates!=null && filename.equals(lastFilename) && System.currentTimeMillis() < cacheExpires) { + if(lastTemplates!=null && filename.equals(lastFilename) && + cacheExpiresTimeout != null && ! cacheExpiresTimeout.hasTimedOut()) { if(log.isDebugEnabled()) { log.debug("Using cached Templates:" + filename); } @@ -117,8 +121,8 @@ public class TransformerProvider { lastFilename = filename; lastTemplates = result; - cacheExpires = System.currentTimeMillis() + (cacheLifetimeSeconds * 1000); - + cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS); + return result; } } diff --git a/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java b/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java index e5efa6fae28..68d1ace223e 100644 --- a/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java +++ b/solr/core/src/test/org/apache/solr/DistributedIntervalFacetingTest.java @@ -192,7 +192,7 @@ public class DistributedIntervalFacetingTest extends } private String getFieldWithKey(String field) { - return "{!key='_some_key_for_" + field + "_" + System.currentTimeMillis() + "'}" + field; + return "{!key='_some_key_for_" + field + "_" + random().nextInt() + "'}" + field; } /** diff --git a/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java b/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java index f077f02e7fa..057ba5005b2 100644 --- a/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java +++ b/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java @@ -48,7 +48,6 @@ public class SolrInfoMBeanTest extends SolrTestCaseJ4 public void testCallMBeanInfo() throws Exception { List classes = new ArrayList<>(); classes.addAll(getClassesForPackage(StandardRequestHandler.class.getPackage().getName())); - classes.addAll(getClassesForPackage(SearchHandler.class.getPackage().getName())); classes.addAll(getClassesForPackage(SearchComponent.class.getPackage().getName())); classes.addAll(getClassesForPackage(LukeRequestHandler.class.getPackage().getName())); classes.addAll(getClassesForPackage(DefaultSolrHighlighter.class.getPackage().getName())); diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java index c89c8bec3e6..d2d2a5f2ae5 100644 --- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java @@ -52,6 +52,7 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.NamedList; import org.apache.solr.update.DirectUpdateHandler2; import org.apache.solr.util.DefaultSolrThreadFactory; +import org.apache.solr.util.TimeOut; import org.junit.BeforeClass; import org.junit.Test; @@ -318,11 +319,11 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase { long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound(); ModifiableSolrParams params = new ModifiableSolrParams(); params.set("commitWithin", 10); - add(cloudClient, params , getDoc("id", 300)); - - long timeout = System.currentTimeMillis() + 45000; + add(cloudClient, params, getDoc("id", 300)); + + TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS); while (cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound() != before + 1) { - if (timeout <= System.currentTimeMillis()) { + if (timeout.hasTimedOut()) { fail("commitWithin did not work"); } Thread.sleep(100); @@ -374,10 +375,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase { } catch (Exception e) { } - - long timeout = System.currentTimeMillis() + 15000; + + TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS); while (cloudClient.getZkStateReader().getZkClient().exists("/collections/the_core_collection", true)) { - if (timeout <= System.currentTimeMillis()) { + if (timeout.hasTimedOut()) { fail(cloudClient.getZkStateReader().getZkClient().getChildren("/collections", null, true).toString() + " Collection zk node still exists"); } Thread.sleep(100); diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java index ddb2df48335..b462dab12ef 100644 --- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java @@ -80,6 +80,7 @@ import org.apache.solr.core.CoreContainer; import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrInfoMBean.Category; import org.apache.solr.servlet.SolrDispatchFilter; +import org.apache.solr.util.TimeOut; import org.junit.Test; import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES; @@ -280,10 +281,10 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa request.setPath("/admin/collections"); makeRequest(baseUrl, request); - - long timeout = System.currentTimeMillis() + 10000; + + TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS); while (cloudClient.getZkStateReader().getClusterState().hasCollection("halfdeletedcollection2")) { - if (System.currentTimeMillis() > timeout) { + if (timeout.hasTimedOut()) { throw new AssertionError("Timeout waiting to see removed collection leave clusterstate"); } @@ -934,12 +935,12 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa } private boolean waitForReloads(String collectionName, Map urlToTimeBefore) throws SolrServerException, IOException { - - - long timeoutAt = System.currentTimeMillis() + 45000; + + + TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS); boolean allTimesAreCorrect = false; - while (System.currentTimeMillis() < timeoutAt) { + while (! timeout.hasTimedOut()) { Map urlToTimeAfter = new HashMap<>(); collectStartTimes(collectionName, urlToTimeAfter); @@ -1164,10 +1165,10 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa addReplica.setProperties(props); } client.request(addReplica); - long timeout = System.currentTimeMillis() + 3000; + TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS); Replica newReplica = null; - for (; System.currentTimeMillis() < timeout; ) { + for (; ! timeout.hasTimedOut(); ) { Slice slice = client.getZkStateReader().getClusterState().getSlice(collectionName, shard); newReplica = slice.getReplica(newReplicaName); } @@ -1218,9 +1219,9 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa request.setPath("/admin/collections"); client.request(request); - long timeOut = System.currentTimeMillis() + 3000; + TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS); boolean changed = false; - while(System.currentTimeMillis() replicasToCheck, int maxWaitSecs) throws Exception { - long startMs = System.currentTimeMillis(); + final RTimer timer = new RTimer(); ZkStateReader zkr = cloudClient.getZkStateReader(); zkr.updateClusterState(); // force the state to be fresh @@ -565,7 +566,6 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase { fail("Didn't see replicas "+ replicasToCheck + " come up within " + maxWaitMs + " ms! ClusterState: " + printClusterStateInfo(testCollectionName)); - long diffMs = (System.currentTimeMillis() - startMs); - log.info("Took " + diffMs + " ms to see replicas ["+replicasToCheck+"] become active."); + log.info("Took {} ms to see replicas [{}] become active.", timer.getTime(), replicasToCheck); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java index b2e020982c4..322bb7d7b9c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java @@ -23,6 +23,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.Replica; +import org.apache.solr.util.RTimer; import org.junit.Test; import java.io.File; @@ -157,13 +158,12 @@ public class LeaderInitiatedRecoveryOnCommitTest extends BasicDistributedZkTest protected void sendCommitWithRetry(Replica replica) throws Exception { String replicaCoreUrl = replica.getCoreUrl(); log.info("Sending commit request to: "+replicaCoreUrl); - long startMs = System.currentTimeMillis(); + final RTimer timer = new RTimer(); try (HttpSolrClient client = new HttpSolrClient(replicaCoreUrl)) { try { client.commit(); - long tookMs = System.currentTimeMillis() - startMs; - log.info("Sent commit request to "+replicaCoreUrl+" OK, took: "+tookMs); + log.info("Sent commit request to {} OK, took {}ms", replicaCoreUrl, timer.getTime()); } catch (Exception exc) { Throwable rootCause = SolrException.getRootCause(exc); if (rootCause instanceof NoHttpResponseException) { diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java index 753d13bcce6..553728d5e32 100644 --- a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java @@ -31,6 +31,7 @@ import org.apache.solr.common.cloud.RoutingRule; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.Utils; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.KeeperException; import org.junit.Test; @@ -38,6 +39,7 @@ import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES; import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE; @@ -64,7 +66,8 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest { private boolean waitForRuleToExpire(String splitKey, long finishTime) throws KeeperException, InterruptedException, SolrServerException, IOException { ClusterState state;Slice slice; boolean ruleRemoved = false; - while (System.currentTimeMillis() - finishTime < 60000) { + long expiryTime = finishTime + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS); + while (System.nanoTime() < expiryTime) { getCommonCloudSolrClient().getZkStateReader().updateClusterState(); state = getCommonCloudSolrClient().getZkStateReader().getClusterState(); slice = state.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2); @@ -74,7 +77,7 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest { break; } SolrInputDocument doc = new SolrInputDocument(); - doc.addField("id", splitKey + System.currentTimeMillis()); + doc.addField("id", splitKey + random().nextInt()); cloudClient.add(doc); Thread.sleep(1000); } @@ -160,7 +163,7 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest { assertEquals("DocCount on target collection does not match", 0, collectionClient.query(solrQuery).getResults().getNumFound()); invokeMigrateApi(AbstractDistribZkTestBase.DEFAULT_COLLECTION, splitKey + "/" + BIT_SEP + "!", targetCollection); - long finishTime = System.currentTimeMillis(); + long finishTime = System.nanoTime(); indexer.join(); splitKeyCount[0] += indexer.getSplitKeyCount(); @@ -207,8 +210,8 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest { @Override public void run() { - long start = System.currentTimeMillis(); - for (int id = 26*3; id < 500 && System.currentTimeMillis() - start <= seconds*1000; id++) { + TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS); + for (int id = 26*3; id < 500 && ! timeout.hasTimedOut(); id++) { String shardKey = "" + (char) ('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution SolrInputDocument doc = new SolrInputDocument(); doc.addField("id", shardKey + (bitSep != -1 ? "/" + bitSep : "") + "!" + id); diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java index 812f25844a2..cb56775e397 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionProcessorTest.java @@ -36,6 +36,7 @@ import org.apache.solr.handler.component.ShardHandler; import org.apache.solr.handler.component.ShardHandlerFactory; import org.apache.solr.handler.component.ShardRequest; import org.apache.solr.handler.component.ShardResponse; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.CreateMode; import org.easymock.Capture; import org.easymock.EasyMock; @@ -59,6 +60,7 @@ import java.util.Map.Entry; import java.util.Queue; import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; import static org.easymock.EasyMock.anyBoolean; import static org.easymock.EasyMock.anyObject; @@ -562,10 +564,10 @@ public class OverseerCollectionProcessorTest extends SolrTestCaseJ4 { } protected void waitForEmptyQueue(long maxWait) throws Exception { - long start = System.currentTimeMillis(); + final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS); while (queue.peek() != null) { - if ((System.currentTimeMillis() - start) > maxWait) fail(" Queue not empty within " - + maxWait + " ms" + System.currentTimeMillis()); + if (timeout.hasTimedOut()) + fail("Queue not empty within " + maxWait + " ms"); Thread.sleep(100); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java index 5e76de6e2c7..34d62fbd411 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java @@ -31,6 +31,7 @@ import org.apache.solr.common.params.CollectionParams.CollectionAction; import org.apache.solr.common.params.MapSolrParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.Utils; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.data.Stat; import org.junit.Test; @@ -40,6 +41,7 @@ import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.solr.cloud.OverseerCollectionProcessor.getLeaderNode; import static org.apache.solr.cloud.OverseerCollectionProcessor.getSortedOverseerNodeNames; @@ -83,9 +85,9 @@ public class OverseerRolesTest extends AbstractFullDistribZkTestBase{ String s = (String) m.get("id"); String leader = LeaderElector.getNodeName(s); Overseer.getInQueue(zk).offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower()))); - long timeout = System.currentTimeMillis()+10000; + final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS); String newLeader=null; - for(;System.currentTimeMillis() < timeout;){ + for(;! timeout.hasTimedOut();){ newLeader = OverseerCollectionProcessor.getLeaderNode(zk); if(newLeader!=null && !newLeader.equals(leader)) break; Thread.sleep(100); @@ -118,10 +120,10 @@ public class OverseerRolesTest extends AbstractFullDistribZkTestBase{ log.info("overseerDesignate {}",overseerDesignate); setOverseerRole(client, CollectionAction.ADDROLE,overseerDesignate); - long timeout = System.currentTimeMillis()+15000; + TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS); boolean leaderchanged = false; - for(;System.currentTimeMillis() < timeout;){ + for(;!timeout.hasTimedOut();){ if(overseerDesignate.equals(OverseerCollectionProcessor.getLeaderNode(client.getZkStateReader().getZkClient()))){ log.info("overseer designate is the new overseer"); leaderchanged =true; @@ -177,9 +179,9 @@ public class OverseerRolesTest extends AbstractFullDistribZkTestBase{ OverseerCollectionProcessor.getSortedElectionNodes(client.getZkStateReader().getZkClient(), OverseerElectionContext.PATH + LeaderElector.ELECTION_NODE)); ChaosMonkey.stop(leaderJetty); - timeout = System.currentTimeMillis() + 10000; + timeout = new TimeOut(10, TimeUnit.SECONDS); leaderchanged = false; - for (; System.currentTimeMillis() < timeout; ) { + for (; !timeout.hasTimedOut(); ) { currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient()); if (anotherOverseer.equals(currentOverseer)) { leaderchanged = true; diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java index 786b9eaacc1..e741e7fe312 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java @@ -24,7 +24,6 @@ import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.core.NodeConfig; import org.apache.solr.servlet.SolrDispatchFilter; import org.junit.After; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.RuleChain; @@ -74,7 +73,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 { System.setProperty("zkClientTimeout", "8000"); zkDir = tmpDir.getAbsolutePath() + File.separator - + "zookeeper" + System.currentTimeMillis() + "/server1/data"; + + "zookeeper" + System.nanoTime() + "/server1/data"; zkServer = new ZkTestServer(zkDir); zkServer.run(); System.setProperty("zkHost", zkServer.getZkAddress()); diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java index 0e57058e9ab..9cbc95cf4f9 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java @@ -26,6 +26,7 @@ import org.apache.solr.common.params.CollectionParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.Utils; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.KeeperException; import org.junit.Test; @@ -34,6 +35,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase { @@ -114,9 +116,9 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase { // 2> All the replicas we _think_ are leaders are in the 0th position in the leader election queue. // 3> The node that ZooKeeper thinks is the leader is the one we think should be the leader. void checkConsistency() throws InterruptedException, KeeperException { - long start = System.currentTimeMillis(); + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS); - while ((System.currentTimeMillis() - start) < timeoutMs) { + while (! timeout.hasTimedOut()) { if (checkAppearOnce() && checkElectionZero() && checkZkLeadersAgree()) { @@ -234,7 +236,6 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase { byte[] getZkData(CloudSolrClient client, String path) { org.apache.zookeeper.data.Stat stat = new org.apache.zookeeper.data.Stat(); - long start = System.currentTimeMillis(); try { byte[] data = client.getZkStateReader().getZkClient().getData(path, null, stat, true); if (data != null) { @@ -299,8 +300,8 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase { boolean waitForAllPreferreds() throws KeeperException, InterruptedException { boolean goAgain = true; - long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < timeoutMs) { + TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS); + while (! timeout.hasTimedOut()) { goAgain = false; cloudClient.getZkStateReader().updateClusterState(); Map slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap(); diff --git a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java index 75ed794febd..635cd1c4543 100644 --- a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java @@ -33,6 +33,7 @@ import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.update.DirectUpdateHandler2; import org.apache.solr.util.DefaultSolrThreadFactory; +import org.apache.solr.util.TimeOut; import org.junit.Test; import java.io.File; @@ -108,9 +109,9 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest { // there should be only one shard int slices = getCommonCloudSolrClient().getZkStateReader().getClusterState().getSlices(collection).size(); - long timeoutAt = System.currentTimeMillis() + 45000; + final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS); while (slices != 1) { - if (System.currentTimeMillis() > timeoutAt) { + if (timeout.hasTimedOut()) { printLayout(); fail("Expected to find only one slice in " + collection); } @@ -127,9 +128,9 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest { //printLayout(); // the collection should be gone - long timeoutAt = System.currentTimeMillis() + 30000; + final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); while (getCommonCloudSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) { - if (System.currentTimeMillis() > timeoutAt) { + if (timeout.hasTimedOut()) { printLayout(); fail("Still found collection"); } @@ -145,7 +146,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest { private void testCoreUnloadAndLeaders() throws Exception { File tmpDir = createTempDir().toFile(); - String core1DataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection1" + "_1n"; + String core1DataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection1" + "_1n"; // create a new collection collection SolrClient client = clients.get(0); @@ -175,7 +176,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest { Create createCmd = new Create(); createCmd.setCoreName("unloadcollection2"); createCmd.setCollection("unloadcollection"); - String core2dataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection1" + "_2n"; + String core2dataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection1" + "_2n"; createCmd.setDataDir(getDataDir(core2dataDir)); adminClient.request(createCmd); } @@ -212,7 +213,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest { Create createCmd = new Create(); createCmd.setCoreName("unloadcollection3"); createCmd.setCollection("unloadcollection"); - String core3dataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection" + "_3n"; + String core3dataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection" + "_3n"; createCmd.setDataDir(getDataDir(core3dataDir)); adminClient.request(createCmd); } @@ -282,7 +283,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest { Create createCmd = new Create(); createCmd.setCoreName("unloadcollection4"); createCmd.setCollection("unloadcollection"); - String core4dataDir = tmpDir.getAbsolutePath() + File.separator + System.currentTimeMillis() + "unloadcollection" + "_4n"; + String core4dataDir = tmpDir.getAbsolutePath() + File.separator + System.nanoTime() + "unloadcollection" + "_4n"; createCmd.setDataDir(getDataDir(core4dataDir)); adminClient.request(createCmd); } diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java index ec3b5048fd4..bcd1c298f91 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java @@ -186,7 +186,7 @@ public class ZkCLITest extends SolrTestCaseJ4 { "list"}; ZkCLI.main(args); } - + @Test public void testUpConfigLinkConfigClearZk() throws Exception { File tmpDir = createTempDir().toFile(); @@ -218,7 +218,7 @@ public class ZkCLITest extends SolrTestCaseJ4 { // test down config File confDir = new File(tmpDir, - "solrtest-confdropspot-" + this.getClass().getName() + "-" + System.currentTimeMillis()); + "solrtest-confdropspot-" + this.getClass().getName() + "-" + System.nanoTime()); assertFalse(confDir.exists()); args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd", @@ -268,7 +268,7 @@ public class ZkCLITest extends SolrTestCaseJ4 { this.zkClient.create(getNode, data, CreateMode.PERSISTENT, true); File file = new File(tmpDir, - "solrtest-getfile-" + this.getClass().getName() + "-" + System.currentTimeMillis()); + "solrtest-getfile-" + this.getClass().getName() + "-" + System.nanoTime()); String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd", "getfile", getNode, file.getAbsolutePath()}; ZkCLI.main(args); diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java index b9617a60d3b..8b647473e00 100644 --- a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java @@ -37,6 +37,7 @@ import org.apache.solr.common.params.CollectionParams.CollectionAction; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.util.BadHdfsThreadsFilter; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -49,6 +50,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.TimeUnit; @Slow @ThreadLeakFilters(defaultFilters = true, filters = { @@ -202,10 +204,10 @@ public class StressHdfsTest extends BasicDistributedZkTest { QueryRequest request = new QueryRequest(params); request.setPath("/admin/collections"); cloudClient.request(request); - - long timeout = System.currentTimeMillis() + 10000; + + final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS); while (cloudClient.getZkStateReader().getClusterState().hasCollection(DELETE_DATA_DIR_COLLECTION)) { - if (System.currentTimeMillis() > timeout) { + if (timeout.hasTimedOut()) { throw new AssertionError("Timeout waiting to see removed collection leave clusterstate"); } diff --git a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java index 24edf855270..cc0932f8b1e 100644 --- a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java +++ b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java @@ -27,6 +27,7 @@ import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.client.solrj.response.UpdateResponse; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.util.TimeOut; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; @@ -40,6 +41,7 @@ import java.util.Locale; import java.util.Map; import java.util.Random; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -309,8 +311,6 @@ public class OpenCloseCoreStressTest extends SolrTestCaseJ4 { } class Indexer { - static volatile long stopTime; - static AtomicInteger idUnique = new AtomicInteger(0); static AtomicInteger errors = new AtomicInteger(0); @@ -322,13 +322,15 @@ class Indexer { static AtomicInteger updateCounts = new AtomicInteger(0); static volatile int lastCount; - static volatile long nextTime; + + static volatile TimeOut stopTimeout; + private static volatile TimeOut nextTimeout; ArrayList _threads = new ArrayList<>(); public Indexer(OpenCloseCoreStressTest OCCST, String url, List clients, int numThreads, int secondsToRun, Random random) { - stopTime = System.currentTimeMillis() + (secondsToRun * 1000); - nextTime = System.currentTimeMillis() + 60000; + stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS); + nextTimeout = new TimeOut(60, TimeUnit.SECONDS); docsThisCycle.set(0); qTimesAccum.set(0); updateCounts.set(0); @@ -358,11 +360,11 @@ class Indexer { } synchronized static void progress(int myId, String core) { - if (nextTime - System.currentTimeMillis() <= 0) { + if (nextTimeout.hasTimedOut()) { SolrTestCaseJ4.log.info(String.format(Locale.ROOT, " s indexed: [run %,8d] [cycle %,8d] [last minute %,8d] Last core updated: %s. Seconds left in cycle %,4d", - myId, docsThisCycle.get(), myId - lastCount, core, stopTime - (System.currentTimeMillis() / 1000))); + myId, docsThisCycle.get(), myId - lastCount, core, stopTimeout.timeLeft(TimeUnit.SECONDS))); lastCount = myId; - nextTime += (System.currentTimeMillis() / 1000) * 60; + nextTimeout = new TimeOut(60, TimeUnit.SECONDS); } } @@ -385,7 +387,7 @@ class OneIndexer extends Thread { public void run() { SolrTestCaseJ4.log.info(String.format(Locale.ROOT, "Starting indexing thread: " + getId())); - while (Indexer.stopTime > System.currentTimeMillis()) { + while (! Indexer.stopTimeout.hasTimedOut()) { int myId = Indexer.idUnique.incrementAndGet(); Indexer.docsThisCycle.incrementAndGet(); String core = OCCST.getRandomCore(random); diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java index e5aa76f4a86..de1f74a7e1a 100644 --- a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java +++ b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java @@ -637,7 +637,7 @@ public class TestCoreContainer extends SolrTestCaseJ4 { private long getCoreStartTime(final CoreContainer cc, final String name) { try (SolrCore tmp = cc.getCore(name)) { - return tmp.getStartTime(); + return tmp.getStartTimeStamp().getTime(); } } } diff --git a/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java b/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java index d9cd5d0ca28..5c9fa38f944 100644 --- a/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java +++ b/solr/core/src/test/org/apache/solr/handler/TestBlobHandler.java @@ -37,6 +37,7 @@ import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.util.StrUtils; import org.apache.solr.common.util.Utils; import org.apache.solr.core.ConfigOverlay; +import org.apache.solr.util.RTimer; import org.apache.solr.util.SimplePostTool; import org.junit.Test; import org.noggit.JSONParser; @@ -120,7 +121,7 @@ public class TestBlobHandler extends AbstractFullDistribZkTestBase { String url; Map map = null; List l; - long start = System.currentTimeMillis(); + final RTimer timer = new RTimer(); int i = 0; for (; i < 150; i++) {//15 secs url = baseUrl + "/.system/blob/" + blobName; @@ -137,7 +138,7 @@ public class TestBlobHandler extends AbstractFullDistribZkTestBase { return; } fail(StrUtils.formatString("Could not successfully add blob after {0} attempts. Expecting {1} items. time elapsed {2} output for url is {3}", - i, count, System.currentTimeMillis() - start, getAsString(map))); + i, count, timer.getTime(), getAsString(map))); } public static String getAsString(Map map) { diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java index 6d8b1049b0d..29aba01f4e7 100644 --- a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java @@ -195,7 +195,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 { @Test public void testDeleteInstanceDir() throws Exception { File solrHomeDirectory = new File(initCoreDataDir, getClass().getName() + "-corex-" - + System.currentTimeMillis()); + + System.nanoTime()); solrHomeDirectory.mkdirs(); copySolrHomeToTemp(solrHomeDirectory, "corex", true); File corex = new File(solrHomeDirectory, "corex"); diff --git a/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java b/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java index 83b91491127..3e94ed6f68a 100644 --- a/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java +++ b/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java @@ -31,6 +31,7 @@ import org.apache.solr.response.BinaryQueryResponseWriter; import org.apache.solr.response.QueryResponseWriter; import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.util.AbstractSolrTestCase; +import org.apache.solr.util.RTimer; import org.junit.BeforeClass; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -109,7 +110,7 @@ public class TestWriterPerf extends AbstractSolrTestCase { ByteArrayOutputStream out=null; System.gc(); - long start = System.currentTimeMillis(); + RTimer timer = new RTimer(); for (int i=0; i registeredSearcherH = newSearcher.getCore().getSearcher(); diff --git a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java index 797876be345..287896644d2 100644 --- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java @@ -371,13 +371,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { @Override public void postSoftCommit() { - postSoftCommitAt.set(System.currentTimeMillis()); + postSoftCommitAt.set(System.nanoTime()); } @Override public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) { newSearcherCount.incrementAndGet(); - newSearcherOpenedAt.set(newSearcher.getOpenTime()); + newSearcherOpenedAt.set(newSearcher.getOpenNanoTime()); } @Override diff --git a/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java b/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java index d1b00ce37a4..9f1f788518a 100644 --- a/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java +++ b/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import org.apache.lucene.util.Constants; import org.apache.lucene.util.LuceneTestCase.Slow; @@ -101,7 +102,7 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { hardTracker.setDocsUpperBound(-1); // Add a single document - long add529 = System.currentTimeMillis(); + long add529 = System.nanoTime(); assertU(adoc("id", "529", "subject", "the doc we care about in this test")); monitor.assertSaneOffers(); @@ -123,22 +124,25 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { Long hard529 = monitor.hard.poll(hardCommitWaitMillis * 5, MILLISECONDS); assertNotNull("hard529 wasn't fast enough", hard529); monitor.assertSaneOffers(); - - assertTrue("soft529 occured too fast: " + - add529 + " + " + softCommitWaitMillis + " !<= " + soft529, - add529 + softCommitWaitMillis <= soft529); - assertTrue("hard529 occured too fast: " + - add529 + " + " + hardCommitWaitMillis + " !<= " + hard529, - add529 + hardCommitWaitMillis <= hard529); + + final long soft529Ms = TimeUnit.MILLISECONDS.convert(soft529 - add529, TimeUnit.NANOSECONDS); + assertTrue("soft529 occured too fast, in " + + soft529Ms + "ms, less than soft commit interval " + softCommitWaitMillis, + soft529Ms >= softCommitWaitMillis); + final long hard529Ms = TimeUnit.MILLISECONDS.convert(hard529 - add529, TimeUnit.NANOSECONDS); + assertTrue("hard529 occured too fast, in " + + hard529Ms + "ms, less than hard commit interval " + hardCommitWaitMillis, + hard529Ms >= hardCommitWaitMillis); // however slow the machine was to do the soft commit compared to expected, // assume newSearcher had some magnitude of that much overhead as well - long slowTestFudge = Math.max(300, 12 * (soft529 - add529 - softCommitWaitMillis)); - assertTrue("searcher529 wasn't soon enough after soft529: " + - searcher529 + " !< " + soft529 + " + " + slowTestFudge + " (fudge)", - searcher529 < soft529 + slowTestFudge ); + long slowTestFudge = Math.max(300, 12 * (soft529Ms - softCommitWaitMillis)); + final long softCommitToSearcherOpenMs = TimeUnit.MILLISECONDS.convert(searcher529 - soft529, TimeUnit.NANOSECONDS); + assertTrue("searcher529 wasn't soon enough after soft529: Took " + + softCommitToSearcherOpenMs + "ms, >= acceptable " + slowTestFudge + "ms (fudge)", + softCommitToSearcherOpenMs < slowTestFudge); - assertTrue("hard529 was before searcher529: " + + assertTrue("hard529 was before searcher529: " + searcher529 + " !<= " + hard529, searcher529 <= hard529); @@ -147,7 +151,7 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { // there may have been (or will be) a second hard commit for 530 Long hard530 = monitor.hard.poll(hardCommitWaitMillis, MILLISECONDS); assertEquals("Tracker reports too many hard commits", - (null == hard530 ? 1 : 2), + (null == hard530 ? 1 : 2), hardTracker.getCommitCount()); // there may have been a second soft commit for 530, @@ -229,7 +233,7 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { monitor.clear(); // Delete the document - long del529 = System.currentTimeMillis(); + long del529 = System.nanoTime(); assertU( delI("529") ); monitor.assertSaneOffers(); @@ -251,22 +255,25 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { hard529 = monitor.hard.poll(hardCommitWaitMillis * 3, MILLISECONDS); assertNotNull("hard529 wasn't fast enough", hard529); monitor.assertSaneOffers(); - - assertTrue("soft529 occured too fast: " + - del529 + " + " + softCommitWaitMillis + " !<= " + soft529, - del529 + softCommitWaitMillis <= soft529); - assertTrue("hard529 occured too fast: " + - del529 + " + " + hardCommitWaitMillis + " !<= " + hard529, - del529 + hardCommitWaitMillis <= hard529); + + final long soft529Ms = TimeUnit.MILLISECONDS.convert(soft529 - del529, TimeUnit.NANOSECONDS); + assertTrue("soft529 occured too fast, in " + soft529Ms + + "ms, less than soft commit interval " + softCommitWaitMillis, + soft529Ms >= softCommitWaitMillis); + final long hard529Ms = TimeUnit.MILLISECONDS.convert(hard529 - del529, TimeUnit.NANOSECONDS); + assertTrue("hard529 occured too fast, in " + + hard529Ms + "ms, less than hard commit interval " + hardCommitWaitMillis, + hard529Ms >= hardCommitWaitMillis); // however slow the machine was to do the soft commit compared to expected, - // assume newSearcher had some magnitude of that much overhead as well - long slowTestFudge = Math.max(150, 3 * (soft529 - del529 - softCommitWaitMillis)); - assertTrue("searcher529 wasn't soon enough after soft529: " + - searcher529 + " !< " + soft529 + " + " + slowTestFudge + " (fudge)", - searcher529 < soft529 + slowTestFudge ); + // assume newSearcher had some magnitude of that much overhead as well + long slowTestFudge = Math.max(300, 12 * (soft529Ms - softCommitWaitMillis)); + final long softCommitToSearcherOpenMs = TimeUnit.MILLISECONDS.convert(searcher529 - soft529, TimeUnit.NANOSECONDS); + assertTrue("searcher529 wasn't soon enough after soft529: Took " + + softCommitToSearcherOpenMs + "ms, >= acceptable " + slowTestFudge + "ms (fudge)", + softCommitToSearcherOpenMs < slowTestFudge); - assertTrue("hard529 was before searcher529: " + + assertTrue("hard529 was before searcher529: " + searcher529 + " !<= " + hard529, searcher529 <= hard529); @@ -298,17 +305,17 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { hardTracker.setDocsUpperBound(-1); // try to add 5 docs really fast - long fast5start = System.currentTimeMillis(); + long fast5start = System.nanoTime(); for( int i=0;i<5; i++ ) { assertU(adoc("id", ""+500 + i, "subject", "five fast docs")); } - long fast5end = System.currentTimeMillis() - 200; // minus a tad of slop - long fast5time = 1 + fast5end - fast5start; + long fast5end = System.nanoTime() - TimeUnit.NANOSECONDS.convert(200, TimeUnit.MILLISECONDS); // minus a tad of slop + long fast5time = 1 + TimeUnit.MILLISECONDS.convert(fast5end - fast5start, TimeUnit.NANOSECONDS); // total time for all 5 adds determines the number of soft to expect - long expectedSoft = (long)Math.ceil(fast5time / softCommitWaitMillis); - long expectedHard = (long)Math.ceil(fast5time / hardCommitWaitMillis); - + long expectedSoft = (long)Math.ceil((double) fast5time / softCommitWaitMillis); + long expectedHard = (long)Math.ceil((double) fast5time / hardCommitWaitMillis); + // note: counting from 1 for multiplication for (int i = 1; i <= expectedSoft; i++) { // Wait for the soft commit with some fudge @@ -318,10 +325,10 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { // have to assume none of the docs were added until // very end of the add window - assertTrue(i + ": soft occured too fast: " + - fast5end + " + (" + softCommitWaitMillis + " * " + i + - ") !<= " + soft, - fast5end + (softCommitWaitMillis * i) <= soft); + long softMs = TimeUnit.MILLISECONDS.convert(soft - fast5end, TimeUnit.NANOSECONDS); + assertTrue(i + ": soft occured too fast: " + + softMs + " < (" + softCommitWaitMillis + " * " + i + ")", + softMs >= (softCommitWaitMillis * i)); } // note: counting from 1 for multiplication @@ -334,10 +341,10 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase { // have to assume none of the docs were added until // very end of the add window - assertTrue(i + ": soft occured too fast: " + - fast5end + " + (" + hardCommitWaitMillis + " * " + i + - ") !<= " + hard, - fast5end + (hardCommitWaitMillis * i) <= hard); + long hardMs = TimeUnit.MILLISECONDS.convert(hard - fast5end, TimeUnit.NANOSECONDS); + assertTrue(i + ": hard occured too fast: " + + hardMs + " < (" + hardCommitWaitMillis + " * " + i + ")", + hardMs >= (hardCommitWaitMillis * i)); } } @@ -361,19 +368,19 @@ class MockEventListener implements SolrEventListener { @Override public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) { - Long now = System.currentTimeMillis(); + Long now = System.nanoTime(); if (!searcher.offer(now)) fail.append(", newSearcher @ " + now); } @Override public void postCommit() { - Long now = System.currentTimeMillis(); + Long now = System.nanoTime(); if (!hard.offer(now)) fail.append(", hardCommit @ " + now); } @Override public void postSoftCommit() { - Long now = System.currentTimeMillis(); + Long now = System.nanoTime(); if (!soft.offer(now)) fail.append(", softCommit @ " + now); } diff --git a/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java b/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java index 02debc57d21..a3a41bfd843 100644 --- a/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java +++ b/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java @@ -21,6 +21,7 @@ import org.apache.solr.common.SolrInputDocument; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.util.AbstractSolrTestCase; import org.apache.solr.common.util.StrUtils; +import org.apache.solr.util.RTimer; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -100,7 +101,7 @@ public class TestIndexingPerformance extends AbstractSolrTestCase { }; ***/ - long start = System.currentTimeMillis(); + final RTimer timer = new RTimer(); AddUpdateCommand add = new AddUpdateCommand(req); add.overwrite = overwrite; @@ -116,9 +117,9 @@ public class TestIndexingPerformance extends AbstractSolrTestCase { } updateHandler.addDoc(add); } - long end = System.currentTimeMillis(); log.info("doc="+ Arrays.toString(fields)); - log.info("iter="+iter +" time=" + (end-start) + " throughput=" + ((long)iter*1000)/(end-start)); + double elapsed = timer.getTime(); + log.info("iter="+iter +" time=" + elapsed + " throughput=" + ((long)iter*1000)/elapsed); //discard all the changes updateHandler.rollback(new RollbackUpdateCommand(req)); diff --git a/solr/core/src/test/org/apache/solr/util/BitSetPerf.java b/solr/core/src/test/org/apache/solr/util/BitSetPerf.java index 76e029f92d9..76d1735a2ee 100644 --- a/solr/core/src/test/org/apache/solr/util/BitSetPerf.java +++ b/solr/core/src/test/org/apache/solr/util/BitSetPerf.java @@ -77,7 +77,7 @@ public class BitSetPerf { int ret=0; - long start = System.currentTimeMillis(); + final RTimer timer = new RTimer(); if ("union".equals(test)) { for (int it=0; it slices = clusterState.getActiveSlices(this.collection); - long time = System.currentTimeMillis(); List shuffler = new ArrayList(); for(Slice slice : slices) { Collection replicas = slice.getReplicas(); @@ -265,7 +264,7 @@ public class ParallelStream extends CloudSolrStream implements Expressible { throw new IOException("Number of workers exceeds nodes in the worker collection"); } - Collections.shuffle(shuffler, new Random(time)); + Collections.shuffle(shuffler, new Random()); for(int w=0; w 0 ? new ConcurrentLRUCache<>(cacheSz,cacheSz-cacheSz/10,cacheSz,cacheSz/10,false,true,null) : null; // the cache in the first version of the patch was 10000,9000,10000,1000,false,true,null final JavaBinCodec.StringCache stringCache = underlyingCache==null ? null : new JavaBinCodec.StringCache(underlyingCache); if (nThreads <= 0) { @@ -586,10 +587,9 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 { } }); } - long end = System.currentTimeMillis(); long n = iter * Math.max(1,nThreads); - System.out.println("ret=" + ret + " THROUGHPUT=" + (n*1000 / (end-start))); + System.out.println("ret=" + ret + " THROUGHPUT=" + (n*1000 / timer.getTime())); if (underlyingCache != null) System.out.println("cache: hits=" + underlyingCache.getStats().getCumulativeHits() + " lookups=" + underlyingCache.getStats().getCumulativeLookups() + " size=" + underlyingCache.getStats().getCurrentSize()); } diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java index 89fdb79a8d6..bea3b1686b2 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java @@ -73,6 +73,8 @@ import org.apache.solr.core.CoreContainer; import org.apache.solr.core.SolrCore; import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.update.DirectUpdateHandler2; +import org.apache.solr.util.RTimer; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.CreateMode; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -1680,10 +1682,10 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes List numShardsNumReplicaList, List nodesAllowedToRunShards) throws Exception { // check for an expectedSlices new collection - we poll the state - long timeoutAt = System.currentTimeMillis() + 120000; + final TimeOut timeout = new TimeOut(120, TimeUnit.SECONDS); boolean success = false; String checkResult = "Didnt get to perform a single check"; - while (System.currentTimeMillis() < timeoutAt) { + while (! timeout.hasTimedOut()) { checkResult = checkCollectionExpectations(collectionName, numShardsNumReplicaList, nodesAllowedToRunShards); if (checkResult == null) { @@ -1743,9 +1745,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes public static void waitForNon403or404or503(HttpSolrClient collectionClient) throws Exception { SolrException exp = null; - long timeoutAt = System.currentTimeMillis() + 30000; + final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS); - while (System.currentTimeMillis() < timeoutAt) { + while (! timeout.hasTimedOut()) { boolean missing = false; try { @@ -1787,7 +1789,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes } protected List ensureAllReplicasAreActive(String testCollectionName, String shardId, int shards, int rf, int maxWaitSecs) throws Exception { - long startMs = System.currentTimeMillis(); + final RTimer timer = new RTimer(); Map notLeaders = new HashMap<>(); @@ -1845,8 +1847,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes if (notLeaders.isEmpty()) fail("Didn't isolate any replicas that are not the leader! ClusterState: " + printClusterStateInfo()); - long diffMs = (System.currentTimeMillis() - startMs); - log.info("Took " + diffMs + " ms to see all replicas become active."); + log.info("Took {} ms to see all replicas become active.", timer.getTime()); List replicas = new ArrayList<>(); replicas.addAll(notLeaders.values()); @@ -1878,9 +1879,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes static String getRequestStateAfterCompletion(String requestId, int waitForSeconds, SolrClient client) throws IOException, SolrServerException { String state = null; - long maxWait = System.nanoTime() + TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS); + final TimeOut timeout = new TimeOut(waitForSeconds, TimeUnit.SECONDS); - while (System.nanoTime() < maxWait) { + while (! timeout.hasTimedOut()) { state = getRequestState(requestId, client); if(state.equals("completed") || state.equals("failed")) return state; diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java index 13d5f77144c..c3ea9760546 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java @@ -30,6 +30,7 @@ import org.apache.solr.core.CoreContainer; import org.apache.solr.core.SolrCore; import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.update.DirectUpdateHandler2; +import org.apache.solr.util.RTimer; import org.apache.zookeeper.KeeperException; import org.eclipse.jetty.servlet.FilterHolder; import org.slf4j.Logger; @@ -79,7 +80,7 @@ public class ChaosMonkey { private boolean causeConnectionLoss; private boolean aggressivelyKillLeaders; private Map shardToLeaderJetty; - private volatile long startTime; + private volatile RTimer runTimer; private List deadPool = new ArrayList<>(); @@ -451,9 +452,9 @@ public class ChaosMonkey { monkeyLog("Jetty will not commit on close"); DirectUpdateHandler2.commitOnClose = false; } - + this.aggressivelyKillLeaders = killLeaders; - startTime = System.currentTimeMillis(); + runTimer = new RTimer(); // TODO: when kill leaders is on, lets kill a higher percentage of leaders stop = false; @@ -510,7 +511,7 @@ public class ChaosMonkey { } } monkeyLog("finished"); - monkeyLog("I ran for " + (System.currentTimeMillis() - startTime)/1000.0f + "sec. I stopped " + stops + " and I started " + starts + monkeyLog("I ran for " + runTimer.getTime() / 1000 + "s. I stopped " + stops + " and I started " + starts + ". I also expired " + expires.get() + " and caused " + connloss + " connection losses"); } @@ -529,10 +530,11 @@ public class ChaosMonkey { } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - + runTimer.stop(); + DirectUpdateHandler2.commitOnClose = true; - - float runtime = (System.currentTimeMillis() - startTime)/1000.0f; + + double runtime = runTimer.getTime()/1000.0f; if (runtime > 30 && stops.get() == 0) { LuceneTestCase.fail("The Monkey ran for over 30 seconds and no jetties were stopped - this is worth investigating!"); } diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java index cd1a9a209fe..b36bec3d0f4 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java @@ -20,6 +20,7 @@ package org.apache.solr.cloud; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.AtomicLongMap; import org.apache.solr.common.cloud.SolrZkClient; +import org.apache.solr.util.TimeOut; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; @@ -56,6 +57,7 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; public class ZkTestServer { public static final int TICK_TIME = 1000; @@ -521,8 +523,8 @@ public class ZkTestServer { } } - public static boolean waitForServerDown(String hp, long timeout) { - long start = System.currentTimeMillis(); + public static boolean waitForServerDown(String hp, long timeoutMs) { + final TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS); while (true) { try { HostPort hpobj = parseHostPortList(hp).get(0); @@ -531,7 +533,7 @@ public class ZkTestServer { return true; } - if (System.currentTimeMillis() > start + timeout) { + if (timeout.hasTimedOut()) { break; } try {