SOLR-7859: Clamp down on use of System.currentTimeMillis

- Use RTimer where currentTimeMillis is used for timing
 - Abstract out a new class TimeOut for when currentTimeMillis/nanoTime
   is used to timeout operations.
 - Used `new Date()` in some cases where that is the logical intent.
 - Deprecated a couple of APIs which were publicly exposing epoch time,
   in favour of Date objects to make the intent clearer.
 - A couple of cases had currentTimeMillis in dead code.
 - In some cases where currentTimeMillis was used to just generate a name,
   used nanoTime instead (really it should be some sequence/random number
   in such a case).
 - In some other cases where stamps were used for SQL queries, HTTP headers,
   comparing against data in external files, ZK etc., used SuppressForbidden.
 - Also currently allow the use of currentTimeMillis in commit data,
   this is then used in replication -- this is concerning since absolute
   times are being compared, but that can be dealt with separately.



git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1694798 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ramkumar Aiyengar 2015-08-08 13:39:58 +00:00
parent a050d3accb
commit b4aafbb606
92 changed files with 647 additions and 472 deletions

View File

@ -30,4 +30,7 @@ java.util.concurrent.ThreadPoolExecutor#<init>(int,int,long,java.util.concurrent
@defaultMessage Use slf4j classes instead
org.apache.log4j.**
java.util.logging.**
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()

View File

@ -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

View File

@ -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);

View File

@ -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);

View File

@ -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;

View File

@ -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();

View File

@ -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 {

View File

@ -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

View File

@ -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<Text, NullWritable>
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();

View File

@ -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);

View File

@ -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();
}
}
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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<String, RoutingRule> 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<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreNames) throws InterruptedException {
Map<String, Replica> 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.");
}

View File

@ -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<SearchComponent> searchComponents = new PluginBag<>(SearchComponent.class, this);
private final PluginBag<UpdateRequestProcessorFactory> 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<Object> 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());

View File

@ -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<ErrorQueueEntry> 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;
}

View File

@ -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<Map<String,Object>> 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<Map<String, Object>> modifiedConfFiles, boolean successfulInstall) throws IOException {
List<String> 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;
}

View File

@ -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<String, FileInfo> 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";

View File

@ -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.
* <table border="1" summary="table of parameters">

View File

@ -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<PerReplicaCallable> 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<String> 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;
}

View File

@ -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<SolrIndexSearcher> searcher = core.getSearcher();
try {

View File

@ -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<Object> dirs = new SimpleOrderedMap<>();

View File

@ -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();

View File

@ -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<String,Object> 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() {

View File

@ -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<GetZkSchemaVersionCallable> 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<String> getActiveReplicaCoreUrls(ZkController zkController, String collection, String localCoreNodeName) {

View File

@ -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;

View File

@ -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<Object> 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

View File

@ -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;
}

View File

@ -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);

View File

@ -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;

View File

@ -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)+'"');
}
}

View File

@ -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;
}

View File

@ -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<IndexWriter> 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<String,String> 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<IndexWriter> iw = solrCoreState.getIndexWriter(core);
try {
final Map<String,String> 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<String,String> 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<String,String> commitData = new HashMap<>();
commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, String.valueOf(System.currentTimeMillis()));
writer.setCommitData(commitData);
setCommitData(writer);
writer.commit();
synchronized (solrCoreState.getUpdateLock()) {

View File

@ -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;
}

View File

@ -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;

View File

@ -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<DocRouter.Range> ranges = rule.getRouteRanges();
if (ranges != null && !ranges.isEmpty()) {
int hash = compositeIdRouter.sliceHash(id, doc, null, coll);

View File

@ -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());
}
/**

View File

@ -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<Method,String> methodAlias = new HashMap<>();

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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;
}
/**

View File

@ -48,7 +48,6 @@ public class SolrInfoMBeanTest extends SolrTestCaseJ4
public void testCallMBeanInfo() throws Exception {
List<Class> 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()));

View File

@ -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);

View File

@ -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<String,Long> 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<String,Long> 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() <timeOut){
while(! timeout.hasTimedOut()){
Thread.sleep(10);
changed = Objects.equals(val,client.getZkStateReader().getClusterProps().get(name));
if(changed) break;

View File

@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
import org.junit.Test;
@ -45,6 +46,7 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import static org.apache.solr.cloud.ReplicaPropertiesBase.verifyUniqueAcrossCollection;
@ -290,10 +292,10 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
assertEquals(0, response.getStatus());
assertTrue(response.isSuccess());
long timeout = System.currentTimeMillis() + 3000;
TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
Replica newReplica = null;
while (System.currentTimeMillis() < timeout && newReplica == null) {
while (! timeout.hasTimedOut() && newReplica == null) {
Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
newReplica = slice.getReplica(newReplicaName);
}
@ -313,10 +315,10 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
response = deleteReplicaRequest.process(cloudClient);
assertEquals(0, response.getStatus());
timeout = System.currentTimeMillis() + 3000;
while (System.currentTimeMillis() < timeout && newReplica != null) {
timeout = new TimeOut(3, TimeUnit.SECONDS);
while (! timeout.hasTimedOut() && newReplica != null) {
Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
newReplica = slice.getReplica(newReplicaName);
}
@ -332,10 +334,10 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
assertEquals(0, response.getStatus());
long timeOut = System.currentTimeMillis() + 3000;
TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
boolean changed = false;
while(System.currentTimeMillis() < timeOut){
while(! timeout.hasTimedOut()){
Thread.sleep(10);
changed = Objects.equals("false",
cloudClient.getZkStateReader().getClusterProps().get(ZkStateReader.LEGACY_CLOUD));
@ -349,9 +351,9 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
.setPropertyValue(null);
clusterPropRequest.process(cloudClient);
timeOut = System.currentTimeMillis() + 3000;
timeout = new TimeOut(3, TimeUnit.SECONDS);
changed = false;
while(System.currentTimeMillis() < timeOut){
while(! timeout.hasTimedOut()) {
Thread.sleep(10);
changed = (cloudClient.getZkStateReader().getClusterProps().get(ZkStateReader.LEGACY_CLOUD) == null);
if(changed)
@ -423,11 +425,11 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
.setPropertyValue("true").process(cloudClient);
assertEquals(0, response.getStatus());
long timeout = System.currentTimeMillis() + 20000;
TimeOut timeout = new TimeOut(20, TimeUnit.SECONDS);
String propertyValue = null;
String replicaName = replica.getName();
while (System.currentTimeMillis() < timeout) {
while (! timeout.hasTimedOut()) {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
propertyValue = replica.getStr("property.preferredleader");
@ -448,10 +450,10 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
.setPropertyName("property.preferredleader").process(cloudClient);
assertEquals(0, response.getStatus());
timeout = System.currentTimeMillis() + 20000;
timeout = new TimeOut(20, TimeUnit.SECONDS);
boolean updated = false;
while (System.currentTimeMillis() < timeout) {
while (! timeout.hasTimedOut()) {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
updated = replica.getStr("property.preferredleader") == null;

View File

@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
@ -142,8 +143,8 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
@Override
public void run() {
final long timeToStop = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(timeToRunSec);
while (System.currentTimeMillis() < timeToStop && failure.get() == null) {
final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS);
while (! timeout.hasTimedOut() && failure.get() == null) {
doWork();
}
}

View File

@ -23,6 +23,7 @@ import java.net.URL;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@ -37,6 +38,7 @@ import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
public class DeleteInactiveReplicaTest extends AbstractFullDistribZkTestBase{
@ -66,9 +68,9 @@ public class DeleteInactiveReplicaTest extends AbstractFullDistribZkTestBase{
StringBuilder sb = new StringBuilder();
Replica replica1 = null;
Slice shard1 = null;
long timeout = System.currentTimeMillis() + 3000;
TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
DocCollection testcoll = null;
while (!stopped && System.currentTimeMillis() < timeout) {
while (!stopped && ! timeout.hasTimedOut()) {
testcoll = client.getZkStateReader().getClusterState().getCollection(collectionName);
for (JettySolrRunner jetty : jettys)
sb.append(jetty.getBaseUrl()).append(",");
@ -102,9 +104,9 @@ public class DeleteInactiveReplicaTest extends AbstractFullDistribZkTestBase{
+ " jettys: " + sb);
}
long endAt = System.currentTimeMillis() + 3000;
timeout = new TimeOut(20, TimeUnit.SECONDS);
boolean success = false;
while (System.currentTimeMillis() < endAt) {
while (! timeout.hasTimedOut()) {
testcoll = client.getZkStateReader()
.getClusterState().getCollection(collectionName);
if (testcoll.getSlice(shard1.getName()).getReplica(replica1.getName()).getState() != Replica.State.ACTIVE) {

View File

@ -29,12 +29,14 @@ import org.apache.solr.common.cloud.ZkStateReader;
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.junit.Test;
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.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
@ -89,10 +91,10 @@ public class DeleteLastCustomShardedReplicaTest extends AbstractFullDistribZkTes
SolrRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
client.request(request);
long endAt = System.currentTimeMillis() + 3000;
TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
boolean success = false;
DocCollection testcoll = null;
while (System.currentTimeMillis() < endAt) {
while (! timeout.hasTimedOut()) {
testcoll = getCommonCloudSolrClient().getZkStateReader()
.getClusterState().getCollection(COLL_NAME);
// In case of a custom sharded collection, the last replica deletion would also lead to

View File

@ -32,6 +32,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
import java.io.File;
@ -39,6 +40,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.cloud.OverseerCollectionMessageHandler.ONLY_IF_DOWN;
@ -133,10 +135,10 @@ public class DeleteReplicaTest extends AbstractFullDistribZkTestBase {
SolrRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
client.request(request);
long endAt = System.currentTimeMillis() + 3000;
TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
boolean success = false;
DocCollection testcoll = null;
while (System.currentTimeMillis() < endAt) {
while (! timeout.hasTimedOut()) {
testcoll = client.getZkStateReader()
.getClusterState().getCollection(COLL_NAME);
success = testcoll.getSlice(shard).getReplica(replica.getName()) == null;

View File

@ -26,6 +26,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactory; // jdoc
import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactoryTest;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -36,6 +37,7 @@ import java.util.Map;
import java.util.Set;
import java.util.HashSet;
import java.util.HashMap;
import java.util.concurrent.TimeUnit;
/** Test of {@link DocExpirationUpdateProcessorFactory} in a cloud setup */
@Slow // Has to do some sleeping to wait for a future expiration
@ -171,10 +173,10 @@ public class DistribDocExpirationUpdateProcessorTest extends AbstractFullDistrib
SolrParams params)
throws SolrServerException, InterruptedException, IOException {
final long giveUpAfter = System.currentTimeMillis() + (1000L * maxTimeLimitSeconds);
final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS);
long numFound = cloudClient.query(params).getResults().getNumFound();
while (0L < numFound && System.currentTimeMillis() < giveUpAfter) {
Thread.sleep(Math.min(5000, giveUpAfter - System.currentTimeMillis()));
while (0L < numFound && ! timeout.hasTimedOut()) {
Thread.sleep(Math.max(1, Math.min(5000, timeout.timeLeft(TimeUnit.MILLISECONDS))));
numFound = cloudClient.query(params).getResults().getNumFound();
}
assertEquals("Give up waiting for no results: " + params,

View File

@ -40,6 +40,7 @@ import org.apache.solr.core.SolrCore;
import org.apache.solr.servlet.SolrDispatchFilter;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.util.RTimer;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -520,7 +521,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
}
protected void waitToSeeReplicasActive(String testCollectionName, String shardId, Set<String> 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);
}
}

View File

@ -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) {

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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());

View File

@ -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<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();

View File

@ -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);
}

View File

@ -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);

View File

@ -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");
}

View File

@ -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<OneIndexer> _threads = new ArrayList<>();
public Indexer(OpenCloseCoreStressTest OCCST, String url, List<HttpSolrClient> 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);

View File

@ -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();
}
}
}

View File

@ -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) {

View File

@ -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");

View File

@ -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<encIter; i++) {
if (w instanceof BinaryQueryResponseWriter) {
BinaryQueryResponseWriter binWriter = (BinaryQueryResponseWriter) w;
@ -126,11 +127,11 @@ public class TestWriterPerf extends AbstractSolrTestCase {
}
}
long encodeTime = Math.max(System.currentTimeMillis() - start, 1);
double encodeTime = timer.getTime();
byte[] arr = out.toByteArray();
start = System.currentTimeMillis();
timer = new RTimer();
writerName = writerName.intern();
for (int i=0; i<decIter; i++) {
ResponseParser rp = null;
@ -145,9 +146,9 @@ public class TestWriterPerf extends AbstractSolrTestCase {
rp.processResponse(in, "UTF-8");
}
long decodeTime = Math.max(System.currentTimeMillis() - start, 1);
double decodeTime = timer.getTime();
log.info("writer "+writerName+", size="+out.size()+", encodeRate="+(encodeTime==1 ? "N/A": ""+(encIter*1000L/encodeTime)) + ", decodeRate="+(decodeTime==1 ? "N/A": ""+(decIter*1000L/decodeTime)) );
log.info("writer "+writerName+", size="+out.size()+", encodeRate="+(encIter*1000L/encodeTime) + ", decodeRate="+(decIter*1000L/decodeTime));
req.close();
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.schema;
import org.apache.lucene.index.StorableField;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.RTimer;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
@ -260,25 +261,25 @@ public abstract class AbstractCurrencyFieldTest extends SolrTestCaseJ4 {
}
for (int j = 0; j < 3; j++) {
long t1 = System.currentTimeMillis();
final RTimer timer = new RTimer();
for (int i = 0; i < 1000; i++) {
double lower = r.nextInt(10) + 1.00;
assertQ(req("fl", "*,score", "q", field()+":[" + lower + ",USD TO " + (lower + (9.99 - (j * 0.01))) + ",USD]"), "//*");
}
System.out.println(System.currentTimeMillis() - t1);
System.out.println(timer.getTime());
}
System.out.println("---");
for (int j = 0; j < 3; j++) {
long t1 = System.currentTimeMillis();
final RTimer timer = new RTimer();
for (int i = 0; i < 1000; i++) {
double lower = r.nextInt(10) + 1.00;
assertQ(req("fl", "*,score", "q", field()+":[" + lower + ",EUR TO " + (lower + (9.99 - (j * 0.01))) + ",EUR]"), "//*");
}
System.out.println(System.currentTimeMillis() - t1);
System.out.println(timer.getTime());
}
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.schema;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.lucene.analysis.util.ResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.core.SolrResourceLoader;
import org.junit.Before;
import org.junit.Test;
@ -86,6 +87,7 @@ public class OpenExchangeRatesOrgProviderTest extends SolrTestCaseJ4 {
assertEquals("USD", oerp.rates.getBaseCurrency());
}
@SuppressForbidden(reason = "Needs currentTimeMillis to construct rates file contents")
@Test
public void testReload() {
// reminder: interval is in minutes

View File

@ -21,6 +21,7 @@ import java.util.Random;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.solr.util.RTimer;
/**
*/
@ -90,7 +91,7 @@ public class DocSetPerf {
hset[i] = hds;
}
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
if ("test".equals(test)) {
for (int it=0; it<iter; it++) {
@ -165,8 +166,7 @@ public class DocSetPerf {
}
}
long end = System.currentTimeMillis();
System.out.println("TIME="+(end-start));
System.out.println("TIME="+timer.getTime());
System.out.println("ret="+ret);
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.search;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.ConcurrentLRUCache;
import org.apache.solr.util.RTimer;
import java.io.IOException;
import java.io.Serializable;
@ -254,7 +255,7 @@ public class TestFastLRUCache extends LuceneTestCase {
}
void doPerfTest(int iter, int cacheSize, int maxKey) {
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
int lowerWaterMark = cacheSize;
int upperWaterMark = (int)(lowerWaterMark * 1.1);
@ -276,8 +277,7 @@ public class TestFastLRUCache extends LuceneTestCase {
}
cache.destroy();
long end = System.currentTimeMillis();
System.out.println("time=" + (end-start) + ", minSize="+minSize+",maxSize="+maxSize);
System.out.println("time=" + timer.getTime() + ", minSize="+minSize+",maxSize="+maxSize);
}
/***
@ -326,7 +326,7 @@ public class TestFastLRUCache extends LuceneTestCase {
fillCache(sc, cacheSize, maxKey);
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
Thread[] threads = new Thread[nThreads];
final AtomicInteger puts = new AtomicInteger(0);
@ -357,8 +357,7 @@ public class TestFastLRUCache extends LuceneTestCase {
}
}
long end = System.currentTimeMillis();
System.out.println("time=" + (end-start) + " impl=" +sc.getClass().getSimpleName()
System.out.println("time=" + timer.getTime() + " impl=" +sc.getClass().getSimpleName()
+" nThreads= " + nThreads + " size="+cacheSize+" maxKey="+maxKey+" gets="+numGets
+" hitRatio="+(1-(((double)puts.get())/numGets)));
}

View File

@ -27,6 +27,7 @@ import org.apache.solr.update.processor.UpdateRequestProcessorChain;
import org.apache.solr.update.processor.UpdateRequestProcessor;
import org.apache.solr.update.AddUpdateCommand;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.util.RTimer;
import org.junit.BeforeClass;
import java.util.*;
@ -139,7 +140,7 @@ public class TestSearchPerf extends AbstractSolrTestCase {
SolrIndexSearcher searcher = req.getSearcher();
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
int ret = 0;
for (int i=0; i<iter; i++) {
@ -147,8 +148,8 @@ public class TestSearchPerf extends AbstractSolrTestCase {
ret += set.size();
}
long end = System.currentTimeMillis();
System.out.println("ret="+ret+ " time="+(end-start)+" throughput="+iter*1000/(end-start+1));
double elapsed = timer.getTime();
System.out.println("ret="+ret+ " time="+elapsed+" throughput="+iter*1000/(elapsed+1));
req.close();
assertTrue(ret>0); // make sure we did some work
@ -160,7 +161,7 @@ public class TestSearchPerf extends AbstractSolrTestCase {
SolrIndexSearcher searcher = req.getSearcher();
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
// These aren't public in SolrIndexSearcher
int NO_CHECK_QCACHE = 0x80000000;
@ -174,8 +175,8 @@ public class TestSearchPerf extends AbstractSolrTestCase {
ret += l.matches();
}
long end = System.currentTimeMillis();
System.out.println("ret="+ret+ " time="+(end-start)+" throughput="+iter*1000/(end-start+1));
double elapsed = timer.getTime();
System.out.println("ret="+ret+ " time="+elapsed+" throughput="+iter*1000/(elapsed+1));
req.close();
assertTrue(ret>0); // make sure we did some work

View File

@ -24,6 +24,7 @@ import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.util.RTimer;
import java.io.IOException;
import java.util.ArrayList;
@ -64,7 +65,7 @@ public class TestSolrJ extends SolrTestCaseJ4 {
client.deleteByQuery("*:*");
client.commit();
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
final int docsPerThread = nDocs / nProducers;
@ -98,8 +99,8 @@ public class TestSolrJ extends SolrTestCaseJ4 {
concurrentClient.blockUntilFinished();
}
long end = System.currentTimeMillis();
System.out.println("time="+(end-start) + " throughput="+(nDocs*1000/(end-start)) + " Exception="+ex);
double elapsed = timer.getTime();
System.out.println("time="+elapsed + " throughput="+(nDocs*1000/elapsed) + " Exception="+ex);
// should server threads be marked as daemon?
// need a server.close()!!!
@ -166,7 +167,7 @@ public class TestSolrJ extends SolrTestCaseJ4 {
try (HttpSolrClient client = new HttpSolrClient("http://127.0.0.1:8983/solr")) {
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
for (int i = 0; i < 10000; i++) {
SolrInputDocument doc = new SolrInputDocument();
@ -175,8 +176,7 @@ public class TestSolrJ extends SolrTestCaseJ4 {
client.commit(true, true, true);
}
long end = System.currentTimeMillis();
System.out.println("TIME: " + (end-start));
System.out.println("TIME: " + timer.getTime());
}
}

View File

@ -17,17 +17,6 @@
package org.apache.solr.search.function;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.TFIDFSimilarity;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.Ignore;
import java.io.FileOutputStream;
import java.io.OutputStreamWriter;
import java.io.Writer;
@ -37,6 +26,13 @@ import java.util.Arrays;
import java.util.List;
import java.util.Random;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.TFIDFSimilarity;
import org.apache.solr.SolrTestCaseJ4;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Tests some basic functionality of Solr while demonstrating good
* Best Practices for using AbstractSolrTestCase
@ -49,7 +45,8 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
String base = "external_foo_extf";
static long start = System.currentTimeMillis();
static long start = System.nanoTime();
void makeExternalFile(String field, String contents) {
String dir = h.getCore().getDataDir();

View File

@ -23,6 +23,7 @@ import org.apache.http.HttpResponse;
import org.apache.http.client.methods.HttpRequestBase;
import org.apache.http.impl.cookie.DateUtils;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.util.SuppressForbidden;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -77,6 +78,7 @@ public class CacheHeaderTest extends CacheHeaderTestBase {
checkVetoHeaders(response, false);
}
@SuppressForbidden(reason = "Needs currentTimeMillis to check against expiry headers from Solr")
protected void checkVetoHeaders(HttpResponse response, boolean checkExpires) throws Exception {
Header head = response.getFirstHeader("Cache-Control");
assertNotNull("We got no Cache-Control header", head);

View File

@ -22,6 +22,7 @@ import org.apache.http.Header;
import org.apache.http.HttpResponse;
import org.apache.http.client.methods.HttpRequestBase;
import org.apache.http.impl.cookie.DateUtils;
import org.apache.solr.common.util.SuppressForbidden;
import org.junit.BeforeClass;
import org.junit.Test;
@ -58,7 +59,8 @@ public class NoCacheHeaderTest extends CacheHeaderTestBase {
doCacheControl("HEAD");
doCacheControl("POST");
}
@SuppressForbidden(reason = "Needs currentTimeMillis for testing caching headers")
@Override
protected void doLastModified(String method) throws Exception {
// We do a first request to get the last modified

View File

@ -20,6 +20,7 @@ package org.apache.solr.update;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.common.params.MapSolrParams;
@ -34,6 +35,7 @@ import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.AbstractSolrTestCase;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.TimeOut;
import org.junit.BeforeClass;
class NewSearcherListener implements SolrEventListener {
@ -85,9 +87,9 @@ class NewSearcherListener implements SolrEventListener {
// log.info("TEST: trigger reset");
}
boolean waitForNewSearcher(int timeout) {
long timeoutTime = System.currentTimeMillis() + timeout;
while (System.currentTimeMillis() < timeoutTime) {
boolean waitForNewSearcher(int timeoutMs) {
TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
while (! timeout.hasTimedOut()) {
if (triggered) {
// check if the new searcher has been registered yet
RefCounted<SolrIndexSearcher> registeredSearcherH = newSearcher.getCore().getSearcher();

View File

@ -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

View File

@ -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);
}

View File

@ -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));

View File

@ -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<iter; it++) {
@ -187,9 +187,8 @@ public class BitSetPerf {
}
}
long end = System.currentTimeMillis();
System.out.println("ret="+ret);
System.out.println("TIME="+(end-start));
System.out.println("TIME="+timer.getTime());
}

View File

@ -149,7 +149,7 @@ public class TestUtils extends SolrTestCaseJ4 {
String sortable = NumberUtils.double2sortableStr( number );
assertEquals( number, NumberUtils.SortableStr2double(sortable), 0.001);
long num = System.currentTimeMillis();
long num = System.nanoTime();
sortable = NumberUtils.long2sortableStr( num );
assertEquals( num, NumberUtils.SortableStr2long(sortable, 0, sortable.length() ) );
assertEquals( Long.toString(num), NumberUtils.SortableStr2long(sortable) );

View File

@ -155,11 +155,12 @@ public class CloudSolrClient extends SolrClient {
ExpiringCachedDocCollection(DocCollection cached) {
this.cached = cached;
this.cachedAt = System.currentTimeMillis();
this.cachedAt = System.nanoTime();
}
boolean isExpired(long timeToLive) {
return (System.currentTimeMillis() - cachedAt) > timeToLive;
boolean isExpired(long timeToLiveMs) {
return (System.nanoTime() - cachedAt)
> TimeUnit.NANOSECONDS.convert(timeToLiveMs, TimeUnit.MILLISECONDS);
}
}

View File

@ -128,9 +128,6 @@ public class LBHttpSolrClient extends SolrClient {
final HttpSolrClient client;
long lastUsed; // last time used for a real request
long lastChecked; // last time checked for liveness
// "standard" servers are used by default. They normally live in the alive list
// and move to the zombie list when unavailable. When they become available again,
// they move back to the alive list.
@ -362,7 +359,6 @@ public class LBHttpSolrClient extends SolrClient {
ServerWrapper wrapper;
wrapper = new ServerWrapper(server);
wrapper.lastUsed = System.currentTimeMillis();
wrapper.standard = false;
zombieServers.put(wrapper.getKey(), wrapper);
startAliveCheckExecutor();
@ -514,7 +510,6 @@ public class LBHttpSolrClient extends SolrClient {
int count = counter.incrementAndGet() & Integer.MAX_VALUE;
ServerWrapper wrapper = serverList[count % serverList.length];
wrapper.lastUsed = System.currentTimeMillis();
try {
return wrapper.client.request(request, collection);
@ -591,9 +586,7 @@ public class LBHttpSolrClient extends SolrClient {
* @param zombieServer a server in the dead pool
*/
private void checkAZombieServer(ServerWrapper zombieServer) {
long currTime = System.currentTimeMillis();
try {
zombieServer.lastChecked = currTime;
QueryResponse resp = zombieServer.client.query(solrQuery);
if (resp.getStatus() == 0) {
// server has come back up.

View File

@ -294,7 +294,6 @@ public class CloudSolrStream extends TupleStream implements Expressible {
throw new Exception("Collection not found:"+this.collection);
}
long time = System.currentTimeMillis();
params.put("distrib","false"); // We are the aggregator.
for(Slice slice : slices) {
@ -304,7 +303,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
shuffler.add(replica);
}
Collections.shuffle(shuffler, new Random(time));
Collections.shuffle(shuffler, new Random());
Replica rep = shuffler.get(0);
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
String url = zkProps.getCoreUrl();

View File

@ -252,7 +252,6 @@ public class ParallelStream extends CloudSolrStream implements Expressible {
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
long time = System.currentTimeMillis();
List<Replica> shuffler = new ArrayList();
for(Slice slice : slices) {
Collection<Replica> 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<workers; w++) {
HashMap params = new HashMap();

View File

@ -21,6 +21,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.solr.common.util.SuppressForbidden;
import org.noggit.JSONUtil;
/**
@ -56,8 +57,14 @@ public class RoutingRule extends ZkNodeProps {
return targetCollectionName;
}
public Long getExpireAt() {
return expireAt;
@SuppressForbidden(reason = "For currentTimeMillis, expiry time depends on external data (should it?)")
public static String makeExpiryAt(long timeMsFromNow) {
return String.valueOf(System.currentTimeMillis() + timeMsFromNow);
}
@SuppressForbidden(reason = "For currentTimeMillis, expiry time depends on external data (should it?)")
public boolean isExpired() {
return (expireAt < System.currentTimeMillis());
}
public String getRouteRangesStr() {

View File

@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -33,6 +34,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
abstract public class SolrExampleTestsBase extends SolrJettyTestBase {
@ -149,7 +151,7 @@ abstract public class SolrExampleTestsBase extends SolrJettyTestBase {
Assert.assertEquals(1, rsp.getResults().getNumFound());
// check if the doc has been deleted every 250 ms for 30 seconds
long timeout = System.currentTimeMillis() + 30000;
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
do {
Thread.sleep(250); // wait 250 ms
@ -157,7 +159,7 @@ abstract public class SolrExampleTestsBase extends SolrJettyTestBase {
if (rsp.getResults().getNumFound() == 0) {
return;
}
} while (System.currentTimeMillis() < timeout);
} while (! timeout.hasTimedOut());
Assert.fail("commitWithin failed to commit");
}

View File

@ -35,6 +35,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.SolrResponseBase;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.util.TimeOut;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.slf4j.Logger;
@ -48,6 +49,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
/**
* Test for LBHttpSolrClient
@ -221,16 +223,16 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 {
// Start the killed server once again
solr[1].startJetty();
// Wait for the alive check to complete
waitForServer(30000, client, 3, "solr1");
waitForServer(30, client, 3, "solr1");
} finally {
myHttpClient.close();
}
}
// wait maximum ms for serverName to come back up
private void waitForServer(int maximum, LBHttpSolrClient client, int nServers, String serverName) throws Exception {
long endTime = System.currentTimeMillis() + maximum;
while (System.currentTimeMillis() < endTime) {
private void waitForServer(int maxSeconds, LBHttpSolrClient client, int nServers, String serverName) throws Exception {
final TimeOut timeout = new TimeOut(maxSeconds, TimeUnit.SECONDS);
while (! timeout.hasTimedOut()) {
QueryResponse resp;
try {
resp = client.query(new SolrQuery("*:*"));

View File

@ -41,6 +41,7 @@ import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.SolrInputField;
import org.apache.solr.util.ConcurrentLRUCache;
import org.apache.solr.util.RTimer;
import org.junit.Test;
import org.noggit.CharArr;
@ -439,7 +440,7 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
}
printMem("after cache init");
long ms = System.currentTimeMillis();
RTimer timer = new RTimer();
final int ITERS = 1000000;
int THREADS = 10;
@ -459,8 +460,8 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
printMem("after cache test");
System.out.println("time taken by LRUCACHE "+ (System.currentTimeMillis()-ms));
ms = System.currentTimeMillis();
System.out.println("time taken by LRUCACHE " + timer.getTime());
timer = new RTimer();
runInThreads(THREADS, new Runnable() {
@Override
@ -477,7 +478,7 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
});
printMem("after new string test");
System.out.println("time taken by string creation "+ (System.currentTimeMillis()-ms));
System.out.println("time taken by string creation "+ timer.getTime());
@ -569,7 +570,7 @@ public class TestJavaBinCodec extends SolrTestCaseJ4 {
}
int ret = 0;
long start = System.currentTimeMillis();
final RTimer timer = new RTimer();
ConcurrentLRUCache underlyingCache = cacheSz > 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());
}

View File

@ -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<Integer> numShardsNumReplicaList,
List<String> 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<Replica> ensureAllReplicasAreActive(String testCollectionName, String shardId, int shards, int rf, int maxWaitSecs) throws Exception {
long startMs = System.currentTimeMillis();
final RTimer timer = new RTimer();
Map<String,Replica> 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<Replica> 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;

View File

@ -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<String,CloudJettyRunner> shardToLeaderJetty;
private volatile long startTime;
private volatile RTimer runTimer;
private List<CloudJettyRunner> 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!");
}

View File

@ -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 {