mirror of https://github.com/apache/lucene.git
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:
parent
1263b95738
commit
9441f0d4ce
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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())) {
|
||||
|
|
Loading…
Reference in New Issue