SOLR-5734: We should use System.nanoTime rather than System.currentTimeMillis when calculating elapsed time.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1573425 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2014-03-03 02:08:45 +00:00
parent 1263b95738
commit 9441f0d4ce
34 changed files with 208 additions and 167 deletions

View File

@ -103,6 +103,9 @@ Bug Fixes
* SOLR-5782: The full MapReduceIndexer help text does not display when using
--help. (Mark Miller, Wolfgang Hoschek)
* SOLR-5734: We should use System.nanoTime rather than System.currentTimeMillis
when calculating elapsed time. (Mark Miller, Ramkumar Aiyengar)
Optimizations
----------------------
* SOLR-1880: Distributed Search skips GET_FIELDS stage if EXECUTE_QUERY

View File

@ -183,7 +183,7 @@ class DebugLogger {
@Override
public Object getData(String query) {
log(DIHLogLevels.ENTITY_META, "query", query);
long start = System.currentTimeMillis();
long start = System.nanoTime();
try {
return ds.getData(query);
} catch (DataImportHandlerException de) {

View File

@ -28,6 +28,7 @@ import org.apache.solr.handler.dataimport.config.EntityField;
import static org.apache.solr.handler.dataimport.SolrWriter.LAST_INDEX_KEY;
import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField;
import org.slf4j.Logger;
@ -35,6 +36,7 @@ import org.slf4j.LoggerFactory;
import java.text.SimpleDateFormat;
import java.util.*;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@ -186,7 +188,7 @@ public class DocBuilder {
try {
dataImporter.store(DataImporter.STATUS_MSGS, statusMessages);
config = dataImporter.getConfig();
final AtomicLong startTime = new AtomicLong(System.currentTimeMillis());
final AtomicLong startTime = new AtomicLong(System.nanoTime());
statusMessages.put(TIME_ELAPSED, new Object() {
@Override
public String toString() {
@ -894,7 +896,7 @@ public class DocBuilder {
public static final String TIME_ELAPSED = "Time Elapsed";
static String getTimeElapsedSince(long l) {
l = System.currentTimeMillis() - l;
l = TimeUnit.MILLISECONDS.convert(System.nanoTime() - l, TimeUnit.NANOSECONDS);
return (l / (60000 * 60)) + ":" + (l / 60000) % 60 + ":" + (l / 1000)
% 60 + "." + l % 1000;
}

View File

@ -18,6 +18,7 @@ package org.apache.solr.handler.dataimport;
import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -27,6 +28,7 @@ import javax.naming.NamingException;
import java.sql.*;
import java.util.*;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
/**
* <p> A DataSource implementation which can fetch data using JDBC. </p> <p/> <p> Refer to <a
@ -132,7 +134,7 @@ public class JdbcDataSource extends
LOG.info("Creating a connection for entity "
+ context.getEntityAttribute(DataImporter.NAME) + " with URL: "
+ url);
long start = System.currentTimeMillis();
long start = System.nanoTime();
Connection c = null;
if (jndiName != null) {
@ -163,7 +165,7 @@ public class JdbcDataSource extends
}
}
LOG.info("Time taken for getConnection(): "
+ (System.currentTimeMillis() - start));
+ TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS));
return c;
}
@ -268,12 +270,12 @@ public class JdbcDataSource extends
stmt.setFetchSize(batchSize);
stmt.setMaxRows(maxRows);
LOG.debug("Executing SQL: " + query);
long start = System.currentTimeMillis();
long start = System.nanoTime();
if (stmt.execute(query)) {
resultSet = stmt.getResultSet();
}
LOG.trace("Time taken for sql :"
+ (System.currentTimeMillis() - start));
+ TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS));
colNames = readFieldNames(resultSet.getMetaData());
} catch (Exception e) {
wrapAndThrow(SEVERE, e, "Unable to execute query: " + query);
@ -386,12 +388,12 @@ public class JdbcDataSource extends
}
private Connection getConnection() throws Exception {
long currTime = System.currentTimeMillis();
long currTime = System.nanoTime();
if (currTime - connLastUsed > CONN_TIME_OUT) {
synchronized (this) {
Connection tmpConn = factory.call();
closeConnection();
connLastUsed = System.currentTimeMillis();
connLastUsed = System.nanoTime();
return conn = tmpConn;
}

View File

@ -201,10 +201,10 @@ class BatchWriter {
context.setStatus("Optimizing Solr");
int maxSegments = context.getConfiguration().getInt(SolrOutputFormat.SOLR_RECORD_WRITER_MAX_SEGMENTS, 1);
LOG.info("Optimizing Solr: forcing merge down to {} segments", maxSegments);
long start = System.currentTimeMillis();
long start = System.nanoTime();
solr.optimize(true, false, maxSegments);
context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_REDUCER_MERGE_TIME.toString()).increment(System.currentTimeMillis() - start);
float secs = (System.currentTimeMillis() - start) / 1000.0f;
context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_REDUCER_MERGE_TIME.toString()).increment(System.nanoTime() - start);
float secs = (System.nanoTime() - start) / (float)(10^9);
LOG.info("Optimizing Solr: done forcing merge down to {} segments in {} secs", maxSegments, secs);
context.setStatus("Committing Solr Phase 2");
solr.commit(true, false);

View File

@ -52,7 +52,7 @@ class GoLive {
public boolean goLive(Options options, FileStatus[] outDirs) {
LOG.info("Live merging of output shards into Solr cluster...");
boolean success = false;
long start = System.currentTimeMillis();
long start = System.nanoTime();
int concurrentMerges = options.goLiveThreads;
ThreadPoolExecutor executor = new ThreadPoolExecutor(concurrentMerges,
concurrentMerges, 1, TimeUnit.SECONDS,
@ -173,7 +173,7 @@ class GoLive {
return true;
} finally {
shutdownNowAndAwaitTermination(executor);
float secs = (System.currentTimeMillis() - start) / 1000.0f;
float secs = (System.nanoTime() - start) / (float)(10^9);
LOG.info("Live merging of index shards into Solr cluster took " + secs + " secs");
if (success) {
LOG.info("Live merging completed successfully");

View File

@ -40,6 +40,7 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit;
import net.sourceforge.argparse4j.ArgumentParsers;
import net.sourceforge.argparse4j.impl.Arguments;
@ -79,8 +80,8 @@ import org.apache.solr.hadoop.morphline.MorphlineMapRunner;
import org.apache.solr.hadoop.morphline.MorphlineMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.kitesdk.morphline.base.Fields;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.io.ByteStreams;
@ -613,7 +614,7 @@ public class MapReduceIndexerTool extends Configured implements Tool {
"which is required for passing files via --files and --libjars");
}
long programStartTime = System.currentTimeMillis();
long programStartTime = System.nanoTime();
if (options.fairSchedulerPool != null) {
getConf().set("mapred.fairscheduler.pool", options.fairSchedulerPool);
}
@ -684,7 +685,7 @@ public class MapReduceIndexerTool extends Configured implements Tool {
LOG.info("Randomizing list of {} input files to spread indexing load more evenly among mappers", numFiles);
long startTime = System.currentTimeMillis();
long startTime = System.nanoTime();
if (numFiles < job.getConfiguration().getInt(MAIN_MEMORY_RANDOMIZATION_THRESHOLD, 100001)) {
// If there are few input files reduce latency by directly running main memory randomization
// instead of launching a high latency MapReduce job
@ -698,7 +699,7 @@ public class MapReduceIndexerTool extends Configured implements Tool {
return -1; // job failed
}
}
float secs = (System.currentTimeMillis() - startTime) / 1000.0f;
float secs = (System.nanoTime() - startTime) / (float)(10^9);
LOG.info("Done. Randomizing list of {} input files took {} secs", numFiles, secs);
@ -765,9 +766,9 @@ public class MapReduceIndexerTool extends Configured implements Tool {
MorphlineMapRunner runner = setupMorphline(options);
if (options.isDryRun && runner != null) {
LOG.info("Indexing {} files in dryrun mode", numFiles);
startTime = System.currentTimeMillis();
startTime = System.nanoTime();
dryRun(runner, fs, fullInputList);
secs = (System.currentTimeMillis() - startTime) / 1000.0f;
secs = (System.nanoTime() - startTime) / (float)(10^9);
LOG.info("Done. Indexing {} files in dryrun mode took {} secs", numFiles, secs);
goodbye(null, programStartTime);
return 0;
@ -778,12 +779,12 @@ public class MapReduceIndexerTool extends Configured implements Tool {
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(SolrInputDocumentWritable.class);
LOG.info("Indexing {} files using {} real mappers into {} reducers", new Object[] {numFiles, realMappers, reducers});
startTime = System.currentTimeMillis();
startTime = System.nanoTime();
if (!waitForCompletion(job, options.isVerbose)) {
return -1; // job failed
}
secs = (System.currentTimeMillis() - startTime) / 1000.0f;
secs = (System.nanoTime() - startTime) / (float)(10^9);
LOG.info("Done. Indexing {} files using {} real mappers into {} reducers took {} secs", new Object[] {numFiles, realMappers, reducers, secs});
int mtreeMergeIterations = 0;
@ -816,14 +817,14 @@ public class MapReduceIndexerTool extends Configured implements Tool {
LOG.info("MTree merge iteration {}/{}: Merging {} shards into {} shards using fanout {}", new Object[] {
mtreeMergeIteration, mtreeMergeIterations, reducers, (reducers / options.fanout), options.fanout});
startTime = System.currentTimeMillis();
startTime = System.nanoTime();
if (!waitForCompletion(job, options.isVerbose)) {
return -1; // job failed
}
if (!renameTreeMergeShardDirs(outputTreeMergeStep, job, fs)) {
return -1;
}
secs = (System.currentTimeMillis() - startTime) / 1000.0f;
secs = (System.nanoTime() - startTime) / (float)(10^9);
LOG.info("MTree merge iteration {}/{}: Done. Merging {} shards into {} shards using fanout {} took {} secs",
new Object[] {mtreeMergeIteration, mtreeMergeIterations, reducers, (reducers / options.fanout), options.fanout, secs});
@ -1343,7 +1344,7 @@ public class MapReduceIndexerTool extends Configured implements Tool {
}
private void goodbye(Job job, long startTime) {
float secs = (System.currentTimeMillis() - startTime) / 1000.0f;
float secs = (System.nanoTime() - startTime) / (float)(10^9);
if (job != null) {
LOG.info("Succeeded with job: " + getJobInfo(job));
}

View File

@ -27,6 +27,7 @@ import java.util.List;
import java.util.Locale;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.filecache.DistributedCache;
@ -98,7 +99,7 @@ class SolrRecordWriter<K, V> extends RecordWriter<K, V> {
private final List<SolrInputDocument> batch;
private final int batchSize;
private long numDocsWritten = 0;
private long nextLogTime = System.currentTimeMillis();
private long nextLogTime = System.nanoTime();
private static HashMap<TaskID, Reducer<?,?,?,?>.Context> contextMap = new HashMap<TaskID, Reducer<?,?,?,?>.Context>();
@ -266,9 +267,9 @@ class SolrRecordWriter<K, V> extends RecordWriter<K, V> {
if (batch.size() >= batchSize) {
batchWriter.queueBatch(batch);
numDocsWritten += batch.size();
if (System.currentTimeMillis() >= nextLogTime) {
if (System.nanoTime() >= nextLogTime) {
LOG.info("docsWritten: {}", numDocsWritten);
nextLogTime += 10000;
nextLogTime += TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
}
batch.clear();
}

View File

@ -22,6 +22,7 @@ import java.io.OutputStreamWriter;
import java.io.Writer;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
@ -133,7 +134,7 @@ 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.currentTimeMillis();
long start = System.nanoTime();
writer.addIndexes(indexes);
// TODO: avoid intermediate copying of files into dst directory; rename the files into the dir instead (cp -> rename)
@ -143,12 +144,12 @@ public class TreeMergeOutputFormat extends FileOutputFormat<Text, NullWritable>
if (LOG.isDebugEnabled()) {
context.getCounter(SolrCounters.class.getName(), SolrCounters.LOGICAL_TREE_MERGE_TIME.toString()).increment(System.currentTimeMillis() - start);
}
float secs = (System.currentTimeMillis() - start) / 1000.0f;
float secs = (System.nanoTime() - start) / (float)(10^9);
LOG.info("Logical merge took {} secs", secs);
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.currentTimeMillis();
start = System.nanoTime();
if (maxSegments < Integer.MAX_VALUE) {
writer.forceMerge(maxSegments);
// TODO: consider perf enhancement for no-deletes merges: bulk-copy the postings data
@ -157,13 +158,13 @@ public class TreeMergeOutputFormat extends FileOutputFormat<Text, NullWritable>
if (LOG.isDebugEnabled()) {
context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_TREE_MERGE_TIME.toString()).increment(System.currentTimeMillis() - start);
}
secs = (System.currentTimeMillis() - start) / 1000.0f;
secs = (System.nanoTime() - start) / (float)(10^9);
LOG.info("Optimizing Solr: done forcing tree merge down to {} segments in {} secs", maxSegments, secs);
start = System.currentTimeMillis();
start = System.nanoTime();
LOG.info("Optimizing Solr: Closing index writer");
writer.close();
secs = (System.currentTimeMillis() - start) / 1000.0f;
secs = (System.nanoTime() - start) / (float)(10^9);
LOG.info("Optimizing Solr: Done closing index writer in {} secs", secs);
context.setStatus("Done");
} finally {

View File

@ -2,6 +2,7 @@ package org.apache.solr.cloud;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.solr.common.SolrException;
@ -74,7 +75,7 @@ public abstract class ElectionContext {
}
}
abstract void runLeaderProcess(boolean weAreReplacement, int pauseTime) throws KeeperException, InterruptedException, IOException;
abstract void runLeaderProcess(boolean weAreReplacement, int pauseBeforeStartMs) throws KeeperException, InterruptedException, IOException;
public void checkIfIamLeaderFired() {}
@ -357,9 +358,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
return false;
}
private void waitForReplicasToComeUp(boolean weAreReplacement,
int timeout) throws InterruptedException {
long timeoutAt = System.currentTimeMillis() + timeout;
private void waitForReplicasToComeUp(boolean weAreReplacement, int timeoutms) throws InterruptedException {
long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS);
final String shardsElectZkPath = electionPath + LeaderElector.ELECTION_NODE;
Slice slices = zkController.getClusterState().getSlice(collection, shardId);
@ -383,11 +383,11 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
if (cnt % 40 == 0) {
log.info("Waiting until we see more replicas up for shard " + shardId + ": total="
+ slices.getReplicasMap().size() + " found=" + found
+ " timeoutin=" + (timeoutAt - System.currentTimeMillis()));
+ " timeoutin=" + (timeoutAt - System.nanoTime() / (float)(10^9)) + "ms");
}
}
if (System.currentTimeMillis() > timeoutAt) {
if (System.nanoTime() > timeoutAt) {
log.info("Was waiting for replicas to come up, but they are taking too long - assuming they won't come back till later");
return;
}
@ -472,7 +472,7 @@ final class OverseerElectionContext extends ElectionContext {
}
@Override
void runLeaderProcess(boolean weAreReplacement, int pauseBeforeStart) throws KeeperException,
void runLeaderProcess(boolean weAreReplacement, int pauseBeforeStartMs) throws KeeperException,
InterruptedException {
log.info("I am going to be the leader {}", id);
final String id = leaderSeqPath
@ -481,10 +481,11 @@ final class OverseerElectionContext extends ElectionContext {
zkClient.makePath(leaderPath, ZkStateReader.toJSON(myProps),
CreateMode.EPHEMERAL, true);
if(pauseBeforeStart >0){
if(pauseBeforeStartMs >0){
try {
Thread.sleep(pauseBeforeStart);
Thread.sleep(pauseBeforeStartMs);
} catch (InterruptedException e) {
Thread.interrupted();
log.warn("Wait interrupted ", e);
}
}

View File

@ -17,6 +17,11 @@ package org.apache.solr.cloud;
* the License.
*/
import static java.util.Collections.singletonMap;
import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@ -26,6 +31,7 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClosableThread;
@ -46,11 +52,6 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Collections.singletonMap;
import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP;
/**
* Cluster leader. Responsible node assignments, cluster state file?
*/
@ -194,12 +195,12 @@ public class Overseer {
stateUpdateQueue.poll();
if (System.currentTimeMillis() - lastUpdatedTime > STATE_UPDATE_DELAY) break;
if (System.nanoTime() - lastUpdatedTime > TimeUnit.NANOSECONDS.convert(STATE_UPDATE_DELAY, TimeUnit.MILLISECONDS)) break;
// if an event comes in the next 100ms batch it together
head = stateUpdateQueue.peek(100);
}
lastUpdatedTime = System.currentTimeMillis();
lastUpdatedTime = System.nanoTime();
zkClient.setData(ZkStateReader.CLUSTER_STATE,
ZkStateReader.toJSON(clusterState), true);
// clean work queue

View File

@ -18,6 +18,7 @@ package org.apache.solr.cloud;
*/
import com.google.common.collect.ImmutableSet;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpSolrServer;
@ -81,6 +82,7 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static org.apache.solr.cloud.Assign.Node;
import static org.apache.solr.cloud.Assign.getNodesForNewShard;
@ -271,9 +273,9 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
//wait for a while to ensure the designate has indeed come in front
boolean prioritizationComplete = false;
long timeout = System.currentTimeMillis() + 2500;
long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(2500, TimeUnit.MILLISECONDS);
for(;System.currentTimeMillis()< timeout ;){
while (System.nanoTime() < timeout) {
List<String> currentNodeNames = getSortedNodeNames(zk);
int totalLeaders = 0;
@ -519,9 +521,9 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
}
private boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
long waitUntil = System.currentTimeMillis() + timeoutms;
long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS);
boolean deleted = false;
while (System.currentTimeMillis() < waitUntil) {
while (System.nanoTime() < waitUntil) {
Thread.sleep(100);
deleted = zkStateReader.getClusterState().getCollection(collectionName).getSlice(shard).getReplica(replicaName) == null;
if (deleted) break;
@ -565,10 +567,10 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
ZkStateReader.toJSON(m));
// wait for a while until we don't see the collection
long now = System.currentTimeMillis();
long timeout = now + 30000;
long now = System.nanoTime();
long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
boolean removed = false;
while (System.currentTimeMillis() < timeout) {
while (System.nanoTime() < timeout) {
Thread.sleep(100);
removed = !zkStateReader.getClusterState().hasCollection(message.getStr(collection));
if (removed) {
@ -637,11 +639,11 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
private void checkForAlias(String name, String value) {
long now = System.currentTimeMillis();
long timeout = now + 30000;
long now = System.nanoTime();
long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
boolean success = false;
Aliases aliases = null;
while (System.currentTimeMillis() < timeout) {
while (System.nanoTime() < timeout) {
aliases = zkStateReader.getAliases();
String collections = aliases.getCollectionAlias(name);
if (collections != null && collections.equals(value)) {
@ -656,11 +658,11 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
private void checkForAliasAbsence(String name) {
long now = System.currentTimeMillis();
long timeout = now + 30000;
long now = System.nanoTime();
long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
boolean success = false;
Aliases aliases = null;
while (System.currentTimeMillis() < timeout) {
while (System.nanoTime() < timeout) {
aliases = zkStateReader.getAliases();
String collections = aliases.getCollectionAlias(name);
if (collections == null) {
@ -719,9 +721,9 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(message));
// wait for a while until we see the shard
long waitUntil = System.currentTimeMillis() + 30000;
long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);;
boolean created = false;
while (System.currentTimeMillis() < waitUntil) {
while (System.nanoTime() < waitUntil) {
Thread.sleep(100);
created = zkStateReader.getClusterState().getCollection(collectionName).getSlice(shard) != null;
if (created) break;
@ -1254,10 +1256,10 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(m));
// wait for a while until we don't see the shard
long now = System.currentTimeMillis();
long timeout = now + 30000;
long now = System.nanoTime();
long timeout = now + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);;
boolean removed = false;
while (System.currentTimeMillis() < timeout) {
while (System.nanoTime() < timeout) {
Thread.sleep(100);
removed = zkStateReader.getClusterState().getSlice(collection, sliceId) == null;
if (removed) {
@ -1364,6 +1366,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
"routeKey", SolrIndexSplitter.getRouteKey(splitKey) + "!",
"range", splitRange.toString(),
"targetCollection", targetCollection.getName(),
// TODO: look at using nanoTime here?
"expireAt", String.valueOf(System.currentTimeMillis() + timeout));
log.info("Adding routing rule: " + m);
Overseer.getInQueue(zkStateReader.getZkClient()).offer(
@ -1371,9 +1374,9 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
// wait for a while until we see the new rule
log.info("Waiting to see routing rule updated in clusterstate");
long waitUntil = System.currentTimeMillis() + 60000;
long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
boolean added = false;
while (System.currentTimeMillis() < waitUntil) {
while (System.nanoTime() < waitUntil) {
Thread.sleep(100);
Map<String, RoutingRule> rules = zkStateReader.getClusterState().getSlice(sourceCollection.getName(), sourceSlice.getName()).getRoutingRules();
if (rules != null) {
@ -1614,9 +1617,9 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(message));
// wait for a while until we don't see the collection
long waitUntil = System.currentTimeMillis() + 30000;
long waitUntil = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
boolean created = false;
while (System.currentTimeMillis() < waitUntil) {
while (System.nanoTime() < waitUntil) {
Thread.sleep(100);
created = zkStateReader.getClusterState().getCollections().contains(message.getStr("name"));
if(created) break;
@ -1703,7 +1706,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
private Map<String, Replica> lookupReplicas(String collectionName, Collection<String> coreNames) throws InterruptedException {
Map<String, Replica> result = new HashMap<String, Replica>();
long endTime = System.currentTimeMillis() +3000;
long endTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS);
for(;;) {
DocCollection coll = zkStateReader.getClusterState().getCollection(collectionName);
for (String coreName : coreNames) {
@ -1721,7 +1724,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
if(result.size() == coreNames.size()) {
return result;
}
if( System.currentTimeMillis() > endTime) {
if( System.nanoTime() > endTime) {
//time up . throw exception and go out
throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to create replica entries in ZK");
}

View File

@ -35,6 +35,7 @@ import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.io.FileUtils;
@ -289,7 +290,7 @@ public final class ZkController {
try {
zkClient.delete("/overseer_elect/leader",-1, true);
log.info("Forcing me to be leader {} ", getBaseUrl());
overseerElector.getContext().runLeaderProcess(true, Overseer.STATE_UPDATE_DELAY+100);
overseerElector.getContext().runLeaderProcess(true, Overseer.STATE_UPDATE_DELAY + 100);
} catch (Exception e) {
throw new SolrException(ErrorCode.SERVER_ERROR, " Error becoming overseer ",e);
@ -601,10 +602,10 @@ public final class ZkController {
}
// now wait till the updates are in our state
long now = System.currentTimeMillis();
long timeout = now + 1000 * 30;
long now = System.nanoTime();
long timeout = now + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
boolean foundStates = false;
while (System.currentTimeMillis() < timeout) {
while (System.nanoTime() < timeout) {
clusterState = zkStateReader.getClusterState();
collections = clusterState.getCollections();
for (String collectionName : collections) {
@ -1392,9 +1393,9 @@ public final class ZkController {
String coreNodeName = cloudDesc.getCoreNodeName();
assert coreNodeName != null;
if (cloudDesc.getShardId() == null) throw new SolrException(ErrorCode.SERVER_ERROR ,"No shard id for :" + cd);
long endTime = System.currentTimeMillis()+3000;
long endTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS);
String errMessage= null;
for (; System.currentTimeMillis()<endTime; ) {
for (; System.nanoTime()<endTime; ) {
Thread.sleep(100);
errMessage = null;
Slice slice = zkStateReader.getClusterState().getSlice(cd.getCollectionName(), cloudDesc.getShardId());

View File

@ -25,6 +25,7 @@ import org.apache.solr.update.SolrIndexWriter;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
/**
@ -75,7 +76,7 @@ public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
* @param reserveTime time in milliseconds for which the commit point is to be reserved
*/
public void setReserveDuration(Long indexGen, long reserveTime) {
long timeToSet = System.currentTimeMillis() + reserveTime;
long timeToSet = System.nanoTime() + TimeUnit.NANOSECONDS.convert(reserveTime, TimeUnit.MILLISECONDS);
for(;;) {
Long previousTime = reserves.put(indexGen, timeToSet);
@ -92,7 +93,7 @@ public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
}
private void cleanReserves() {
long currentTime = System.currentTimeMillis();
long currentTime = System.nanoTime();
for (Map.Entry<Long, Long> entry : reserves.entrySet()) {
if (entry.getValue() < currentTime) {
reserves.remove(entry.getKey());
@ -175,7 +176,7 @@ public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
public void delete() {
Long gen = delegate.getGeneration();
Long reserve = reserves.get(gen);
if (reserve != null && System.currentTimeMillis() < reserve) return;
if (reserve != null && System.nanoTime() < reserve) return;
if(savedCommits.containsKey(gen)) return;
delegate.delete();
}

View File

@ -35,6 +35,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantLock;
import java.util.zip.Adler32;
@ -800,7 +801,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
private long getTimeElapsed(SnapPuller snapPuller) {
long timeElapsed = 0;
if (snapPuller.getReplicationStartTime() > 0)
timeElapsed = (System.currentTimeMillis() - snapPuller.getReplicationStartTime()) / 1000;
timeElapsed = TimeUnit.SECONDS.convert(System.currentTimeMillis() - snapPuller.getReplicationStartTime(), TimeUnit.MILLISECONDS);
return timeElapsed;
}

View File

@ -17,47 +17,6 @@ package org.apache.solr.handler.admin;
* limitations under the License.
*/
import com.google.common.collect.ImmutableSet;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpSolrServer;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestSyncShard;
import org.apache.solr.cloud.DistributedQueue.QueueEvent;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.OverseerCollectionProcessor;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
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.core.CoreContainer;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
import static org.apache.solr.cloud.OverseerCollectionProcessor.COLL_CONF;
import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATESHARD;
@ -75,6 +34,47 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.AD
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpSolrServer;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestSyncShard;
import org.apache.solr.cloud.DistributedQueue.QueueEvent;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.OverseerCollectionProcessor;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
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.core.CoreContainer;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.ImmutableSet;
public class CollectionsHandler extends RequestHandlerBase {
protected static Logger log = LoggerFactory.getLogger(CollectionsHandler.class);
protected final CoreContainer coreContainer;
@ -243,7 +243,7 @@ public class CollectionsHandler extends RequestHandlerBase {
private void handleResponse(String operation, ZkNodeProps m,
SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException {
long time = System.currentTimeMillis();
long time = System.nanoTime();
QueueEvent event = coreContainer.getZkController()
.getOverseerCollectionQueue()
.offer(ZkStateReader.toJSON(m), timeout);
@ -256,7 +256,7 @@ public class CollectionsHandler extends RequestHandlerBase {
rsp.setException(new SolrException(code != null && code != -1 ? ErrorCode.getErrorCode(code) : ErrorCode.SERVER_ERROR, (String)exp.get("msg")));
}
} else {
if (System.currentTimeMillis() - time >= timeout) {
if (System.nanoTime() - time >= TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS)) {
throw new SolrException(ErrorCode.SERVER_ERROR, operation
+ " the collection time out:" + timeout / 1000 + "s");
} else if (event.getWatchedEvent() != null) {

View File

@ -53,6 +53,7 @@ import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
public class HttpShardHandler extends ShardHandler {
@ -127,7 +128,7 @@ public class HttpShardHandler extends ShardHandler {
srsp.setShard(shard);
SimpleSolrResponse ssr = new SimpleSolrResponse();
srsp.setSolrResponse(ssr);
long startTime = System.currentTimeMillis();
long startTime = System.nanoTime();
try {
params.remove(CommonParams.WT); // use default (currently javabin)
@ -174,7 +175,7 @@ public class HttpShardHandler extends ShardHandler {
}
}
ssr.elapsedTime = System.currentTimeMillis() - startTime;
ssr.elapsedTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
return srsp;
}

View File

@ -29,6 +29,7 @@ import org.slf4j.LoggerFactory;
import org.xml.sax.InputSource;
import java.io.ByteArrayInputStream;
import java.util.concurrent.TimeUnit;
/** Keeps a ManagedIndexSchema up-to-date when changes are made to the serialized managed schema in ZooKeeper */
public class ZkIndexSchemaReader {
@ -95,12 +96,12 @@ public class ZkIndexSchemaReader {
byte[] data = zkClient.getData(managedSchemaPath, watcher, stat, true);
if (stat.getVersion() != oldSchema.schemaZkVersion) {
log.info("Retrieved schema from ZooKeeper");
long start = System.currentTimeMillis();
long start = System.nanoTime();
InputSource inputSource = new InputSource(new ByteArrayInputStream(data));
ManagedIndexSchema newSchema = oldSchema.reloadFields(inputSource, stat.getVersion());
managedIndexSchemaFactory.setSchema(newSchema);
long stop = System.currentTimeMillis();
log.info("Finished refreshing schema in " + (stop - start) + " ms");
long stop = System.nanoTime();
log.info("Finished refreshing schema in " + TimeUnit.MILLISECONDS.convert(stop - start, TimeUnit.NANOSECONDS) + " ms");
}
}
}

View File

@ -26,6 +26,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
/**
* SolrCache based on ConcurrentLRUCache implementation.
@ -143,7 +144,7 @@ public class FastLRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V> {
@Override
public void warm(SolrIndexSearcher searcher, SolrCache old) {
if (regenerator == null) return;
long warmingStartTime = System.currentTimeMillis();
long warmingStartTime = System.nanoTime();
FastLRUCache other = (FastLRUCache) old;
// warm entries
if (isAutowarmingOn()) {
@ -165,7 +166,7 @@ public class FastLRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V> {
}
}
}
warmupTime = System.currentTimeMillis() - warmingStartTime;
warmupTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - warmingStartTime, TimeUnit.NANOSECONDS);
}

View File

@ -28,6 +28,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
/**
* SolrCache based on ConcurrentLFUCache implementation.
@ -165,7 +166,7 @@ public class LFUCache<K, V> implements SolrCache<K, V> {
@Override
public void warm(SolrIndexSearcher searcher, SolrCache old) {
if (regenerator == null) return;
long warmingStartTime = System.currentTimeMillis();
long warmingStartTime = System.nanoTime();
LFUCache other = (LFUCache) old;
// warm entries
if (autowarmCount != 0) {
@ -187,7 +188,7 @@ public class LFUCache<K, V> implements SolrCache<K, V> {
}
}
}
warmupTime = System.currentTimeMillis() - warmingStartTime;
warmupTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - warmingStartTime, TimeUnit.NANOSECONDS);
}

View File

@ -31,6 +31,7 @@ import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.document.Document;
@ -2142,7 +2143,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
*/
public void warm(SolrIndexSearcher old) throws IOException {
// Make sure this is first! filters can help queryResults execute!
long warmingStartTime = System.currentTimeMillis();
long warmingStartTime = System.nanoTime();
// warm the caches in order...
ModifiableSolrParams params = new ModifiableSolrParams();
params.add("warming","true");
@ -2169,7 +2170,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
if (debug) log.debug("autowarming result for " + this + "\n\t" + this.cacheList[i]);
}
warmupTime = System.currentTimeMillis() - warmingStartTime;
warmupTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - warmingStartTime, TimeUnit.NANOSECONDS);
}
/**

View File

@ -411,6 +411,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()) {
List<DocRouter.Range> ranges = rule.getRouteRanges();
if (ranges != null && !ranges.isEmpty()) {

View File

@ -24,6 +24,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.UpdateParams;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
/**
@ -112,10 +113,11 @@ public abstract class AbstractUpdateRequest extends SolrRequest implements IsUpd
@Override
public UpdateResponse process( SolrServer server ) throws SolrServerException, IOException
{
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
UpdateResponse res = new UpdateResponse();
res.setResponse( server.request( this ) );
res.setElapsedTime( System.currentTimeMillis()-startTime );
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.client.solrj.request;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServer;
@ -266,10 +267,11 @@ public class CollectionAdminRequest extends SolrRequest
@Override
public CollectionAdminResponse process(SolrServer server) throws SolrServerException, IOException
{
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
CollectionAdminResponse res = new CollectionAdminResponse();
res.setResponse( server.request( this ) );
res.setElapsedTime( System.currentTimeMillis()-startTime );
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}

View File

@ -32,6 +32,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeUnit;
/**
* This class is experimental and subject to change.
@ -487,10 +488,11 @@ public class CoreAdminRequest extends SolrRequest
@Override
public CoreAdminResponse process(SolrServer server) throws SolrServerException, IOException
{
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
CoreAdminResponse res = new CoreAdminResponse();
res.setResponse( server.request( this ) );
res.setElapsedTime( System.currentTimeMillis()-startTime );
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.client.solrj.request;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
@ -63,10 +64,10 @@ public class DirectXmlRequest extends SolrRequest implements IsUpdateRequest
@Override
public UpdateResponse process( SolrServer server ) throws SolrServerException, IOException
{
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
UpdateResponse res = new UpdateResponse();
res.setResponse( server.request( this ) );
res.setElapsedTime( System.currentTimeMillis()-startTime );
res.setElapsedTime( TimeUnit.MILLISECONDS.convert(System.nanoTime()-startTime, TimeUnit.NANOSECONDS) );
return res;
}
}

View File

@ -32,6 +32,7 @@ import java.io.StringWriter;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeUnit;
/**
* A request for the org.apache.solr.handler.DocumentAnalysisRequestHandler.
@ -87,10 +88,11 @@ public class DocumentAnalysisRequest extends SolrRequest {
*/
@Override
public DocumentAnalysisResponse process(SolrServer server) throws SolrServerException, IOException {
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
DocumentAnalysisResponse res = new DocumentAnalysisResponse();
res.setResponse(server.request(this));
res.setElapsedTime(System.currentTimeMillis() - startTime);
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}

View File

@ -30,6 +30,7 @@ import java.io.IOException;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.TimeUnit;
/**
* A request for the org.apache.solr.handler.FieldAnalysisRequestHandler.
@ -102,10 +103,11 @@ public class FieldAnalysisRequest extends SolrRequest {
if (fieldValue == null) {
throw new IllegalStateException("The field value must be set");
}
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
FieldAnalysisResponse res = new FieldAnalysisResponse();
res.setResponse(server.request(this));
res.setElapsedTime(System.currentTimeMillis() - startTime);
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
@ -116,10 +117,11 @@ public class LukeRequest extends SolrRequest
@Override
public LukeResponse process( SolrServer server ) throws SolrServerException, IOException
{
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
LukeResponse res = new LukeResponse();
res.setResponse( server.request( this ) );
res.setElapsedTime( System.currentTimeMillis()-startTime );
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}
}

View File

@ -27,6 +27,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ContentStream;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
/**
*
@ -86,9 +87,10 @@ public class QueryRequest extends SolrRequest
public QueryResponse process( SolrServer server ) throws SolrServerException
{
try {
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
QueryResponse res = new QueryResponse( server.request( this ), server );
res.setElapsedTime( System.currentTimeMillis()-startTime );
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
} catch (SolrServerException e){
throw e;

View File

@ -19,6 +19,7 @@ package org.apache.solr.client.solrj.request;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServer;
@ -64,10 +65,11 @@ public class SolrPing extends SolrRequest {
@Override
public SolrPingResponse process(SolrServer server)
throws SolrServerException, IOException {
long startTime = System.currentTimeMillis();
long startTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
SolrPingResponse res = new SolrPingResponse();
res.setResponse(server.request(this));
res.setElapsedTime(System.currentTimeMillis() - startTime);
long endTime = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
res.setElapsedTime(endTime - startTime);
return res;
}

View File

@ -17,6 +17,7 @@ package org.apache.solr.common.cloud;
* limitations under the License.
*/
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.solr.common.SolrException;
@ -65,7 +66,7 @@ public class ConnectionManager implements Watcher {
public boolean isLikelyExpired(long timeToExpire) {
return stateType == StateType.EXPIRED
|| ( stateType == StateType.TRACKING_TIME && (System.currentTimeMillis() - lastDisconnectTime > timeToExpire));
|| ( stateType == StateType.TRACKING_TIME && (System.nanoTime() - lastDisconnectTime > TimeUnit.NANOSECONDS.convert(timeToExpire, TimeUnit.MILLISECONDS)));
}
}
@ -90,7 +91,7 @@ public class ConnectionManager implements Watcher {
connected = false;
// record the time we expired unless we are already likely expired
if (!likelyExpiredState.isLikelyExpired(0)) {
likelyExpiredState = new LikelyExpiredState(LikelyExpiredState.StateType.TRACKING_TIME, System.currentTimeMillis());
likelyExpiredState = new LikelyExpiredState(LikelyExpiredState.StateType.TRACKING_TIME, System.nanoTime());
}
notifyAll();
}
@ -204,7 +205,7 @@ public class ConnectionManager implements Watcher {
public synchronized void waitForConnected(long waitForConnection)
throws TimeoutException {
log.info("Waiting for client to connect to ZooKeeper");
long expire = System.currentTimeMillis() + waitForConnection;
long expire = System.nanoTime() + TimeUnit.NANOSECONDS.convert(waitForConnection, TimeUnit.MILLISECONDS);
long left = 1;
while (!connected && left > 0) {
if (isClosed) {
@ -216,7 +217,7 @@ public class ConnectionManager implements Watcher {
Thread.currentThread().interrupt();
break;
}
left = expire - System.currentTimeMillis();
left = expire - System.nanoTime();
}
if (!connected) {
throw new TimeoutException("Could not connect to ZooKeeper " + zkServerAddress + " within " + waitForConnection + " ms");
@ -226,11 +227,11 @@ public class ConnectionManager implements Watcher {
public synchronized void waitForDisconnected(long timeout)
throws InterruptedException, TimeoutException {
long expire = System.currentTimeMillis() + timeout;
long expire = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS);
long left = timeout;
while (connected && left > 0) {
wait(left);
left = expire - System.currentTimeMillis();
left = expire - System.nanoTime();
}
if (connected) {
throw new TimeoutException("Did not disconnect");

View File

@ -17,13 +17,12 @@ package org.apache.solr.common.cloud;
* limitations under the License.
*/
import org.apache.solr.common.SolrException;
import org.noggit.JSONUtil;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.noggit.JSONUtil;
/**
* Used for routing docs with particular keys into another collection
*/

View File

@ -522,8 +522,8 @@ public class ZkStateReader {
* Get shard leader properties, with retry if none exist.
*/
public Replica getLeaderRetry(String collection, String shard, int timeout) throws InterruptedException {
long timeoutAt = System.currentTimeMillis() + timeout;
while (System.currentTimeMillis() < timeoutAt && !closed) {
long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS);
while (System.nanoTime() < timeoutAt && !closed) {
if (clusterState != null) {
Replica replica = clusterState.getLeader(collection, shard);
if (replica != null && getClusterState().liveNodesContain(replica.getNodeName())) {