LUCENE-7788: fail precommit on unparameterised log messages and examine for wasted work/objects

This commit is contained in:
Erick Erickson 2020-04-29 10:56:54 -04:00
parent 28e747950f
commit 6e96d01efc
60 changed files with 631 additions and 366 deletions

View File

@ -35,9 +35,9 @@ allprojects {
if (project.hasProperty('srcDir')) {
srcDir.addAll(project.getProperty('srcDir').split(','))
} else { // Remove this later, make it optional
//nocommit
//TODO
throw new BuildException(String.format(Locale.ENGLISH,
'''Until we get all the calls cleaned up, you MUST specify -PsrcDir=relative_path, e.g.
'''Until we get all the calls cleaned up, you MUST specify -PsrcDir=relative_path, e.g.
"-PsrcDir=solr/core/src/java/org/apache/solr/core". This task will recursively check all
"*.java" files under that directory'''))
}
@ -63,66 +63,77 @@ class ValidateLogCallsTask extends DefaultTask {
@Input
boolean checkPlus
// nocommit, remove when you go to project-based checking.
Set<String> dirsToCheck = ["solr/core/src/java/org/apache/solr/analysis"
, "solr/core/src/java/org/apache/solr/api"
, "solr/core/src/java/org/apache/solr/client"
, "solr/core/src/java/org/apache/solr/cloud" // 120
, "solr/core/src/java/org/apache/solr/cloud/api"
, "solr/core/src/java/org/apache/solr/cloud/autoscaling"
, "solr/core/src/java/org/apache/solr/cloud/cdcr"
, "solr/core/src/java/org/apache/solr/cloud/hdfs"
, "solr/core/src/java/org/apache/solr/cloud/overseer"
, "solr/core/src/java/org/apache/solr/cloud/rule"
, "solr/core/src/java/org/apache/solr/core"
, "solr/core/src/java/org/apache/solr/filestore"
, "solr/core/src/java/org/apache/solr/handler/admin"
, "solr/core/src/java/org/apache/solr/handler/component"
, "solr/core/src/java/org/apache/solr/handler/export"
, "solr/core/src/java/org/apache/solr/handler/loader"
, "solr/core/src/java/org/apache/solr/handler/tagger"
, "solr/core/src/java/org/apache/solr/highlight"
, "solr/core/src/java/org/apache/solr/index"
, "solr/core/src/java/org/apache/solr/internal"
, "solr/core/src/java/org/apache/solr/legacy"
, "solr/core/src/java/org/apache/solr/logging"
, "solr/core/src/java/org/apache/solr/metrics"
, "solr/core/src/java/org/apache/solr/packagemanager"
, "solr/core/src/java/org/apache/solr/parser"
, "solr/core/src/java/org/apache/solr/pkg"
, "solr/core/src/java/org/apache/solr/query"
, "solr/core/src/java/org/apache/solr/request"
, "solr/core/src/java/org/apache/solr/response"
, "solr/core/src/java/org/apache/solr/rest"
, "solr/core/src/java/org/apache/solr/schema"
, "solr/core/src/java/org/apache/solr/search"
, "solr/core/src/java/org/apache/solr/security"
, "solr/core/src/java/org/apache/solr/servlet"
, "solr/core/src/java/org/apache/solr/spelling"
, "solr/core/src/java/org/apache/solr/store"
, "solr/core/src/java/org/apache/solr/uninverting"
, "solr/core/src/java/org/apache/solr/update"
, "solr/core/src/java/org/apache/solr/util"
, "solr/solrj/src/java/org/apache/solr/common/util"
, "solr/solrj/src/java/org/apache/solr/common/cloud" // 18
, "solr/solrj/src/java/org/apache/solr/client/solrj/impl" // 27
, "solr/solrj/src/java/org/apache/solr/client/solrj/io" // 8
, "solr/solrj/src/java/org/apache/solr/client/solrj/cloud" // 26
, "solr/solrj/src/java/org/apache/solr/client/solrj/routing" //2
, "solr/solrj/src/test/org/apache/solr/common/cloud" //1
, "solr/solrj/src/test/org/apache/solr/client/solrj/impl" //15
, "solr/solrj/src/test/org/apache/solr/client/solrj" //34
, "solr/solrj/src/test/org/apache/solr/client/solrj/io/stream" // 1
, "solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling" //4
// TODO, remove when you go to project-based checking.
Set<String> dirsToCheck = [
"solr/core/src/java/org/apache/solr/analysis"
, "solr/core/src/java/org/apache/solr/api"
, "solr/core/src/java/org/apache/solr/client"
, "solr/core/src/java/org/apache/solr/cloud" // 120
, "solr/core/src/java/org/apache/solr/cloud/api"
, "solr/core/src/java/org/apache/solr/cloud/autoscaling"
, "solr/core/src/java/org/apache/solr/cloud/cdcr"
, "solr/core/src/java/org/apache/solr/cloud/hdfs"
, "solr/core/src/java/org/apache/solr/cloud/overseer"
, "solr/core/src/java/org/apache/solr/cloud/rule"
, "solr/core/src/java/org/apache/solr/core"
, "solr/core/src/java/org/apache/solr/filestore"
, "solr/core/src/java/org/apache/solr/handler/admin"
, "solr/core/src/java/org/apache/solr/handler/component"
, "solr/core/src/java/org/apache/solr/handler/export"
, "solr/core/src/java/org/apache/solr/handler/loader"
, "solr/core/src/java/org/apache/solr/handler/tagger"
, "solr/core/src/java/org/apache/solr/highlight"
, "solr/core/src/java/org/apache/solr/index"
, "solr/core/src/java/org/apache/solr/internal"
, "solr/core/src/java/org/apache/solr/legacy"
, "solr/core/src/java/org/apache/solr/logging"
, "solr/core/src/java/org/apache/solr/metrics"
, "solr/core/src/java/org/apache/solr/packagemanager"
, "solr/core/src/java/org/apache/solr/parser"
, "solr/core/src/java/org/apache/solr/pkg"
, "solr/core/src/java/org/apache/solr/query"
, "solr/core/src/java/org/apache/solr/request"
, "solr/core/src/java/org/apache/solr/response"
, "solr/core/src/java/org/apache/solr/rest"
, "solr/core/src/java/org/apache/solr/schema"
, "solr/core/src/java/org/apache/solr/search"
, "solr/core/src/java/org/apache/solr/security"
, "solr/core/src/java/org/apache/solr/servlet"
, "solr/core/src/java/org/apache/solr/spelling"
, "solr/core/src/java/org/apache/solr/store"
, "solr/core/src/java/org/apache/solr/uninverting"
, "solr/core/src/java/org/apache/solr/update"
, "solr/core/src/java/org/apache/solr/util"
, "solr/solrj"
, "solr/core/src/java/org/apache/solr/handler"
, "solr/core/src/test/org/apache/solr/cloud/api"
, "solr/core/src/test/org/apache/solr/cloud/autoscaling"
// , "solr/core/src/test/org/apache/solr/cloud"
// , "solr/core/src/test/org/apache/solr/cloud/cdcr"
// , "solr/core/src/test/org/apache/solr/handler"
// , "solr/core/src/test/org/apache/solr/metrics"
// , "solr/core/src/test/org/apache/solr/request"
// , "solr/core/src/test/org/apache/solr/response"
// , "solr/core/src/test/org/apache/solr/schema"
// , "solr/core/src/test/org/apache/solr/search"
// , "solr/core/src/test/org/apache/solr/security"
// , "solr/core/src/test/org/apache/solr/spelling"
// , "solr/core/src/test/org/apache/solr"
// , "solr/core/src/test/org/apache/solr/update"
// , "solr/core/src/test/org/apache/solr/util"
// , "solr/core/src/test"
// , "solr/core"
]
//nocommit REMOVE ME! Really! and the check for bare parens. Several times I've put in () when I meant {} and only
//TODO REMOVE ME! Really! and the check for bare parens. Several times I've put in () when I meant {} and only
// caught it by chance. So for this mass edit, I created a check with a a lot of false positives. This is a list of
// them and we won't report them.
Map<String, List<Integer>> parenHack = [
"AddReplicaCmd.java" : [99]
, "Assign.java" : [329]
, "CloudSolrClientTest.java" : [1083]
, "CommitTracker.java" : [135]
, "DeleteReplicaCmd.java" : [75]
, "DirectUpdateHandler2.java" : [838, 859]
@ -133,6 +144,9 @@ class ValidateLogCallsTask extends DefaultTask {
, "SliceMutator.java" : [61]
, "SolrDispatchFilter.java" : [150, 205, 242]
, "Suggester.java" : [147, 181]
, "TestSimTriggerIntegration.java" : [710, 713]
, "TestSolrJErrorHandling.java" : [289]
, "TriggerIntegrationTest.java" : [427, 430]
, "UpdateLog.java" : [1976]
, "V2HttpCall.java" : [158]
// checking against 8x in master, take these out usually.
@ -155,7 +169,7 @@ class ValidateLogCallsTask extends DefaultTask {
}
// We have a log.something line, check for patterns we're not fond of.
def checkLogLine(File file, String line, int lineNumber, String previous) {
def checkLine(File file, String line, int lineNumber, String previous) {
boolean violation = false
@ -184,15 +198,27 @@ class ValidateLogCallsTask extends DefaultTask {
.replaceFirst(/.*?\(/, " ") // Get rid of "log.info("
.replaceFirst(/\);/, " ") // get rid of the closing ");"
.replaceFirst("/\\*.*?\\*/", " ") // replace embedded comments "/*....*/"
.replaceAll(/".*?"/, " ") // remove anything between quotes. This is a bit fragile if there are embedded double quotes.
.replaceAll(/".*?"/, '""') // remove anything between quotes. This is a bit fragile if there are embedded double quotes.
.replaceAll(/timeLeft\(.*?\)/, " ") // used all over tests, it's benign
.replaceAll(/TimeUnit\..*?\.convert\(.*?\)/, " ") // again, a pattern that's efficient
.replaceAll("\\s", "")
def m = stripped =~ "\\(.*?\\)"
def hasParens = m.find()
def hasPlus = stripped.contains("+")
// The compiler will pre-assemble patterns like 'log.info("string const1 {}" + " string const2 {}", obj1, obj2)'
// to log.info("string const1 {} string const2 {}", obj1, obj2)', so don't worry about any plus preceeded and
// followed by double quotes.
def hasPlus = false
for (int idx = 0; idx < stripped.length(); ++idx) {
if (stripped.charAt(idx) == '+') {
if (idx == 0 || idx == stripped.length() - 1
|| stripped.charAt(idx - 1) != '"' || stripped.charAt(idx + 1) != '"') {
hasPlus = true
break
}
}
}
// Check that previous line isn't an if statement for always-reported log levels. Arbitrary decision: we don't
// really care about checking for awkward constructions for WARN and above, so report a violation if the previous
// line contains an if for those levels.
@ -222,13 +248,16 @@ class ValidateLogCallsTask extends DefaultTask {
violation = true
}
}
// Always report toString(). Note, this over-reports constructs like Arrays.toString(something), but just add //logOK.
// Always report toString(). Note, this over-reports some constructs
// but just add //logOK if it's really OK.
if (line.contains("toString(") == true) {
violation = true
if (line.replaceAll(/Arrays.toString\(/, "").contains("toString(") && prevLineNotIf) {
violation = true
}
}
if (violation) {
reportViolation(String.format("Suspicious logging call, Parameterize and possibly surround with 'if (log.is*Enabled) {..}'. Help at: 'gradlew helpValidateLogCalls' %s %s:%d"
, System.lineSeparator, file.getAbsolutePath(), lineNumber))
, System.lineSeparator, file.getAbsolutePath(), lineNumber))
}
return
}
@ -259,7 +288,6 @@ class ValidateLogCallsTask extends DefaultTask {
int state = 0 // 0 == not collecting a log line, 1 == collecting a log line, 2 == just collected the last.
int lineNumber = 0
StringBuilder sb = new StringBuilder();
boolean foundViolation = false
// We have to assemble line-by-line to get the full log statement. We require that all loggers (except requestLog
// and slowLog) be exactly "log". That will be checked as well.
@ -300,7 +328,7 @@ class ValidateLogCallsTask extends DefaultTask {
case 1:
break;
case 2:
checkLogLine(file, sb.toString(), lineNumber, prevLine)
checkLine(file, sb.toString(), lineNumber, prevLine)
state = 0
break;
default:
@ -315,7 +343,7 @@ class ValidateLogCallsTask extends DefaultTask {
// println srcDir
dirsToCheck.addAll(srcDir)
//nocommit. This is here to check 8x on another branch since I can't run Gradle
//TODO. This is here to check 8x on another branch since I can't run Gradle
// over 8x. Used periodically as a sanity check.
// new File("/Users/Erick/apache/solrJiras/master").traverse(type: groovy.io.FileType.FILES, nameFilter: ~/.*\.java/) { File it ->
// if (dirsToCheck.any { dir ->
@ -323,7 +351,7 @@ class ValidateLogCallsTask extends DefaultTask {
// }) {
// if (checkFile(it)) {
// println(it.getAbsolutePath())
// // nocommit. This just makes it much easier to get to the files during this mass migration!
// // TODO. This just makes it much easier to get to the files during this mass migration!
// }
// }
@ -333,13 +361,13 @@ class ValidateLogCallsTask extends DefaultTask {
// }
// }
//
//nocommit
//TODO
// println project
// This is the real stuff
project.sourceSets.each { srcSet ->
srcSet.java.each { f ->
if (srcDir.contains("all")) {
if (srcDir.contains("all")) { // TODO
checkFile(f)
} else if (dirsToCheck.any {
f.getCanonicalPath().contains(it)
@ -351,7 +379,7 @@ class ValidateLogCallsTask extends DefaultTask {
if (errsFound > 0) {
throw new BuildException(String.format(Locale.ENGLISH, 'Found %d violations in source files (%s).',
errsFound, violations.join(', ')));
errsFound, violations.join(', ')));
}
}
}

View File

@ -13,8 +13,9 @@ Recorder. It's particularly egregious when complex operations
are performed for, say, log.debug or trace calls, which
are rarely actually used.
- log.info("some stuff " + "some other stuff") will concatenate the
strings and create an object
- log.info("some stuff " + some_object) will concatenate the
strings and create an object. some_object is anytying except
a string constant, i.e. something enclosed in quotes.
- log.info("some stuff {}", object.method()) will execute the
method.
@ -49,6 +50,11 @@ NOTES:
checks. Adding //logok, with or without spaces will cause the line to pass
no matter what. Please use this hack sparingly.
- String constants may be added with '+' OK, but they _must_ be literal strings.
The compiler is smart enough to concatenate them when compiling. For example:
log.info("some {} "+ "more nonsense {}", object1, object2) is fine.
log.info("some{} " + " more nonsense " + object1, object2) is NOT fine.
For a fuller discussion, see LUCENE-7788 and the other JIRAs linked
from there.

View File

@ -124,7 +124,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
"q", "md5:" + md5,
"fl", "id,size,version,timestamp,blobName")),
rsp);
log.warn("duplicate entry for blob :" + blobName);
log.warn("duplicate entry for blob : {}", blobName);
return;
}
@ -149,9 +149,13 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
"size", payload.limit(),
"blob", payload);
verifyWithRealtimeGet(blobName, version, req, doc);
log.info(StrUtils.formatString("inserting new blob {0} ,size {1}, md5 {2}", doc.get(ID), String.valueOf(payload.limit()), md5));
if (log.isInfoEnabled()) {
log.info(StrUtils.formatString("inserting new blob {0} ,size {1}, md5 {2}", doc.get(ID), String.valueOf(payload.limit()), md5));
}
indexMap(req, rsp, doc);
log.info(" Successfully Added and committed a blob with id {} and size {} ", id, payload.limit());
if (log.isInfoEnabled()) {
log.info(" Successfully Added and committed a blob with id {} and size {} ", id, payload.limit());
}
break;
}
@ -244,9 +248,9 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
try (UpdateRequestProcessor processor = processorChain.createProcessor(req, rsp)) {
AddUpdateCommand cmd = new AddUpdateCommand(req);
cmd.solrDoc = solrDoc;
log.info("Adding doc: " + doc);
log.info("Adding doc: {}", doc);
processor.processAdd(cmd);
log.info("committing doc: " + doc);
log.info("committing doc: {}", doc);
processor.processCommit(new CommitUpdateCommand(req, false));
processor.finish();
}

View File

@ -121,7 +121,9 @@ class CdcrBufferStateManager extends CdcrStateManager {
zkClient.makePath(this.getZnodeBase(), null, CreateMode.PERSISTENT, null, false, true); // Should be a no-op if node exists
}
zkClient.create(this.getZnodePath(), DEFAULT_STATE.getBytes(), CreateMode.PERSISTENT, true);
log.info("Created znode {}", this.getZnodePath());
if (log.isInfoEnabled()) {
log.info("Created znode {}", this.getZnodePath());
}
}
} catch (KeeperException.NodeExistsException ne) {
// Someone got in first and created the node.
@ -164,7 +166,7 @@ class CdcrBufferStateManager extends CdcrStateManager {
log.info("Received new CDCR buffer state from watcher: {} @ {}:{}", state, collectionName, shard);
CdcrBufferStateManager.this.setState(state);
} catch (KeeperException | InterruptedException e) {
log.warn("Failed synchronising new state @ " + collectionName + ":" + shard, e);
log.warn("Failed synchronising new state @ {}:{}", collectionName, shard, e);
}
}

View File

@ -150,7 +150,7 @@ class CdcrLeaderStateManager extends CdcrStateManager {
CdcrLeaderStateManager.this.checkIfIAmLeader();
}
} catch (KeeperException | InterruptedException e) {
log.warn("Failed updating leader state and setting watch @ " + collectionName + ":" + shard, e);
log.warn("Failed updating leader state and setting watch @ {}: {}", collectionName, shard, e);
}
}

View File

@ -121,7 +121,9 @@ class CdcrProcessStateManager extends CdcrStateManager {
zkClient.makePath(this.getZnodeBase(), null, CreateMode.PERSISTENT, null, false, true);
}
zkClient.create(this.getZnodePath(), DEFAULT_STATE.getBytes(), CreateMode.PERSISTENT, true);
log.info("Created znode {}", this.getZnodePath());
if (log.isInfoEnabled()) {
log.info("Created znode {}", this.getZnodePath());
}
}
} catch (KeeperException.NodeExistsException ne) {
// Someone got in first and created the node.
@ -164,7 +166,7 @@ class CdcrProcessStateManager extends CdcrStateManager {
log.info("Received new CDCR process state from watcher: {} @ {}:{}", state, collectionName, shard);
CdcrProcessStateManager.this.setState(state);
} catch (KeeperException | InterruptedException e) {
log.warn("Failed synchronising new state @ " + collectionName + ":" + shard, e);
log.warn("Failed synchronising new state @ {}: {}", collectionName, shard, e);
}
}

View File

@ -124,7 +124,9 @@ public class CdcrReplicator implements Runnable {
// we might have read a single commit operation and reached the end of the update logs
logReader.forwardSeek(subReader);
log.info("Forwarded {} updates to target {}", counter, state.getTargetCollection());
if (log.isInfoEnabled()) {
log.info("Forwarded {} updates to target {}", counter, state.getTargetCollection());
}
} catch (Exception e) {
// report error and update error stats
this.handleException(e);
@ -182,10 +184,10 @@ public class CdcrReplicator implements Runnable {
log.warn("Failed to forward update request {} to target: {}. Got response {}", req, state.getTargetCollection(), rsp);
state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
} else if (e instanceof CloudSolrClient.RouteException) {
log.warn("Failed to forward update request to target: " + state.getTargetCollection(), e);
log.warn("Failed to forward update request to target: {}", state.getTargetCollection(), e);
state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
} else {
log.warn("Failed to forward update request to target: " + state.getTargetCollection(), e);
log.warn("Failed to forward update request to target: {}", state.getTargetCollection(), e);
state.reportError(CdcrReplicatorState.ErrorType.INTERNAL);
}
}

View File

@ -166,8 +166,10 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
state.closeLogReader();
try {
long checkpoint = this.getCheckpoint(state);
log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
checkpoint, collectionName, shard);
if (log.isInfoEnabled()) {
log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
checkpoint, collectionName, shard);
}
CdcrUpdateLog.CdcrLogReader reader = ulog.newLogReader();
boolean seek = reader.seek(checkpoint);
state.init(reader);
@ -187,9 +189,9 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
}
}
} catch (IOException | SolrServerException | SolrException e) {
log.warn("Unable to instantiate the log reader for target collection " + state.getTargetCollection(), e);
log.warn("Unable to instantiate the log reader for target collection {}", state.getTargetCollection(), e);
} catch (InterruptedException e) {
log.warn("Thread interrupted while instantiate the log reader for target collection " + state.getTargetCollection(), e);
log.warn("Thread interrupted while instantiate the log reader for target collection {}", state.getTargetCollection(), e);
Thread.currentThread().interrupt();
}
}
@ -295,8 +297,10 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
} else if (status == BootstrapStatus.COMPLETED) {
log.info("CDCR bootstrap successful in {} seconds", BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
long checkpoint = CdcrReplicatorManager.this.getCheckpoint(state);
log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
checkpoint, collectionName, shard);
if (log.isInfoEnabled()) {
log.info("Create new update log reader for target {} with checkpoint {} @ {}:{}", state.getTargetCollection(),
checkpoint, collectionName, shard);
}
CdcrUpdateLog.CdcrLogReader reader1 = ulog.newLogReader();
reader1.seek(checkpoint);
success = true;
@ -316,8 +320,11 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
retries++;
}
} else if (status == BootstrapStatus.NOTFOUND || status == BootstrapStatus.CANCELLED) {
log.info("CDCR bootstrap " + (status == BootstrapStatus.NOTFOUND ? "not found" : "cancelled") + "in {} seconds",
BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
if (log.isInfoEnabled()) {
log.info("CDCR bootstrap {} in {} seconds"
, (status == BootstrapStatus.NOTFOUND ? "not found" : "cancelled")
, BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
}
// the leader of the target shard may have changed and therefore there is no record of the
// bootstrap process so we must retry the operation
while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED) {
@ -326,8 +333,10 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
retries = 1;
timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
} else if (status == BootstrapStatus.UNKNOWN || status == BootstrapStatus.SUBMITTED) {
log.info("CDCR bootstrap is " + (status == BootstrapStatus.UNKNOWN ? "unknown" : "submitted"),
BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
if (log.isInfoEnabled()) {
log.info("CDCR bootstrap is {} {}", (status == BootstrapStatus.UNKNOWN ? "unknown" : "submitted"),
BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
}
// we were not able to query the status on the remote end
// so just sleep for a bit and try again
timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
@ -338,7 +347,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
state.reportError(CdcrReplicatorState.ErrorType.INTERNAL);
Thread.currentThread().interrupt();
} catch (IOException | SolrServerException | SolrException e) {
log.error("Unable to bootstrap the target collection " + targetCollection + " shard: " + shard, e);
log.error("Unable to bootstrap the target collection {} shard: {}", targetCollection, shard, e);
state.reportError(CdcrReplicatorState.ErrorType.BAD_REQUEST);
} finally {
if (success) {

View File

@ -80,7 +80,9 @@ class CdcrReplicatorScheduler {
if (!state.isBootstrapInProgress()) {
new CdcrReplicator(state, batchSize).run();
} else {
log.debug("Replicator state is bootstrapping, skipping replication for target collection {}", state.getTargetCollection());
if (log.isDebugEnabled()) {
log.debug("Replicator state is bootstrapping, skipping replication for target collection {}", state.getTargetCollection());
}
}
} finally {
statesQueue.offer(state);

View File

@ -146,8 +146,10 @@ class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
try {
NamedList response = server.request(request);
lastVersion = (Long) response.get(CdcrParams.LAST_PROCESSED_VERSION);
log.debug("My leader {} says its last processed _version_ number is: {}. I am {}", leaderUrl, lastVersion,
core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
if (log.isDebugEnabled()) {
log.debug("My leader {} says its last processed _version_ number is: {}. I am {}", leaderUrl, lastVersion,
core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
}
} catch (IOException | SolrServerException e) {
log.warn("Couldn't get last processed version from leader {}: {}", leaderUrl, e.getMessage());
return;

View File

@ -104,7 +104,7 @@ public class DocumentAnalysisRequestHandler extends AnalysisRequestHandlerBase {
} catch (IllegalArgumentException ex) {
// Other implementations will likely throw this exception since "reuse-instance"
// is implementation specific.
log.debug("Unable to set the 'reuse-instance' property for the input factory: " + inputFactory);
log.debug("Unable to set the 'reuse-instance' property for the input factory: {}", inputFactory);
}
}
@ -304,7 +304,7 @@ public class DocumentAnalysisRequestHandler extends AnalysisRequestHandlerBase {
text.setLength(0);
String localName = reader.getLocalName();
if (!"field".equals(localName)) {
log.warn("unexpected XML tag doc/" + localName);
log.warn("unexpected XML tag doc/{}", localName);
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "unexpected XML tag doc/" + localName);
}

View File

@ -246,7 +246,7 @@ public class IndexFetcher {
"'masterUrl' is required for a slave");
if (masterUrl != null && masterUrl.endsWith(ReplicationHandler.PATH)) {
masterUrl = masterUrl.substring(0, masterUrl.length()-12);
log.warn("'masterUrl' must be specified without the "+ReplicationHandler.PATH+" suffix");
log.warn("'masterUrl' must be specified without the {} suffix", ReplicationHandler.PATH);
}
this.masterUrl = masterUrl;
@ -331,7 +331,7 @@ public class IndexFetcher {
filesToDownload = Collections.synchronizedList(files);
else {
filesToDownload = Collections.emptyList();
log.error("No files to download for index generation: "+ gen);
log.error("No files to download for index generation: {}", gen);
}
files = (List<Map<String,Object>>) response.get(CONF_FILES);
@ -390,11 +390,15 @@ public class IndexFetcher {
return IndexFetchResult.EXPECTING_NON_LEADER;
}
if (replica.getState() != Replica.State.ACTIVE) {
log.info("Replica {} is leader but it's state is {}, skipping replication", replica.getName(), replica.getState());
if (log.isInfoEnabled()) {
log.info("Replica {} is leader but it's state is {}, skipping replication", replica.getName(), replica.getState());
}
return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
}
if (!solrCore.getCoreContainer().getZkController().getClusterState().liveNodesContain(replica.getNodeName())) {
log.info("Replica {} is leader but it's not hosted on a live node, skipping replication", replica.getName());
if (log.isInfoEnabled()) {
log.info("Replica {} is leader but it's not hosted on a live node, skipping replication", replica.getName());
}
return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
}
if (!replica.getCoreUrl().equals(masterUrl)) {
@ -412,10 +416,10 @@ public class IndexFetcher {
} catch (Exception e) {
final String errorMsg = e.toString();
if (!Strings.isNullOrEmpty(errorMsg) && errorMsg.contains(INTERRUPT_RESPONSE_MESSAGE)) {
log.warn("Master at: " + masterUrl + " is not available. Index fetch failed by interrupt. Exception: " + errorMsg);
log.warn("Master at: {} is not available. Index fetch failed by interrupt. Exception: {}", masterUrl, errorMsg);
return new IndexFetchResult(IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE, false, e);
} else {
log.warn("Master at: " + masterUrl + " is not available. Index fetch failed by exception: " + errorMsg);
log.warn("Master at: {} is not available. Index fetch failed by exception: {}", masterUrl, errorMsg);
return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e);
}
}
@ -423,8 +427,8 @@ public class IndexFetcher {
long latestVersion = (Long) response.get(CMD_INDEX_VERSION);
long latestGeneration = (Long) response.get(GENERATION);
log.info("Master's generation: " + latestGeneration);
log.info("Master's version: " + latestVersion);
log.info("Master's generation: {}", latestGeneration);
log.info("Master's version: {}", latestVersion);
// TODO: make sure that getLatestCommit only returns commit points for the main index (i.e. no side-car indexes)
IndexCommit commit = solrCore.getDeletionPolicy().getLatestCommit();
@ -444,8 +448,10 @@ public class IndexFetcher {
}
}
log.info("Slave's generation: " + commit.getGeneration());
log.info("Slave's version: " + IndexDeletionPolicyWrapper.getCommitTimestamp(commit));
if (log.isInfoEnabled()) {
log.info("Slave's generation: {}", commit.getGeneration());
log.info("Slave's version: {}", IndexDeletionPolicyWrapper.getCommitTimestamp(commit)); // logOK
}
if (latestVersion == 0L) {
if (commit.getGeneration() != 0) {
@ -487,9 +493,13 @@ public class IndexFetcher {
if (filesToDownload.isEmpty()) {
return IndexFetchResult.PEER_INDEX_COMMIT_DELETED;
}
log.info("Number of files in latest index in master: " + filesToDownload.size());
if (log.isInfoEnabled()) {
log.info("Number of files in latest index in master: {}", filesToDownload.size());
}
if (tlogFilesToDownload != null) {
log.info("Number of tlog files in master: " + tlogFilesToDownload.size());
if (log.isInfoEnabled()) {
log.info("Number of tlog files in master: {}", tlogFilesToDownload.size());
}
}
// Create the sync service
@ -555,7 +565,7 @@ public class IndexFetcher {
}
}
if (c > 0) {
log.info("IndexFetcher slept for " + (c * 1000) + "ms for unused lucene index files to be delete-able");
log.info("IndexFetcher slept for {}ms for unused lucene index files to be delete-able", c * 1000);
}
} finally {
writer.decref();
@ -642,7 +652,9 @@ public class IndexFetcher {
// we must reload the core after we open the IW back up
if (successfulInstall && (reloadCore || forceCoreReload)) {
log.info("Reloading SolrCore {}", solrCore.getName());
if (log.isInfoEnabled()) {
log.info("Reloading SolrCore {}", solrCore.getName());
}
reloadCore();
}
@ -651,7 +663,7 @@ public class IndexFetcher {
// let the system know we are changing dir's and the old one
// may be closed
if (indexDir != null) {
log.info("removing old index directory " + indexDir);
log.info("removing old index directory {}", indexDir);
solrCore.getDirectoryFactory().doneWithDirectory(indexDir);
solrCore.getDirectoryFactory().remove(indexDir);
}
@ -763,7 +775,7 @@ public class IndexFetcher {
String[] allFiles = indexDir.listAll();
for (String file : allFiles) {
if (!file.equals(segmentsFileName) && !currentFiles.contains(file) && !file.endsWith(".lock")) {
log.info("Found unused file: " + file);
log.info("Found unused file: {}", file);
return true;
}
}
@ -947,7 +959,7 @@ public class IndexFetcher {
}
private void downloadConfFiles(List<Map<String, Object>> confFilesToDownload, long latestGeneration) throws Exception {
log.info("Starting download of configuration files from master: " + confFilesToDownload);
log.info("Starting download of configuration files from master: {}", confFilesToDownload);
confFilesDownloaded = Collections.synchronizedList(new ArrayList<>());
File tmpconfDir = new File(solrCore.getResourceLoader().getConfigDir(), "conf." + getDateAsStr(new Date()));
try {
@ -976,7 +988,7 @@ public class IndexFetcher {
* Download all the tlog files to the temp tlog directory.
*/
private long downloadTlogFiles(File tmpTlogDir, long latestGeneration) throws Exception {
log.info("Starting download of tlog files from master: " + tlogFilesToDownload);
log.info("Starting download of tlog files from master: {}", tlogFilesToDownload);
tlogFilesDownloaded = Collections.synchronizedList(new ArrayList<>());
long bytesDownloaded = 0;
@ -1011,7 +1023,7 @@ public class IndexFetcher {
String indexDirPath, String tmpIndexDirPath, long latestGeneration)
throws Exception {
if (log.isDebugEnabled()) {
log.debug("Download files to dir: " + Arrays.asList(indexDir.listAll()));
log.debug("Download files to dir: {}", Arrays.asList(indexDir.listAll()));
}
long bytesDownloaded = 0;
long bytesSkippedCopying = 0;
@ -1027,7 +1039,9 @@ public class IndexFetcher {
totalSpaceRequired += size;
}
log.info("tmpIndexDir_type : " + tmpIndexDir.getClass() + " , " + FilterDirectory.unwrap(tmpIndexDir));
if (log.isInfoEnabled()) {
log.info("tmpIndexDir_type : {} , {}", tmpIndexDir.getClass(), FilterDirectory.unwrap(tmpIndexDir));
}
long usableSpace = usableDiskSpaceProvider.apply(tmpIndexDirPath);
if (getApproxTotalSpaceReqd(totalSpaceRequired) > usableSpace) {
deleteFilesInAdvance(indexDir, indexDirPath, totalSpaceRequired, usableSpace);
@ -1038,13 +1052,17 @@ public class IndexFetcher {
long size = (Long) file.get(SIZE);
CompareResult compareResult = compareFile(indexDir, filename, size, (Long) file.get(CHECKSUM));
boolean alwaysDownload = filesToAlwaysDownloadIfNoChecksums(filename, size, compareResult);
log.debug("Downloading file={} size={} checksum={} alwaysDownload={}", filename, size, file.get(CHECKSUM), alwaysDownload);
if (log.isDebugEnabled()) {
log.debug("Downloading file={} size={} checksum={} alwaysDownload={}", filename, size, file.get(CHECKSUM), alwaysDownload);
}
if (!compareResult.equal || downloadCompleteIndex || alwaysDownload) {
File localFile = new File(indexDirPath, filename);
if (downloadCompleteIndex && doDifferentialCopy && compareResult.equal && compareResult.checkSummed
&& localFile.exists()) {
log.info("Don't need to download this file. Local file's path is: {}, checksum is: {}",
localFile.getAbsolutePath(), file.get(CHECKSUM));
if (log.isInfoEnabled()) {
log.info("Don't need to download this file. Local file's path is: {}, checksum is: {}",
localFile.getAbsolutePath(), file.get(CHECKSUM));
}
// A hard link here should survive the eventual directory move, and should be more space efficient as
// compared to a file copy. TODO: Maybe we could do a move safely here?
Files.createLink(new File(tmpIndexDirPath, filename).toPath(), localFile.toPath());
@ -1058,7 +1076,9 @@ public class IndexFetcher {
}
filesDownloaded.add(new HashMap<>(file));
} else {
log.debug("Skipping download for {} because it already exists", file.get(NAME));
if (log.isDebugEnabled()) {
log.debug("Skipping download for {} because it already exists", file.get(NAME));
}
}
}
log.info("Bytes downloaded: {}, Bytes skipped downloading: {}", bytesDownloaded, bytesSkippedCopying);
@ -1121,8 +1141,8 @@ public class IndexFetcher {
// after considering the files actually available locally we really don't need to do any delete
return;
}
log.info("This disk does not have enough space to download the index from leader/master. So cleaning up the local index. " +
" This may lead to loss of data/or node if index replication fails in between");
log.info("This disk does not have enough space to download the index from leader/master. So cleaning up the local index. "
+ " This may lead to loss of data/or node if index replication fails in between");
//now we should disable searchers and index writers because this core will not have all the required files
this.clearLocalIndexFirst = true;
this.solrCore.searchEnabled = false;
@ -1191,7 +1211,8 @@ public class IndexFetcher {
return compareResult;
} else {
log.warn("File {} did not match. expected checksum is {} and actual is checksum {}. " +
"expected length is {} and actual length is {}", filename, backupIndexFileChecksum, indexFileChecksum,
"expected length is {} and actual length is {}"
, filename, backupIndexFileChecksum, indexFileChecksum,
backupIndexFileLen, indexFileLen);
compareResult.equal = false;
return compareResult;
@ -1201,7 +1222,7 @@ public class IndexFetcher {
compareResult.equal = false;
return compareResult;
} catch (IOException e) {
log.error("Could not read file " + filename + ". Downloading it again", e);
log.error("Could not read file {}. Downloading it again", filename, e);
compareResult.equal = false;
return compareResult;
}
@ -1259,7 +1280,7 @@ public class IndexFetcher {
boolean success = false;
try {
if (slowFileExists(indexDir, fname)) {
log.warn("Cannot complete replication attempt because file already exists:" + fname);
log.warn("Cannot complete replication attempt because file already exists: {}", fname);
// we fail - we downloaded the files we need, if we can't move one in, we can't
// count on the correct index
@ -1284,8 +1305,10 @@ public class IndexFetcher {
private boolean moveIndexFiles(Directory tmpIdxDir, Directory indexDir) {
if (log.isDebugEnabled()) {
try {
log.info("From dir files:" + Arrays.asList(tmpIdxDir.listAll()));
log.info("To dir files:" + Arrays.asList(indexDir.listAll()));
if (log.isInfoEnabled()) {
log.info("From dir files: {}", Arrays.asList(tmpIdxDir.listAll()));
log.info("To dir files: {}", Arrays.asList(indexDir.listAll())); //logOk
}
} catch (IOException e) {
throw new RuntimeException(e);
}
@ -1505,7 +1528,7 @@ public class IndexFetcher {
org.apache.lucene.util.IOUtils.rm(dir.toPath());
return true;
} catch (IOException e) {
log.warn("Unable to delete directory : " + dir, e);
log.warn("Unable to delete directory : {}", dir, e);
return false;
}
}

View File

@ -279,7 +279,7 @@ public class MoreLikeThisHandler extends RequestHandlerBase
}
}
} catch (ExitableDirectoryReader.ExitingReaderException ex) {
log.warn( "Query: " + req.getParamString() + "; " + ex.getMessage());
log.warn( "Query: {}; {}", req.getParamString(), ex.getMessage());
} finally {
SolrQueryTimeoutImpl.reset();
}

View File

@ -156,11 +156,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
try {
version = Long.parseLong(commitTime);
} catch (NumberFormatException e) {
log.warn("Version in commitData was not formatted correctly: " + commitTime, e);
log.warn("Version in commitData was not formatted correctly: {}", commitTime, e);
}
}
} catch (IOException e) {
log.warn("Unable to get version from commitData, commit: " + commit, e);
log.warn("Unable to get version from commitData, commit: {}", commit, e);
}
return new CommitVersionInfo(generation, version);
}
@ -376,7 +376,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
nl.add(CMD_GET_FILE_LIST, commitList);
l.add(nl);
} catch (IOException e) {
log.warn("Exception while reading files for commit " + c, e);
log.warn("Exception while reading files for commit {}", c, e);
}
}
return l;
@ -393,7 +393,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
checksum.update(buffer, 0, bytesRead);
return checksum.getValue();
} catch (Exception e) {
log.warn("Exception in finding checksum of " + f, e);
log.warn("Exception in finding checksum of {}", f, e);
} finally {
IOUtils.closeQuietly(fis);
}
@ -656,7 +656,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
fileMeta.put(CHECKSUM, checksum);
} catch (Exception e) {
//TODO Should this trigger a larger error?
log.warn("Could not read checksum from index file: " + file, e);
log.warn("Could not read checksum from index file: {}", file, e);
}
}
@ -675,13 +675,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
fileMeta.put(CHECKSUM, CodecUtil.retrieveChecksum(in));
} catch (Exception e) {
//TODO Should this trigger a larger error?
log.warn("Could not read checksum from index file: " + infos.getSegmentsFileName(), e);
log.warn("Could not read checksum from index file: {}", infos.getSegmentsFileName(), e);
}
}
}
result.add(fileMeta);
} catch (IOException e) {
log.error("Unable to get file names for indexCommit generation: " + commit.getGeneration(), e);
log.error("Unable to get file names for indexCommit generation: {}", commit.getGeneration(), e);
reportErrorOnResponse(rsp, "unable to get file names for given index generation", e);
return;
} finally {
@ -698,11 +698,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
if (solrParams.getBool(TLOG_FILES, false)) {
try {
List<Map<String, Object>> tlogfiles = getTlogFileList(commit);
log.info("Adding tlog files to list: " + tlogfiles);
log.info("Adding tlog files to list: {}", tlogfiles);
rsp.add(TLOG_FILES, tlogfiles);
}
catch (IOException e) {
log.error("Unable to get tlog file names for indexCommit generation: " + commit.getGeneration(), e);
log.error("Unable to get tlog file names for indexCommit generation: {}", commit.getGeneration(), e);
reportErrorOnResponse(rsp, "unable to get tlog file names for given index generation", e);
return;
}
@ -710,7 +710,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
if (confFileNameAlias.size() < 1 || core.getCoreContainer().isZooKeeperAware())
return;
log.debug("Adding config files to list: " + includeConfFiles);
log.debug("Adding config files to list: {}", includeConfFiles);
//if configuration files need to be included get their details
rsp.add(CONF_FILES, getConfFileInfoFromCache(confFileNameAlias, confFileInfoCache));
rsp.add(STATUS, OK_STATUS);
@ -817,7 +817,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
private void disablePoll(SolrQueryResponse rsp) {
if (pollingIndexFetcher != null){
pollDisabled.set(true);
log.info("inside disable poll, value of pollDisabled = " + pollDisabled);
log.info("inside disable poll, value of pollDisabled = {}", pollDisabled);
rsp.add(STATUS, OK_STATUS);
} else {
reportErrorOnResponse(rsp, "No slave configured", null);
@ -827,7 +827,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
private void enablePoll(SolrQueryResponse rsp) {
if (pollingIndexFetcher != null){
pollDisabled.set(false);
log.info("inside enable poll, value of pollDisabled = " + pollDisabled);
log.info("inside enable poll, value of pollDisabled = {}", pollDisabled);
rsp.add(STATUS, OK_STATUS);
} else {
reportErrorOnResponse(rsp, "No slave configured", null);
@ -1245,9 +1245,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
if (enableMaster || (enableSlave && !currentIndexFetcher.fetchFromLeader)) {
if (core.getCoreContainer().getZkController() != null) {
log.warn("SolrCloud is enabled for core " + core.getName() + " but so is old-style replication. Make sure you" +
" intend this behavior, it usually indicates a mis-configuration. Master setting is " +
Boolean.toString(enableMaster) + " and slave setting is " + Boolean.toString(enableSlave));
log.warn("SolrCloud is enabled for core {} but so is old-style replication. "
+ "Make sure you intend this behavior, it usually indicates a mis-configuration. "
+ "Master setting is {} and slave setting is {}"
, core.getName(), enableMaster, enableSlave);
}
}
@ -1266,7 +1267,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
// if there is an alias add it or it is null
confFileNameAlias.add(strs[0], strs.length > 1 ? strs[1] : null);
}
log.info("Replication enabled for following config files: " + includeConfFiles);
log.info("Replication enabled for following config files: {}", includeConfFiles);
}
List backup = master.getAll("backupAfter");
boolean backupOnCommit = backup.contains("commit");
@ -1290,7 +1291,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
solrPolicy.setMaxOptimizedCommitsToKeep(1);
}
} else {
log.warn("Replication can't call setMaxOptimizedCommitsToKeep on " + policy);
log.warn("Replication can't call setMaxOptimizedCommitsToKeep on {}", policy);
}
}
@ -1356,7 +1357,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
}
}
log.info("Commits will be reserved for " + reserveCommitDuration + "ms.");
log.info("Commits will be reserved for {} ms", reserveCommitDuration);
}
// check master or slave is enabled
@ -1609,7 +1610,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
fos.write(buf, 0, read);
fos.flush();
log.debug("Wrote {} bytes for file {}", offset + read, fileName);
log.debug("Wrote {} bytes for file {}", offset + read, fileName); // logOK
//Pause if necessary
maxBytesBeforePause += read;
@ -1626,7 +1627,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
in.seek(offset);
}
} catch (IOException e) {
log.warn("Exception while writing response for params: " + params, e);
log.warn("Exception while writing response for params: {}", params, e);
} finally {
if (in != null) {
in.close();
@ -1694,7 +1695,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
writeNothingAndFlush();
}
} catch (IOException e) {
log.warn("Exception while writing response for params: " + params, e);
log.warn("Exception while writing response for params: {}", params, e);
} finally {
IOUtils.closeQuietly(inputStream);
extendReserveAndReleaseCommitPoint();

View File

@ -84,7 +84,7 @@ public class RestoreCore implements Callable<Boolean> {
try {
checksum = CodecUtil.retrieveChecksum(indexInput);
} catch (Exception e) {
log.warn("Could not read checksum from index file: " + filename, e);
log.warn("Could not read checksum from index file: {}", filename, e);
}
long length = indexInput.length();
IndexFetcher.CompareResult compareResult = IndexFetcher.compareFile(indexDir, filename, length, checksum);

View File

@ -147,8 +147,8 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
ManagedIndexSchema managed = (ManagedIndexSchema) schema;
zkVersion = managed.getSchemaZkVersion();
if (refreshIfBelowVersion != -1 && zkVersion < refreshIfBelowVersion) {
log.info("REFRESHING SCHEMA (refreshIfBelowVersion=" + refreshIfBelowVersion +
", currentVersion=" + zkVersion + ") before returning version!");
log.info("REFRESHING SCHEMA (refreshIfBelowVersion={}, currentVersion={}) before returning version!"
, refreshIfBelowVersion, zkVersion);
ZkSolrResourceLoader zkSolrResourceLoader = (ZkSolrResourceLoader) req.getCore().getResourceLoader();
ZkIndexSchemaReader zkIndexSchemaReader = zkSolrResourceLoader.getZkIndexSchemaReader();
managed = zkIndexSchemaReader.refreshSchemaFromZk(refreshIfBelowVersion);

View File

@ -186,7 +186,9 @@ public class SnapShooter {
final Optional<IndexCommit> namedCommit = snapshotMgr.getIndexCommitByName(commitName);
if (namedCommit.isPresent()) {
final IndexCommit commit = namedCommit.get();
log.debug("Using named commit: name={}, generation={}", commitName, commit.getGeneration());
if (log.isDebugEnabled()) {
log.debug("Using named commit: name={}, generation={}", commitName, commit.getGeneration());
}
delPolicy.saveCommitPoint(commit.getGeneration());
return commit;
}
@ -200,7 +202,9 @@ public class SnapShooter {
throw new SolrException(ErrorCode.BAD_REQUEST, "Index does not yet have any commits for core " +
solrCore.getName());
}
log.debug("Using latest commit: generation={}", commit.getGeneration());
if (log.isDebugEnabled()) {
log.debug("Using latest commit: generation={}", commit.getGeneration());
}
return commit;
}
@ -240,7 +244,9 @@ public class SnapShooter {
*/
protected NamedList createSnapshot(final IndexCommit indexCommit) throws Exception {
assert indexCommit != null;
log.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName) + " at " + baseSnapDirPath);
if (log.isInfoEnabled()) {
log.info("Creating backup snapshot {} at {}", (snapshotName == null ? "<not named>" : snapshotName), baseSnapDirPath);
}
boolean success = false;
try {
NamedList<Object> details = new NamedList<>();
@ -262,7 +268,9 @@ public class SnapShooter {
details.add("snapshotCompletedAt", new Date().toString());//bad; should be Instant.now().toString()
details.add("snapshotName", snapshotName);
details.add("directoryName", directoryName);
log.info("Done creating backup snapshot: {} into {}", (snapshotName == null ? "<not named>" : snapshotName), snapshotDirPath);
if (log.isInfoEnabled()) {
log.info("Done creating backup snapshot: {} into {}", (snapshotName == null ? "<not named>" : snapshotName), snapshotDirPath);
}
success = true;
return details;
} finally {
@ -270,7 +278,7 @@ public class SnapShooter {
try {
backupRepo.deleteDirectory(snapshotDirPath);
} catch (Exception excDuringDelete) {
log.warn("Failed to delete "+snapshotDirPath+" after snapshot creation failed due to: "+excDuringDelete);
log.warn("Failed to delete {} after snapshot creation failed due to: {}", snapshotDirPath, excDuringDelete);
}
}
}
@ -300,7 +308,7 @@ public class SnapShooter {
}
protected void deleteNamedSnapshot(ReplicationHandler replicationHandler) {
log.info("Deleting snapshot: " + snapshotName);
log.info("Deleting snapshot: {}", snapshotName);
NamedList<Object> details = new NamedList<>();
details.add("snapshotName", snapshotName);
@ -314,7 +322,7 @@ public class SnapShooter {
} catch (IOException e) {
details.add("status", "Unable to delete snapshot: " + snapshotName);
log.warn("Unable to delete snapshot: " + snapshotName, e);
log.warn("Unable to delete snapshot: {}", snapshotName, e);
}
replicationHandler.snapShootDetails = details;

View File

@ -237,7 +237,9 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
}
}, SolrConfigHandler.class.getSimpleName() + "-refreshconf").start();
} else {
log.info("isStale {} , resourceloader {}", isStale, req.getCore().getResourceLoader().getClass().getName());
if (log.isInfoEnabled()) {
log.info("isStale {} , resourceloader {}", isStale, req.getCore().getResourceLoader().getClass().getName());
}
}
} else {
@ -365,7 +367,9 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
break;//succeeded . so no need to go over the loop again
} catch (ZkController.ResourceModifiedInZkException e) {
//retry
log.info("Race condition, the node is modified in ZK by someone else " + e.getMessage());
if (log.isInfoEnabled()) {
log.info("Race condition, the node is modified in ZK by someone else {}", e.getMessage());
}
}
}
} catch (Exception e) {
@ -456,7 +460,9 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
if (ops.isEmpty()) {
ZkController.touchConfDir(zkLoader);
} else {
log.debug("persisting params data : {}", Utils.toJSONString(params.toMap(new LinkedHashMap<>())));
if (log.isDebugEnabled()) {
log.debug("persisting params data : {}", Utils.toJSONString(params.toMap(new LinkedHashMap<>())));
}
int latestVersion = ZkController.persistConfigResourceToZooKeeper(zkLoader,
params.getZnodeVersion(), RequestParams.RESOURCE, params.toByteArray(), true);
@ -510,7 +516,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
}
List errs = CommandOperation.captureErrors(ops);
if (!errs.isEmpty()) {
log.error("ERROR:" + Utils.toJSONString(errs));
log.error("ERROR:{}", Utils.toJSONString(errs));
throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "error processing commands", errs);
}
@ -774,8 +780,10 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
}
if (concurrentTasks.isEmpty()) return; // nothing to wait for ...
log.info(formatString("Waiting up to {0} secs for {1} replicas to set the property {2} to be of version {3} for collection {4}",
maxWaitSecs, concurrentTasks.size(), prop, expectedVersion, collection));
if (log.isInfoEnabled()) {
log.info(formatString("Waiting up to {0} secs for {1} replicas to set the property {2} to be of version {3} for collection {4}",
maxWaitSecs, concurrentTasks.size(), prop, expectedVersion, collection));
}
// use an executor service to invoke schema zk version requests in parallel with a max wait time
int poolSize = Math.min(concurrentTasks.size(), 10);
@ -801,7 +809,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
if (!success) {
String coreUrl = concurrentTasks.get(f).coreUrl;
log.warn("Core " + coreUrl + " could not get the expected version " + expectedVersion);
log.warn("Core {} could not get the expected version {}", coreUrl, expectedVersion);
if (failedList == null) failedList = new ArrayList<>();
failedList.add(coreUrl);
}
@ -822,8 +830,10 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
ExecutorUtil.shutdownAndAwaitTermination(parallelExecutor);
}
log.info("Took {}ms to set the property {} to be of version {} for collection {}",
timer.getTime(), prop, expectedVersion, collection);
if (log.isInfoEnabled()) {
log.info("Took {}ms to set the property {} to be of version {} for collection {}",
timer.getTime(), prop, expectedVersion, collection);
}
}
public static List<String> getActiveReplicaCoreUrls(ZkController zkController,
@ -907,12 +917,14 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
}
attempts++;
log.info(formatString("Could not get expectedVersion {0} from {1} for prop {2} after {3} attempts", expectedZkVersion, coreUrl, prop, attempts));
if (log.isInfoEnabled()) {
log.info(formatString("Could not get expectedVersion {0} from {1} for prop {2} after {3} attempts", expectedZkVersion, coreUrl, prop, attempts));
}
} catch (Exception e) {
if (e instanceof InterruptedException) {
break; // stop looping
} else {
log.warn("Failed to get /schema/zkversion from " + coreUrl + " due to: " + e);
log.warn("Failed to get /schema/zkversion from {} due to: ", coreUrl, e);
}
}
}

View File

@ -716,7 +716,7 @@ public class IndexSchema {
Arrays.sort(dFields);
if (log.isTraceEnabled()) {
log.trace("Dynamic Field Ordering: {}", Arrays.toString(dFields)); // logok
log.trace("Dynamic Field Ordering: {}", Arrays.toString(dFields));
}
return dFields;

View File

@ -184,7 +184,7 @@ public class SpellCheckCollator {
collations.add(collation);
}
if (log.isDebugEnabled()) {
log.debug("Collation: {} {}", collationQueryStr, (verifyCandidateWithQuery ? (" will return " + hits + " hits.") : "")); // LOGOK
log.debug("Collation: {} {}", collationQueryStr, (verifyCandidateWithQuery ? (" will return " + hits + " hits.") : "")); // logOk
}
}
return collations;

View File

@ -274,7 +274,7 @@ public class HdfsDirectory extends BaseDirectory {
@Override
public void sync(Collection<String> names) throws IOException {
if (log.isDebugEnabled()) {
log.debug("Sync called on {}", Arrays.toString(names.toArray())); // logok
log.debug("Sync called on {}", Arrays.toString(names.toArray()));
}
}

View File

@ -311,9 +311,8 @@ public class SolrIndexSplitter {
t.resume();
for (int segmentNumber = 0; segmentNumber<leaves.size(); segmentNumber++) {
if (log.isInfoEnabled()) {
log.info("SolrIndexSplitter: partition #{} partitionCount={} segment #{} segmentCount={}"
, partitionNumber, numPieces + (ranges != null ? " range=" + ranges.get(partitionNumber) : "")
, segmentNumber, leaves.size()); //LOGOK
log.info("SolrIndexSplitter: partition # {} partitionCount={} {} segment #={} segmentCount={}", partitionNumber, numPieces
, (ranges != null ? " range=" + ranges.get(partitionNumber) : ""), segmentNumber, leaves.size()); // logOk
}
CodecReader subReader = SlowCodecReaderWrapper.wrap(leaves.get(segmentNumber).reader());
iw.addIndexes(new LiveDocsReader(subReader, segmentDocSets.get(segmentNumber)[partitionNumber]));

View File

@ -308,7 +308,7 @@ public class DocBasedVersionConstraintsProcessor extends UpdateRequestProcessor
if (ignoreOldUpdates) {
if (log.isDebugEnabled()) {
log.debug("Dropping update since user version is not high enough: {}; old user version={}",
Arrays.toString(newUserVersions), Arrays.toString(oldUserVersions)); // logok
Arrays.toString(newUserVersions), Arrays.toString(oldUserVersions));
}
return false;
} else {

View File

@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory;
public class DiskChecker {
public static final Object SOLR_HACK_FOR_CLASS_VERIFICATION = new Object();
public static final Logger LOG = LoggerFactory.getLogger(DiskChecker.class);
public static final Logger log = LoggerFactory.getLogger(DiskChecker.class);
public static class DiskErrorException extends IOException {
public DiskErrorException(String msg) {
@ -293,7 +293,7 @@ public class DiskChecker {
}
file = null;
} finally {
IOUtils.cleanupWithLogger(LOG, fos);
IOUtils.cleanupWithLogger(log, fos);
FileUtils.deleteQuietly(file);
}
}

View File

@ -346,8 +346,10 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
int computeRestoreMaxShardsPerNode = (int) Math.ceil((restoreReplFactor * numShards/(double) cluster.getJettySolrRunners().size()));
if (restoreReplFactor > backupReplFactor) { //else the backup maxShardsPerNode should be enough
log.info("numShards={} restoreReplFactor={} maxShardsPerNode={} totalNodes={}",
numShards, restoreReplFactor, computeRestoreMaxShardsPerNode, cluster.getJettySolrRunners().size());
if (log.isInfoEnabled()) {
log.info("numShards={} restoreReplFactor={} maxShardsPerNode={} totalNodes={}",
numShards, restoreReplFactor, computeRestoreMaxShardsPerNode, cluster.getJettySolrRunners().size());
}
if (random().nextBoolean()) { //set it to -1
isMaxShardsUnlimited = true;
@ -430,7 +432,9 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
Map<String, Integer> restoredCollectionPerShardCountAfterIndexing = getShardToDocCountMap(client, restoreCollection);
int restoredCollectionFinalDocCount = restoredCollectionPerShardCountAfterIndexing.values().stream().mapToInt(Number::intValue).sum();
log.info("Original doc count in restored collection:" + restoredCollectionDocCount + ", number of newly added documents to the restored collection: " + numberNewDocsIndexed + ", after indexing: " + restoredCollectionFinalDocCount);
log.info("Original doc count in restored collection:{} , number of newly added documents to the restored collection: {}"
+ ", after indexing: {}"
, restoredCollectionDocCount, numberNewDocsIndexed, restoredCollectionFinalDocCount);
assertEquals((restoredCollectionDocCount + numberNewDocsIndexed), restoredCollectionFinalDocCount);
}

View File

@ -75,7 +75,7 @@ public class CollectionReloadTest extends SolrCloudTestCase {
cluster.expireZkSession(cluster.getReplicaJetty(leader));
waitForState("Timed out waiting for core to re-register as ACTIVE after session expiry", testCollectionName, (n, c) -> {
log.info("Collection state: {}", c.toString());
log.info("Collection state: {}", c);
Replica expiredReplica = c.getReplica(leader.getName());
return expiredReplica.getState() == Replica.State.ACTIVE && c.getZNodeVersion() > initialStateVersion;
});

View File

@ -255,11 +255,15 @@ public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
}
try {
log.info("{} - Reloading Collection.", Thread.currentThread().getName());
if (log.isInfoEnabled()) {
log.info("{} - Reloading Collection.", Thread.currentThread().getName());
}
reloadCollectionRequest.processAsync("repeatedId", clients[random().nextInt(clients.length)]);
numSuccess.incrementAndGet();
} catch (SolrServerException e) {
log.info(e.getMessage());
if (log.isInfoEnabled()) {
log.info(e.getMessage());
}
assertEquals("Task with the same requestid already exists.", e.getMessage());
numFailure.incrementAndGet();
} catch (IOException e) {

View File

@ -188,12 +188,14 @@ public class ConcurrentCreateCollectionTest extends SolrCloudTestCase {
/***
SolrRequest req = CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.GET, null);
SolrResponse response = req.process(client);
log.info("######### AUTOSCALE " + response);
log.info("######### AUTOSCALE {}", response);
***/
byte[] data = client.getZkStateReader().getZkClient().getData("/autoscaling.json", null, null, true);
log.info("AUTOSCALE DATA: " + new String(data, "UTF-8"));
if (log.isInfoEnabled()) {
log.info("AUTOSCALE DATA: {}", new String(data, "UTF-8"));
}
final AtomicInteger collectionNum = new AtomicInteger();
Thread[] indexThreads = new Thread[nThreads];
@ -258,7 +260,7 @@ public class ConcurrentCreateCollectionTest extends SolrCloudTestCase {
if (expectBalanced) {
failed = true;
}
log.error("UNBALANCED CLUSTER: expected replicas per node " + expectedPerNode + " but got " + replicas.size());
log.error("UNBALANCED CLUSTER: expected replicas per node {} but got {}", expectedPerNode, replicas.size());
}
}
@ -271,14 +273,14 @@ public class ConcurrentCreateCollectionTest extends SolrCloudTestCase {
if (prev != null) {
failed = true;
// NOTE: with a replication factor > 2, this will print multiple times per bad slice.
log.error("MULTIPLE REPLICAS OF SINGLE SHARD ON SAME NODE: r1=" + prev + " r2=" + replica);
log.error("MULTIPLE REPLICAS OF SINGLE SHARD ON SAME NODE: r1={} r2={}", prev, replica);
}
}
}
}
if (failed) {
log.error("Cluster state " + cstate.getCollectionsMap());
log.error("Cluster state {}", cstate.getCollectionsMap());
}
assertEquals(replicaMap.size(), NODES); // make sure something was created

View File

@ -263,7 +263,9 @@ public class ShardSplitTest extends BasicDistributedZkTest {
HttpSolrClient client = new HttpSolrClient.Builder(replica.getCoreUrl())
.withHttpClient(cloudClient.getLbClient().getHttpClient()).build();
QueryResponse response = client.query(new SolrQuery("q", "*:*", "distrib", "false"));
log.info("Found numFound={} on replica: {}", response.getResults().getNumFound(), replica.getCoreUrl());
if (log.isInfoEnabled()) {
log.info("Found numFound={} on replica: {}", response.getResults().getNumFound(), replica.getCoreUrl());
}
if (numFound == Long.MIN_VALUE) {
numFound = response.getResults().getNumFound();
} else {
@ -382,7 +384,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
cloudClient.getZkStateReader().forceUpdateCollection(collectionName);
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
DocCollection coll = clusterState.getCollection(collectionName);
log.info("coll: " + coll);
log.info("coll: {}", coll);
// verify the original shard
verifyShard(coll, SHARD1, Slice.State.INACTIVE, 2, 0, 2);
@ -458,7 +460,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
stop.set(true);
return true;
} catch (Exception e) {
log.error("Monkey unable to kill jetty at port " + cjetty.jetty.getLocalPort(), e);
log.error("Monkey unable to kill jetty at port {}", cjetty.jetty.getLocalPort(), e);
}
}
}
@ -485,7 +487,9 @@ public class ShardSplitTest extends BasicDistributedZkTest {
// indexed are available in SolrCloud and if the split succeeded then all replicas of the sub-shard
// must be consistent (i.e. have same numdocs)
log.info("Shard split request state is {}", splitStatus == null ? "unknown" : splitStatus.getKey());
if (log.isInfoEnabled()) {
log.info("Shard split request state is {}", splitStatus == null ? "unknown" : splitStatus.getKey());
}
stop.set(true);
monkeyThread.join();
Set<String> addFails = new HashSet<>();
@ -496,10 +500,14 @@ public class ShardSplitTest extends BasicDistributedZkTest {
}
CloudJettyRunner cjetty = shardToLeaderJetty.get(SHARD1);
log.info("Starting shard1 leader jetty at port {}", cjetty.jetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Starting shard1 leader jetty at port {}", cjetty.jetty.getLocalPort());
}
cjetty.jetty.start();
cloudClient.getZkStateReader().forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
log.info("Current collection state: {}", printClusterStateInfo(AbstractDistribZkTestBase.DEFAULT_COLLECTION));
if (log.isInfoEnabled()) {
log.info("Current collection state: {}", printClusterStateInfo(AbstractDistribZkTestBase.DEFAULT_COLLECTION));
}
// true if sub-shard states switch to 'active' eventually
AtomicBoolean areSubShardsActive = new AtomicBoolean(false);
@ -604,7 +612,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
trySplit(collectionName, null, SHARD1, 1);
fail("expected to fail due to locking but succeeded");
} catch (Exception e) {
log.info("Expected failure: " + e.toString());
log.info("Expected failure: {}", e);
}
// make sure the lock still exists
@ -728,7 +736,9 @@ public class ShardSplitTest extends BasicDistributedZkTest {
cloudClient.getZkStateReader().forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
clusterState = cloudClient.getZkStateReader().getClusterState();
log.debug("-- COLLECTION: {}", clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION));
if (log.isDebugEnabled()) {
log.debug("-- COLLECTION: {}", clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION));
}
del("*:*");
for (int id = 0; id <= 100; id++) {
String shardKey = "" + (char)('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution
@ -740,7 +750,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
Random random = random();
int max = atLeast(random, 401);
int sleep = atLeast(random, 25);
log.info("SHARDSPLITTEST: Going to add " + max + " number of docs at 1 doc per " + sleep + "ms");
log.info("SHARDSPLITTEST: Going to add {} number of docs at 1 doc per {} ms", max, sleep);
Set<String> deleted = new HashSet<>();
for (int id = 101; id < max; id++) {
try {
@ -758,7 +768,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
}
}
} catch (Exception e) {
log.error("Exception while adding doc id = " + id, e);
log.error("Exception while adding doc id = {}", id, e);
// do not select this id for deletion ever
deleted.add(String.valueOf(id));
}
@ -777,7 +787,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
if (e.code() != 500) {
throw e;
}
log.error("SPLITSHARD failed. " + (i < 2 ? " Retring split" : ""), e);
log.error("SPLITSHARD failed. {}", (i < 2 ? " Retring split" : ""), e);
if (i == 2) {
fail("SPLITSHARD was not successful even after three tries");
}
@ -844,7 +854,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
for (int i = 0; i < docCounts.length; i++) {
int docCount = docCounts[i];
log.info("Shard {} docCount = {}", "shard1_" + i, docCount);
log.info("Shard shard1_{} docCount = {}", i, docCount);
}
collectionClient.commit();
@ -912,7 +922,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
for (int i = 0; i < docCounts.length; i++) {
int docCount = docCounts[i];
log.info("Shard {} docCount = {}", "shard1_" + i, docCount);
log.info("Shard shard1_{} docCount = {}", i, docCount);
}
log.info("Route key doc count = {}", splitKeyDocCount);
@ -942,7 +952,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
if (e.code() != 500) {
throw e;
}
log.error("SPLITSHARD failed. " + (i < maxTries - 1 ? " Retring split" : ""), e);
log.error("SPLITSHARD failed. {}", (i < maxTries - 1 ? " Retring split" : ""), e);
if (i == 2) {
fail("SPLITSHARD was not successful even after three tries");
}
@ -1019,7 +1029,9 @@ public class ShardSplitTest extends BasicDistributedZkTest {
response = client.query(query);
}
numFound[c++] = response.getResults().getNumFound();
log.info("Shard: " + shard + " Replica: {} has {} docs", coreUrl, String.valueOf(response.getResults().getNumFound()));
if (log.isInfoEnabled()) {
log.info("Shard: {} Replica: {} has {} docs", shard, coreUrl, String.valueOf(response.getResults().getNumFound()));
}
assertTrue("Shard: " + shard + " Replica: " + coreUrl + " has 0 docs", response.getResults().getNumFound() > 0);
}
for (int i = 0; i < slice.getReplicasMap().size(); i++) {
@ -1057,7 +1069,9 @@ public class ShardSplitTest extends BasicDistributedZkTest {
try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl, 30000, 60000 * 5)) {
NamedList<Object> rsp = baseServer.request(request);
log.info("Shard split response: " + Utils.toJSONString(rsp));
if (log.isInfoEnabled()) {
log.info("Shard split response: {}", Utils.toJSONString(rsp));
}
}
}
@ -1113,7 +1127,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
idVsVersion.put(document.getFieldValue("id").toString(), document.getFieldValue("_version_").toString());
SolrDocument old = shard10Docs.put(document.getFieldValue("id").toString(), document);
if (old != null) {
log.error("EXTRA: ID: " + document.getFieldValue("id") + " on shard1_0. Old version: " + old.getFieldValue("_version_") + " new version: " + document.getFieldValue("_version_"));
log.error("EXTRA: ID: {} on shard1_0. Old version: {} new version: {}", document.getFieldValue("id"), old.getFieldValue("_version_"), document.getFieldValue("_version_"));
}
found.add(document.getFieldValue("id").toString());
}
@ -1122,21 +1136,22 @@ public class ShardSplitTest extends BasicDistributedZkTest {
String value = document.getFieldValue("id").toString();
String version = idVsVersion.get(value);
if (version != null) {
log.error("DUPLICATE: ID: " + value + " , shard1_0Version: " + version + " shard1_1Version:" + document.getFieldValue("_version_"));
log.error("DUPLICATE: ID: {}, shard1_0Version {} shard1_1Version: {}", value, version, document.getFieldValue("_version_"));
}
SolrDocument old = shard11Docs.put(document.getFieldValue("id").toString(), document);
if (old != null) {
log.error("EXTRA: ID: " + document.getFieldValue("id") + " on shard1_1. Old version: " + old.getFieldValue("_version_") + " new version: " + document.getFieldValue("_version_"));
log.error("EXTRA: ID: {} on shard1_1. Old version: {} new version: {}"
,document.getFieldValue("id"), old.getFieldValue("_version_"), document.getFieldValue("_version_"));
}
found.add(document.getFieldValue("id").toString());
}
if (found.size() < documentIds.size()) {
documentIds.removeAll(found);
log.error("MISSING: ID: " + documentIds);
log.error("MISSING: ID: {}", documentIds);
} else if (found.size() > documentIds.size()) {
found.removeAll(documentIds);
log.error("EXTRA: ID: " + found);
log.error("EXTRA: ID: {}", found);
}
}
}

View File

@ -182,7 +182,7 @@ public class SplitByPrefixTest extends SolrCloudTestCase {
if (uniquePrefixes.size() % 2 == 1) { // make it an even sized list so we can split it exactly in two
uniquePrefixes.remove(uniquePrefixes.size()-1);
}
log.info("Unique prefixes: " + uniquePrefixes);
log.info("Unique prefixes: {}", uniquePrefixes);
for (Prefix prefix : uniquePrefixes) {
client.add( getDoc(prefix.key, "doc1") );

View File

@ -99,9 +99,11 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
@ -115,7 +117,9 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
JettySolrRunner lostJetty = random().nextBoolean() ? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
@ -125,8 +129,10 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION);
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
@ -147,9 +153,11 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
@ -165,8 +173,10 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
JettySolrRunner lostJetty = random().nextBoolean() ? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
@ -176,8 +186,10 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
waitForState(COLLECTION + "=(2,2)", COLLECTION,
clusterShape(2, 2), 90, TimeUnit.SECONDS);
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
@ -200,9 +212,11 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
@ -227,7 +241,9 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
@ -237,8 +253,10 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
waitForState(COLLECTION + "=(2,4) w/o down replicas",
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION);
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
@ -271,20 +289,24 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(false) // NOTE: false
.setMaxShardsPerNode(2)
.process(cluster.getSolrClient());
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", ALT_COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", ALT_COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(ALT_COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true) // NOTE: true
@ -298,7 +320,9 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
@ -310,8 +334,10 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
ALT_COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
waitForState(COLLECTION + "=(2,2)", COLLECTION, clusterShape(2, 2));
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
// save time, don't bother waiting for lostJetty to start until after updating collection prop...
@ -329,7 +355,9 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
// make sure lostJetty is fully up before stopping again...
waitForNodeLive(lostJetty);
log.info("Re-Stopping (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Re-Stopping (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
@ -341,8 +369,10 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
waitForState(COLLECTION + "=(2,4) w/o down replicas",
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION);
log.info("Re-Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("Re-Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
@ -405,7 +435,9 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
*/
private void waitForNodeLive(final JettySolrRunner jetty)
throws InterruptedException, TimeoutException, IOException {
log.info("waitForNodeLive: {}/{}", jetty.getNodeName(), jetty.getLocalPort());
if (log.isInfoEnabled()) {
log.info("waitForNodeLive: {}/{}", jetty.getNodeName(), jetty.getLocalPort());
}
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while(!timeout.hasTimedOut()) {

View File

@ -874,12 +874,16 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
assertEquals(3l, getObjectByPath(violation, true, "violation/replica/NRT"));
assertNotNull(violation.get("clause"));
}
log.info("Before starting new jetty ,{}", cluster.getJettySolrRunners()
.stream()
.map(jettySolrRunner -> jettySolrRunner.getNodeName()).collect(Collectors.toList()));
if (log.isInfoEnabled()) {
log.info("Before starting new jetty ,{}", cluster.getJettySolrRunners()
.stream()
.map(jettySolrRunner -> jettySolrRunner.getNodeName()).collect(Collectors.toList()));
}
JettySolrRunner runner1 = cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
log.info("started new jetty {}", runner1.getNodeName());
if (log.isInfoEnabled()) {
log.info("started new jetty {}", runner1.getNodeName());
}
response = waitForResponse(namedList -> {
List l = (List) namedList._get("diagnostics/liveNodes",null);

View File

@ -162,11 +162,17 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
for (String node: cloudManager.getClusterStateProvider().getLiveNodes()) {
Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, ImplicitSnitch.tags);
log.debug("* Node values: " + node + "\n" + Utils.toJSONString(values));
if (log.isDebugEnabled()) {
log.debug("* Node values: {}\n{}", node, Utils.toJSONString(values));
}
}
if (log.isDebugEnabled()) {
log.debug("* Live nodes: {}", cloudManager.getClusterStateProvider().getLiveNodes());
}
log.debug("* Live nodes: " + cloudManager.getClusterStateProvider().getLiveNodes());
ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
state.forEachCollection(coll -> log.debug("* Collection " + coll.getName() + " state: " + coll));
if (log.isDebugEnabled()) {
state.forEachCollection(coll -> log.debug("* Collection {} state: {}", coll.getName(), coll));
}
}
@AfterClass
@ -322,7 +328,9 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
assertNotNull(context);
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null "+ getNodeStateProviderState() + actionContextPropsRef.get(), operations);
operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
if (log.isInfoEnabled()) {
operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
}
assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
for (SolrRequest solrRequest : operations) {
@ -437,7 +445,9 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
@Test
//2018-06-18 (commented) @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
public void testSelectedCollections() throws Exception {
log.info("Found number of jetties: {}", cluster.getJettySolrRunners().size());
if (log.isInfoEnabled()) {
log.info("Found number of jetties: {}", cluster.getJettySolrRunners().size());
}
// start 3 more nodes
cluster.startJettySolrRunner();
cluster.startJettySolrRunner();

View File

@ -106,7 +106,7 @@ public class IndexSizeTriggerMixedBoundsTest extends SolrCloudTestCase {
ActionContext context, Throwable error, String message) {
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
CapturedEvent ev = new CapturedEvent(cloudManager.getTimeSource().getTimeNs(), context, config, stage, actionName, event, message);
log.info("=======> " + ev);
log.info("=======> {}", ev);
lst.add(ev);
}
}

View File

@ -119,7 +119,7 @@ public class IndexSizeTriggerSizeEstimationTest extends SolrCloudTestCase {
ActionContext context, Throwable error, String message) {
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
CapturedEvent ev = new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message);
log.info("=======> " + ev);
log.info("=======> {}", ev);
lst.add(ev);
}
}

View File

@ -116,7 +116,9 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
@After
public void restoreDefaults() throws Exception {
if (!realCluster) {
log.info(((SimCloudManager) cloudManager).dumpClusterState(true));
if (log.isInfoEnabled()) {
log.info(((SimCloudManager) cloudManager).dumpClusterState(true));
}
((SimCloudManager) cloudManager).getSimClusterStateProvider().simDeleteAllCollections();
((SimCloudManager) cloudManager).simClearSystemCollection();
((SimCloudManager) cloudManager).getSimClusterStateProvider().simResetLeaderThrottles();
@ -239,7 +241,7 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
ActionContext context, Throwable error, String message) {
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
CapturedEvent ev = new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message);
log.info("=======> " + ev);
log.info("=======> {}", ev);
lst.add(ev);
}
}

View File

@ -100,7 +100,9 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
// clear any persisted auto scaling configuration
Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
log.info(SOLR_AUTOSCALING_CONF_PATH + " reset, new znode version {}", stat.getVersion());
if (log.isInfoEnabled()) {
log.info("{} reset, new znode version {}", SOLR_AUTOSCALING_CONF_PATH, stat.getVersion());
}
cluster.getSolrClient().setDefaultCollection(null);

View File

@ -93,7 +93,9 @@ public class NodeLostTriggerIntegrationTest extends SolrCloudTestCase {
// clear any persisted auto scaling configuration
Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
log.info(SOLR_AUTOSCALING_CONF_PATH + " reset, new znode version {}", stat.getVersion());
if (log.isInfoEnabled()) {
log.info("{} reset, new znode version {}", SOLR_AUTOSCALING_CONF_PATH, stat.getVersion());
}
cluster.getSolrClient().setDefaultCollection(null);

View File

@ -153,7 +153,7 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
@Test
public void testTriggerDefaults() throws Exception {
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
log.info(autoScalingConfig.toString());
log.info("{}", autoScalingConfig);
AutoScalingConfig.TriggerConfig triggerConfig = autoScalingConfig.getTriggerConfigs().get(AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_NAME);
assertNotNull(triggerConfig);
assertEquals(3, triggerConfig.actions.size());
@ -181,7 +181,7 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
ActionContext context, Throwable error, String message) {
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
CapturedEvent ev = new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message);
log.info("=======> " + ev);
log.info("=======> {}", ev);
lst.add(ev);
}
}

View File

@ -104,7 +104,9 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
cluster.deleteAllCollections();
// clear any persisted auto scaling configuration
Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
log.info(SOLR_AUTOSCALING_CONF_PATH + " reset, new znode version {}", stat.getVersion());
if (log.isInfoEnabled()) {
log.info("{} reset, new znode version {}", SOLR_AUTOSCALING_CONF_PATH, stat.getVersion());
}
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
@ -715,7 +717,7 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
log.warn("Ignoring captured event since latch is 'full': {}", ev);
} else {
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
log.info("=======> " + ev);
log.info("=======> {}", ev);
lst.add(ev);
latch.countDown();
}

View File

@ -150,7 +150,9 @@ public class SystemLogListenerTest extends SolrCloudTestCase {
// Stop a node (that's safe to stop for the purposes of this test)
final JettySolrRunner stoppedJetty = pickNodeToStop();
log.info("Stopping node " + stoppedJetty.getNodeName());
if (log.isInfoEnabled()) {
log.info("Stopping node {}", stoppedJetty.getNodeName());
}
cluster.stopJettySolrRunner(stoppedJetty);
cluster.waitForJettyToStop(stoppedJetty);

View File

@ -166,7 +166,9 @@ public class TestPolicyCloud extends SolrCloudTestCase {
for (Row row : session.getSortedNodes()) {
Object val = row.getVal(Type.TOTALDISK.tagName, null);
log.info("node: {} , totaldisk : {}, freedisk : {}", row.node, val, row.getVal("freedisk",null));
if (log.isInfoEnabled()) {
log.info("node: {} , totaldisk : {}, freedisk : {}", row.node, val, row.getVal("freedisk", null));
}
assertTrue(val != null);
}

View File

@ -133,7 +133,9 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
// clear any persisted auto scaling configuration
Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
log.info(SOLR_AUTOSCALING_CONF_PATH + " reset, new znode version {}", stat.getVersion());
if (log.isInfoEnabled()) {
log.info("{} reset, new znode version {}", SOLR_AUTOSCALING_CONF_PATH, stat.getVersion());
}
cluster.deleteAllCollections();
cluster.getSolrClient().setDefaultCollection(null);
@ -291,20 +293,29 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
long currentTime = actionContext.getCloudManager().getTimeSource().getTimeNs();
if (lastActionExecutedAt.get() != 0) {
long minDiff = TimeUnit.MILLISECONDS.toNanos(throttlingDelayMs.get() - DELTA_MS);
log.info("last action at " + lastActionExecutedAt.get() + " current time = " + currentTime +
"\nreal diff: " + (currentTime - lastActionExecutedAt.get()) +
"\n min diff: " + minDiff);
if (log.isInfoEnabled()) {
log.info("last action at {} current time = {}\nreal diff: {}\n min diff: {}"
, lastActionExecutedAt.get(), currentTime
, (currentTime - lastActionExecutedAt.get())
, minDiff);
}
if (currentTime - lastActionExecutedAt.get() < minDiff) {
log.info("action executed again before minimum wait time from {}", event.getSource());
if (log.isInfoEnabled()) {
log.info("action executed again before minimum wait time from {}", event.getSource());
}
fail("TriggerListener was fired before the throttling period");
}
}
if (onlyOnce.compareAndSet(false, true)) {
log.info("action executed from {}", event.getSource());
if (log.isInfoEnabled()) {
log.info("action executed from {}", event.getSource());
}
lastActionExecutedAt.set(currentTime);
getTriggerFiredLatch().countDown();
} else {
log.info("action executed more than once from {}", event.getSource());
if (log.isInfoEnabled()) {
log.info("action executed more than once from {}", event.getSource());
}
fail("Trigger should not have fired more than once!");
}
} finally {

View File

@ -80,20 +80,26 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
final AutoScaling.Trigger t1 = new MockTrigger(TriggerEventType.NODELOST, "mock-timestamper") {
@Override
public void run() {
log.info("Running {} in {}", this.getName(), Thread.currentThread().getName());
if (log.isInfoEnabled()) {
log.info("Running {} in {}", this.getName(), Thread.currentThread().getName());
}
timestamps.offer(solrCloudManager.getTimeSource().getTimeNs());
}
};
log.info("Configuring simple scheduler and adding trigger: {}", t1.getName());
if (log.isInfoEnabled()) {
log.info("Configuring simple scheduler and adding trigger: {}", t1.getName());
}
t1.configure(resourceLoader, solrCloudManager, Collections.emptyMap());
scheduledTriggers.add(t1);
waitForAndDiffTimestamps("conf(default delay)",
ScheduledTriggers.DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS,
timestamps);
log.info("Reconfiguing scheduler to use 4s delay and clearing queue for trigger: {}", t1.getName());
if (log.isInfoEnabled()) {
log.info("Reconfiguing scheduler to use 4s delay and clearing queue for trigger: {}", t1.getName());
}
config = config.withProperties(Collections.singletonMap
(AutoScalingParams.TRIGGER_SCHEDULE_DELAY_SECONDS, 4));
scheduledTriggers.setAutoScalingConfig(config);
@ -102,8 +108,10 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
waitForAndDiffTimestamps("conf(four sec delay)",
4, TimeUnit.SECONDS,
timestamps);
log.info("Removing trigger: {}", t1.getName());
if (log.isInfoEnabled()) {
log.info("Removing trigger: {}", t1.getName());
}
scheduledTriggers.remove(t1.getName());
log.info("Reconfiguing scheduler to use default props");
@ -146,11 +154,15 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
"mock-blocking-trigger-" + i) {
@Override
public void run() {
log.info("Running {} in {}", this.getName(), Thread.currentThread().getName());
if (log.isInfoEnabled()) {
log.info("Running {} in {}", this.getName(), Thread.currentThread().getName());
}
CyclicBarrier barrier = null;
synchronized (latch) {
if (triggerNames.add(this.getName())) {
log.info("{}: No-Op since we've already recorded a run", this.getName());
if (log.isInfoEnabled()) {
log.info("{}: No-Op since we've already recorded a run", this.getName());
}
return;
}
threadNames.add(Thread.currentThread().getName());
@ -158,12 +170,14 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
}
try {
log.info("{}: waiting on barrier to hog a thread", this.getName());
if (log.isInfoEnabled()) {
log.info("{}: waiting on barrier to hog a thread", this.getName());
}
barrier.await(30, TimeUnit.SECONDS);
completionSemaphore.release();
} catch (Exception e) {
fails.incrementAndGet();
log.error(this.getName() + ": failure waiting on cyclic barrier: " + e.toString(), e);
log.error("{} : failure waiting on cyclic barrier: {}", this.getName(), e, e);
}
}
};
@ -171,7 +185,9 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
trigger.configure(resourceLoader, solrCloudManager, Collections.emptyMap());
triggerList.add(trigger);
completionSemaphore.acquire();
log.info("Adding trigger {} to scheduler", trigger.getName());
if (log.isInfoEnabled()) {
log.info("Adding trigger {} to scheduler", trigger.getName());
}
scheduledTriggers.add(trigger);
}
@ -222,7 +238,7 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
final TimeUnit minExpectedDeltaUnit,
final BlockingQueue<Long> timestamps) {
try {
log.info(label + ": Waiting for 2 timestamps to be recorded");
log.info("{}: Waiting for 2 timestamps to be recorded", label);
Long firstTs = timestamps.poll(minExpectedDelta * 3, minExpectedDeltaUnit);
assertNotNull(label + ": Couldn't get first timestampe after max allowed polling", firstTs);
Long secondTs = timestamps.poll(minExpectedDelta * 3, minExpectedDeltaUnit);
@ -234,7 +250,7 @@ public class TriggerSetPropertiesIntegrationTest extends SolrCloudTestCase {
"at least as much as min expected delay: " + minExpectedDeltaInNanos + "ns",
deltaInNanos >= minExpectedDeltaInNanos);
} catch (InterruptedException e) {
log.error(label + ": interupted", e);
log.error("{}: interupted", label, e);
fail(label + ": interupted:" + e.toString());
}
}

View File

@ -109,7 +109,9 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
public void tearDown() throws Exception {
super.tearDown();
if (cluster != null) {
log.info(cluster.dumpClusterState(false));
if (log.isInfoEnabled()) {
log.info(cluster.dumpClusterState(false));
}
}
}

View File

@ -73,7 +73,7 @@ public class TestSimClusterStateProvider extends SolrCloudTestCase {
public static void setupCluster() throws Exception {
simulated = random().nextBoolean();
simulated = true;
log.info("####### Using simulated components? " + simulated);
log.info("####### Using simulated components? {}", simulated);
configureCluster(NODE_COUNT)
.addConfig("conf", configset("cloud-minimal"))

View File

@ -110,13 +110,17 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
// test didn't init, nothing to do
return;
}
log.info("-------------_ FINAL STATE --------------");
log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
if (log.isInfoEnabled()) {
log.info("-------------_ FINAL STATE --------------");
log.info("* Node values: {}", Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues())); // logOk
log.info("* Live nodes: {}", cluster.getClusterStateProvider().getLiveNodes()); // logOk
}
ClusterState state = cluster.getClusterStateProvider().getClusterState();
for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
log.info("* Collection " + coll + " state: " + state.getCollection(coll));
if (log.isInfoEnabled()) {
for (String coll : cluster.getSimClusterStateProvider().simListCollections()) {
log.info("* Collection {} state: {}", coll, state.getCollection(coll)); // logOk
}
}
shutdownCluster();
}
@ -249,8 +253,10 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
assertNotNull(context);
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null " + actionContextPropsRef.get() + "\nevent: " + eventRef.get(), operations);
operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
if (log.isInfoEnabled()) {
operations.forEach(solrRequest -> log.info("{}", solrRequest.getParams()));
}
// TODO: this can be 3!
// assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
@ -258,7 +264,7 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
SolrParams params = solrRequest.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
String moved = params.get("replica");
// TODO: this can fail!
// assertTrue(replicasToBeMoved.stream().anyMatch(replica -> replica.getName().equals(moved)));
}
@ -324,14 +330,18 @@ public class TestSimComputePlanAction extends SimSolrCloudTestCase {
assertTrue(fired.get());
Map context = actionContextPropsRef.get();
assertNotNull(context);
log.info("Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("Live nodes: " + cluster.getClusterStateProvider().getLiveNodes() + ", collection state: " + cluster.getClusterStateProvider().getClusterState().getCollection("testNodeAdded"));
if (log.isInfoEnabled()) {
log.info("Node values: {}", Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("Live nodes: {}, collection state: {}"
, cluster.getClusterStateProvider().getLiveNodes(), cluster.getClusterStateProvider().getClusterState().getCollection("testNodeAdded")); // logOk
}
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null" + context, operations);
// TODO: can be 2!
// assertEquals("ComputePlanAction should have computed exactly 1 operation, but was: " + operations, 1, operations.size());
SolrRequest request = operations.get(0);
SolrParams params = request.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));

View File

@ -81,7 +81,9 @@ public class TestSimDistribStateManager extends SolrTestCaseJ4 {
solrZkClient = new SolrZkClient(zkTestServer.getZkHost(), 30000);
stateManager = new ZkDistribStateManager(solrZkClient);
}
log.info("Using " + stateManager.getClass().getName());
if (log.isInfoEnabled()) {
log.info("Using {}", stateManager.getClass().getName());
}
}
private DistribStateManager createDistribStateManager() {

View File

@ -74,13 +74,15 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
// test didn't init, nothing to do
return;
}
log.info("-------------_ FINAL STATE --------------");
log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
ClusterState state = cluster.getClusterStateProvider().getClusterState();
for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
log.info("* Collection " + coll + " state: " + state.getCollection(coll));
if (log.isInfoEnabled()) {
log.info("-------------_ FINAL STATE --------------");
log.info("* Node values: {}", Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues())); // logOk
log.info("* Live nodes: {}", cluster.getClusterStateProvider().getLiveNodes()); // logOk
ClusterState state = cluster.getClusterStateProvider().getClusterState();
for (String coll : cluster.getSimClusterStateProvider().simListCollections()) {
log.info("* Collection {} state: {}", coll, state.getCollection(coll)); // logOk
}
}
shutdownCluster();
}
@ -95,8 +97,10 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
create.setMaxShardsPerNode(1);
create.process(solrClient);
log.info("Collection ready after " + CloudUtil.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
CloudUtil.clusterShape(1, 2, false, true)) + "ms");
if (log.isInfoEnabled()) {
log.info("Collection ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
CloudUtil.clusterShape(1, 2, false, true)));
}
String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode();
ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
@ -157,8 +161,10 @@ public class TestSimExecutePlanAction extends SimSolrCloudTestCase {
assertNotNull(response.get("success"));
}
log.info("Collection ready after " + CloudUtil.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
CloudUtil.clusterShape(1, 2, false, true)) + "ms");
if (log.isInfoEnabled()) {
log.info("Collection ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
CloudUtil.clusterShape(1, 2, false, true)));
}
}
@Test

View File

@ -124,7 +124,9 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
long batchSize = BATCH_SIZE;
for (long i = 0; i < NUM_BATCHES; i++) {
addDocs(collectionName, i * batchSize, batchSize);
log.info(String.format(Locale.ROOT, "#### Total docs so far: %,d", ((i + 1) * batchSize)));
if (log.isInfoEnabled()) {
log.info(String.format(Locale.ROOT, "#### Total docs so far: %,d", ((i + 1) * batchSize))); // logOk
}
timeSource.sleep(waitForSeconds);
}
timeSource.sleep(60000);

View File

@ -209,8 +209,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
create.setCreateNodeSet(String.join(",", nodes));
create.process(solrClient);
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
CloudUtil.clusterShape(5, 15, false, true)) + "ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
CloudUtil.clusterShape(5, 15, false, true)));
}
int KILL_NODES = 8;
// kill off a number of nodes
@ -218,10 +220,11 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
cluster.simRemoveNode(nodes.get(i), false);
}
// should fully recover
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 90 * KILL_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(5, 15, false, true)) + "ms");
log.info("OP COUNTS: " + cluster.simGetOpCounts());
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 90 * KILL_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(5, 15, false, true)));
log.info("OP COUNTS: {}", cluster.simGetOpCounts()); // logOk
}
long moveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
// simulate a number of flaky nodes
@ -252,10 +255,15 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
fail("did not finish processing all events in time: started=" + triggerStartedCount.get() + ", finished=" + triggerFinishedCount.get());
}
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
CloudUtil.clusterShape(5, 15, false, true)) + "ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 30 * nodes.size(), TimeUnit.SECONDS,
CloudUtil.clusterShape(5, 15, false, true)));
}
long newMoveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
log.info("==== Flaky replicas: {}. Additional MOVEREPLICA count: {}", flakyReplicas, (newMoveReplicaOps - moveReplicaOps));
if (log.isInfoEnabled()) {
log.info("==== Flaky replicas: {}. Additional MOVEREPLICA count: {}", flakyReplicas, (newMoveReplicaOps - moveReplicaOps));
}
// flaky nodes lead to a number of MOVEREPLICA that is non-zero but lower than the number of flaky replicas
assertTrue("there should be new MOVERPLICA ops", newMoveReplicaOps - moveReplicaOps > 0);
assertTrue("there should be less than flakyReplicas=" + flakyReplicas + " MOVEREPLICA ops",
@ -328,8 +336,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
create.setAutoAddReplicas(false);
create.process(solrClient);
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 90 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 90 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)));
}
// start adding nodes
int numAddNode = NUM_NODES / 5;
@ -339,7 +349,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
}
// wait until at least one event is generated
assertTrue("Trigger did not start even after await()ing an excessive amount of time",
triggerStartedLatch.await(60, TimeUnit.SECONDS));
triggerStartedLatch.await(60, TimeUnit.SECONDS));
// wait until started == finished
TimeOut timeOut = new TimeOut(45 * waitForSeconds * NUM_NODES, TimeUnit.SECONDS, cluster.getTimeSource());
@ -378,8 +388,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
log.info("1st check: lastNumOps (MOVEREPLICA) = {}", lastNumOps);
assertTrue("no MOVEREPLICA ops?", lastNumOps > 0);
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)));
}
int count = 1000;
SolrInputDocument finishedEvent = null;
@ -387,7 +399,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
log.info("2nd check: lastNumOps (MOVEREPLICA) = {}", lastNumOps);
while (count-- > 0) {
cluster.getTimeSource().sleep(10000);
if (cluster.simGetOpCount("MOVEREPLICA") < 2) {
log.info("MOVEREPLICA < 2");
continue;
@ -415,10 +427,12 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
}
assertNotNull("did not finish processing changes", finishedEvent);
long delta = (Long)finishedEvent.getFieldValue("event.time_l") - (Long)startedEvent.getFieldValue("event.time_l");
log.info("#### System stabilized after " + TimeUnit.NANOSECONDS.toMillis(delta) + " ms");
long delta = (Long) finishedEvent.getFieldValue("event.time_l") - (Long) startedEvent.getFieldValue("event.time_l");
if (log.isInfoEnabled()) {
log.info("#### System stabilized after {} ms", TimeUnit.NANOSECONDS.toMillis(delta));
}
assertTrue("unexpected number of MOVEREPLICA ops: " + cluster.simGetOpCount("MOVEREPLICA"),
cluster.simGetOpCount("MOVEREPLICA") > 1);
cluster.simGetOpCount("MOVEREPLICA") > 1);
}
@Test
@ -496,7 +510,9 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
}
log.info("===== RESULTS ======");
log.info("waitFor\tdelay\tSTRT\tIGN\toSTRT\toIGN\tmin\tmax\tmean\tstdev\tvar");
results.forEach(s -> log.info(s));
if (log.isInfoEnabled()) {
results.forEach(s -> log.info(s));
}
}
private long doTestNodeLost(int waitFor, long killDelay, int minIgnored) throws Exception {
@ -537,8 +553,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
create.setAutoAddReplicas(false);
create.process(solrClient);
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 60 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 60 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)));
}
// start killing nodes
int numNodes = NUM_NODES / 5;
@ -587,8 +605,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
if (triggerStartedCount.get() == triggerFinishedCount.get()) {
break;
}
log.debug("started=" + triggerStartedCount.get() + ", finished=" + triggerFinishedCount.get() +
", failed=" + listenerEvents.size());
if (log.isDebugEnabled()) {
log.debug("started={}, finished={}, failed={}", triggerStartedCount.get(), triggerFinishedCount.get()
, listenerEvents.size());
}
timeOut.sleep(1000);
}
if (timeOut.hasTimedOut()) {
@ -621,8 +641,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
if (listenerEvents.isEmpty()) {
// no failed movements - verify collection shape
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
CloudUtil.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)));
}
} else {
cluster.getTimeSource().sleep(NUM_NODES * 100);
}
@ -659,7 +681,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
delta = (Long) finishedEvent.getFieldValue("event.time_l")
- (Long) startedEvent.getFieldValue("event.time_l");
delta = TimeUnit.NANOSECONDS.toMillis(delta);
log.info("#### System stabilized after " + delta + " ms");
log.info("#### System stabilized after {} ms", delta);
}
long ops = cluster.simGetOpCount("MOVEREPLICA");
long expectedMinOps = 40;
@ -680,8 +702,10 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
"conf", 2, 10);
create.process(solrClient);
log.info("Ready after " + CloudUtil.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
CloudUtil.clusterShape(2, 10, false, true)) + " ms");
if (log.isInfoEnabled()) {
log.info("Ready after {} ms", CloudUtil.waitForState(cluster, collectionName, 300, TimeUnit.SECONDS,
CloudUtil.clusterShape(2, 10, false, true)));
}
// collect the node names for shard1
Set<String> nodes = new HashSet<>();
@ -811,7 +835,9 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
Map<String, Number> updatedFreedisk2 = getFreeDiskPerNode(nodes);
double delta2 = getDeltaFreeDiskBytes(initialFreedisk, updatedFreedisk2);
// 0 docs - initial freedisk
log.info(cluster.dumpClusterState(true));
if (log.isInfoEnabled()) {
log.info(cluster.dumpClusterState(true));
}
assertEquals(0.0, delta2, delta2 * 0.1);
// test bulk update
@ -840,7 +866,9 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
Map<String, Object> values = cluster.getNodeStateProvider().getNodeValues(node, Arrays.asList(Variable.Type.FREEDISK.tagName));
freediskPerNode.put(node, (Number) values.get(Variable.Type.FREEDISK.tagName));
}
log.info("- freeDiskPerNode: " + Utils.toJSONString(freediskPerNode));
if (log.isInfoEnabled()) {
log.info("- freeDiskPerNode: {}", Utils.toJSONString(freediskPerNode));
}
return freediskPerNode;
}
}

View File

@ -256,19 +256,27 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
try {
if (lastActionExecutedAt.get() != 0) {
log.info("last action at " + lastActionExecutedAt.get() + " time = " + cluster.getTimeSource().getTimeNs());
if (log.isInfoEnabled()) {
log.info("last action at {} time = {}", lastActionExecutedAt.get(), cluster.getTimeSource().getTimeNs());
}
if (TimeUnit.NANOSECONDS.toMillis(cluster.getTimeSource().getTimeNs() - lastActionExecutedAt.get()) <
TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS) - DELTA_MS) {
log.info("action executed again before minimum wait time from {}", event.getSource());
if (log.isInfoEnabled()) {
log.info("action executed again before minimum wait time from {}", event.getSource());
}
fail("TriggerListener was fired before the throttling period");
}
}
if (onlyOnce.compareAndSet(false, true)) {
log.info("action executed from {}", event.getSource());
if (log.isInfoEnabled()) {
log.info("action executed from {}", event.getSource());
}
lastActionExecutedAt.set(cluster.getTimeSource().getTimeNs());
getTriggerFiredLatch().countDown();
} else {
log.info("action executed more than once from {}", event.getSource());
if (log.isInfoEnabled()) {
log.info("action executed more than once from {}", event.getSource());
}
fail("Trigger should not have fired more than once!");
}
} finally {

View File

@ -338,7 +338,7 @@ public class Policy implements MapWriter {
clusterPreferences,
lastComparison[0].node,
lastComparison[1].node,
Utils.writeJson(m, new StringWriter(), true).toString(), exc.toString()); // logOK
Utils.writeJson(m, new StringWriter(), true).toString(), exc.toString()); // logOk
} catch (IOException e1) {
//
}

View File

@ -181,8 +181,7 @@ public class PolicyHelper {
String errorId = "AutoScaling.error.diagnostics." + System.nanoTime();
Policy.Session sessionCopy = suggester.session;
log.error("errorId : {} {}", errorId
, handleExp(log, "", () -> Utils.writeJson(getDiagnostics(sessionCopy), new StringWriter(), true).toString())
); // logOK
, handleExp(log, "", () -> Utils.writeJson(getDiagnostics(sessionCopy), new StringWriter(), true).toString())); // logOk
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, " No node can satisfy the rules " +
Utils.toJSONString(Utils.getDeepCopy(session.expandedClauses, 4, true) + " More details from logs in node : "

View File

@ -139,7 +139,7 @@ public class RequestReplicaListTransformerGenerator {
if (choices.size() > 1) {
if (log.isDebugEnabled()) {
log.debug("Applying the following sorting preferences to replicas: {}",
Arrays.toString(replicaComp.getPreferenceRules().toArray())); // logOK
Arrays.toString(replicaComp.getPreferenceRules().toArray()));
}
// First, sort according to comparator rules.
@ -180,7 +180,7 @@ public class RequestReplicaListTransformerGenerator {
}
if (log.isDebugEnabled()) {
log.debug("Applied sorting preferences to replica list: {}", Arrays.toString(choices.toArray())); // logOK
log.debug("Applied sorting preferences to replica list: {}", Arrays.toString(choices.toArray()));
}
}
}

View File

@ -2139,7 +2139,7 @@ public class ZkStateReader implements SolrCloseable {
setIfNewer(Aliases.fromJSON(modAliasesJson, stat.getVersion()));
return;
} catch (KeeperException.BadVersionException e) {
log.debug("{}", e.toString(), e); // logOk
log.debug("{}", e, e);
log.warn("Couldn't save aliases due to race with another modification; will update and retry until timeout");
// considered a backoff here, but we really do want to compete strongly since the normal case is
// that we will do one update and succeed. This is left as a hot loop for limited tries intentionally.

View File

@ -775,9 +775,7 @@ public class Utils {
int statusCode = rsp.getStatusLine().getStatusCode();
if (statusCode != 200) {
try {
if (log.isErrorEnabled()) {
log.error("Failed a request to: {}, status: {}, body: {}", url, rsp.getStatusLine(), EntityUtils.toString(rsp.getEntity(), StandardCharsets.UTF_8)); // logOk
}
log.error("Failed a request to: {}, status: {}, body: {}", url, rsp.getStatusLine(), EntityUtils.toString(rsp.getEntity(), StandardCharsets.UTF_8)); // logOk
} catch (IOException e) {
log.error("could not print error", e);
}

View File

@ -286,7 +286,7 @@ public class TestSolrJErrorHandling extends SolrJettyTestBase {
try {
code = conn.getResponseCode();
} catch (Throwable th) {
log.error("ERROR DURING conn.getResponseCode():",th); //logOk
log.error("ERROR DURING conn.getResponseCode():",th);
}
/***

View File

@ -478,7 +478,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
shardAddresses.add(shardAddress);
}
if (log.isInfoEnabled()) {
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray())); // logOK
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray()));
}
// Make sure the distributed queries were directed to a single node only
@ -1063,7 +1063,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
}
assertTrue("No responses", shardAddresses.size() > 0);
if (log.isInfoEnabled()) {
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray())); // logOK
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray()));
}
}

View File

@ -466,7 +466,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
shardAddresses.add(shardAddress);
}
if (log.isInfoEnabled()) {
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray())); // logOK
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray()));
}
// Make sure the distributed queries were directed to a single node only
@ -1035,7 +1035,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
}
assertTrue("No responses", shardAddresses.size() > 0);
if (log.isInfoEnabled()) {
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray())); // logOk
log.info("Shards giving the response: {}", Arrays.toString(shardAddresses.toArray()));
}
}