mirror of https://github.com/apache/lucene.git
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:
parent
a050d3accb
commit
b4aafbb606
|
@ -31,3 +31,6 @@ java.util.concurrent.ThreadPoolExecutor#<init>(int,int,long,java.util.concurrent
|
|||
@defaultMessage Use slf4j classes instead
|
||||
org.apache.log4j.**
|
||||
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()
|
||||
|
|
|
@ -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
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -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,7 +624,7 @@ 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
|
||||
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
@ -520,6 +522,7 @@ public abstract class AbstractSqlEntityProcessorTestCase extends
|
|||
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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
@ -47,6 +48,7 @@ public class TestSimplePropertiesWriter extends AbstractDIHJdbcTestCase {
|
|||
fileName = "the.properties";
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Needs currentTimeMillis to construct date stamps")
|
||||
@Test
|
||||
public void testSimplePropertiesWriter() throws Exception {
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
LOG.info("Optimizing Solr: done forcing tree merge down to {} segments in {}ms", maxSegments, timer.getTime());
|
||||
|
||||
start = System.nanoTime();
|
||||
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();
|
||||
|
|
|
@ -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,10 +640,9 @@ 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) {
|
||||
if (timeout.hasTimedOut()) {
|
||||
throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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,11 +1545,14 @@ 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);
|
||||
|
@ -1552,18 +1566,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
|
|||
result += (60 * Integer.parseInt(min));
|
||||
if (hr != null && hr.length() > 0)
|
||||
result += (60 * 60 * Integer.parseInt(hr));
|
||||
result *= 1000;
|
||||
return TimeUnit.NANOSECONDS.convert(result, TimeUnit.SECONDS);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
INTERVAL_ERR_MSG);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, INTERVAL_ERR_MSG);
|
||||
}
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
INTERVAL_ERR_MSG);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, INTERVAL_ERR_MSG);
|
||||
}
|
||||
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private static final String LOCATION = "location";
|
||||
|
|
|
@ -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">
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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;
|
||||
|
@ -146,6 +147,7 @@ public class DebugComponent extends SearchComponent
|
|||
|
||||
}
|
||||
|
||||
@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();
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
@ -85,10 +86,7 @@ public class OpenExchangeRatesOrgProvider implements ExchangeRateProvider {
|
|||
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;
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2243,10 +2244,20 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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)+'"');
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
@ -46,6 +52,7 @@ public class BlockCacheLocation {
|
|||
return bankId;
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Need currentTimeMillis, only used by unused getLastAccess")
|
||||
public void touch() {
|
||||
lastAccess = System.currentTimeMillis();
|
||||
accesses++;
|
||||
|
|
|
@ -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;
|
||||
|
@ -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()) {
|
||||
|
|
|
@ -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));
|
||||
timer.getTime(), String.valueOf(highestVersion));
|
||||
|
||||
return highestVersion;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -52,7 +52,10 @@ public class SolrLogLayout extends Layout {
|
|||
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<>();
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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,7 +48,7 @@ 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,7 +121,7 @@ public class TransformerProvider {
|
|||
|
||||
lastFilename = filename;
|
||||
lastTemplates = result;
|
||||
cacheExpires = System.currentTimeMillis() + (cacheLifetimeSeconds * 1000);
|
||||
cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
@ -320,9 +321,9 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
|||
params.set("commitWithin", 10);
|
||||
add(cloudClient, params, getDoc("id", 300));
|
||||
|
||||
long timeout = System.currentTimeMillis() + 45000;
|
||||
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);
|
||||
|
@ -375,9 +376,9 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
}
|
||||
|
||||
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);
|
||||
|
|
|
@ -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;
|
||||
|
@ -281,9 +282,9 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
|||
|
||||
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");
|
||||
}
|
||||
|
||||
|
@ -936,10 +937,10 @@ 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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
@ -314,9 +316,9 @@ public class CollectionsAPISolrJTests extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertEquals(0, response.getStatus());
|
||||
|
||||
timeout = System.currentTimeMillis() + 3000;
|
||||
timeout = new TimeOut(3, TimeUnit.SECONDS);
|
||||
|
||||
while (System.currentTimeMillis() < timeout && newReplica != null) {
|
||||
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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 = {
|
||||
|
@ -203,9 +205,9 @@ public class StressHdfsTest extends BasicDistributedZkTest {
|
|||
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");
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
@ -59,6 +60,7 @@ public class NoCacheHeaderTest extends CacheHeaderTestBase {
|
|||
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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
@ -124,19 +125,22 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase {
|
|||
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: " +
|
||||
searcher529 + " !<= " + hard529,
|
||||
|
@ -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();
|
||||
|
@ -252,19 +256,22 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase {
|
|||
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 );
|
||||
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: " +
|
||||
searcher529 + " !<= " + hard529,
|
||||
|
@ -298,16 +305,16 @@ 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++) {
|
||||
|
@ -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
|
||||
long softMs = TimeUnit.MILLISECONDS.convert(soft - fast5end, TimeUnit.NANOSECONDS);
|
||||
assertTrue(i + ": soft occured too fast: " +
|
||||
fast5end + " + (" + softCommitWaitMillis + " * " + i +
|
||||
") !<= " + soft,
|
||||
fast5end + (softCommitWaitMillis * i) <= soft);
|
||||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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());
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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) );
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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("*:*"));
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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<>();
|
||||
|
||||
|
@ -453,7 +454,7 @@ public class ChaosMonkey {
|
|||
}
|
||||
|
||||
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!");
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue