SOLR-5563: Quieten down SolrCloud logging

This commit is contained in:
Alan Woodward 2016-09-22 09:41:42 +01:00
parent f87276e865
commit a002aa5248
15 changed files with 99 additions and 115 deletions

View File

@ -161,6 +161,9 @@ Other Changes
"generate-website-quickstart" to convert the bundled version of the tutorial into one suitable
for the website.
* SOLR-5563: Move lots of SolrCloud logging from 'info' to 'debug' (janhoy, Alan
Woodward)
================== 6.2.1 ==================
Bug Fixes

View File

@ -175,7 +175,7 @@ public class CreateCollectionCmd implements Cmd {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not fully create collection: " + collectionName);
if (nodeList.isEmpty()) {
log.info("Finished create command for collection: {}", collectionName);
log.debug("Finished create command for collection: {}", collectionName);
return;
}
@ -183,14 +183,14 @@ public class CreateCollectionCmd implements Cmd {
Map<String, String> requestMap = new HashMap<>();
log.info(formatString("Creating SolrCores for new collection {0}, shardNames {1} , replicationFactor : {2}",
log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , replicationFactor : {2}",
collectionName, shardNames, repFactor));
Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
for (Map.Entry<ReplicaAssigner.Position, String> e : positionVsNodes.entrySet()) {
ReplicaAssigner.Position position = e.getKey();
String nodeName = e.getValue();
String coreName = collectionName + "_" + position.shard + "_replica" + (position.index + 1);
log.info(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
, coreName, position.shard, collectionName, nodeName));
@ -256,10 +256,9 @@ public class CreateCollectionCmd implements Cmd {
// We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
// element, which may be interpreted by the user as a positive ack
ocmh.cleanupCollection(collectionName, new NamedList());
log.info("Cleaned up artifacts for failed create collection for [" + collectionName + "]");
log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
} else {
log.debug("Finished create command on all shards for collection: "
+ collectionName);
log.debug("Finished create command on all shards for collection: {}", collectionName);
}
} catch (SolrException ex) {
throw ex;

View File

@ -150,7 +150,7 @@ public class LeaderElector {
try {
String watchedNode = holdElectionPath + "/" + toWatch;
zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath, watchedNode, getSeq(context.leaderSeqPath), context), null, true);
log.info("Watching path {} to know if I could be the leader", watchedNode);
log.debug("Watching path {} to know if I could be the leader", watchedNode);
} catch (KeeperException.SessionExpiredException e) {
throw e;
} catch (KeeperException.NoNodeException e) {
@ -238,14 +238,14 @@ public class LeaderElector {
while (cont) {
try {
if(joinAtHead){
log.info("Node {} trying to join election at the head", id);
log.debug("Node {} trying to join election at the head", id);
List<String> nodes = OverseerTaskProcessor.getSortedElectionNodes(zkClient, shardsElectZkPath);
if(nodes.size() <2){
leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
CreateMode.EPHEMERAL_SEQUENTIAL, false);
} else {
String firstInLine = nodes.get(1);
log.info("The current head: {}", firstInLine);
log.debug("The current head: {}", firstInLine);
Matcher m = LEADER_SEQ.matcher(firstInLine);
if (!m.matches()) {
throw new IllegalStateException("Could not find regex match in:"
@ -259,7 +259,7 @@ public class LeaderElector {
CreateMode.EPHEMERAL_SEQUENTIAL, false);
}
log.info("Joined leadership election with path: {}", leaderSeqPath);
log.debug("Joined leadership election with path: {}", leaderSeqPath);
context.leaderSeqPath = leaderSeqPath;
cont = false;
} catch (ConnectionLossException e) {
@ -333,7 +333,7 @@ public class LeaderElector {
return;
}
if (canceled) {
log.info("This watcher is not active anymore {}", myNode);
log.debug("This watcher is not active anymore {}", myNode);
try {
zkClient.delete(myNode, -1, true);
} catch (KeeperException.NoNodeException nne) {

View File

@ -122,7 +122,7 @@ public class Overseer implements Closeable {
isLeader = amILeader(); // not a no, not a yes, try ask again
}
log.info("Starting to work on the main queue");
log.debug("Starting to work on the main queue");
try {
ZkStateWriter zkStateWriter = null;
ClusterState clusterState = null;
@ -152,7 +152,7 @@ public class Overseer implements Closeable {
boolean hadWorkItems = data != null;
while (data != null) {
final ZkNodeProps message = ZkNodeProps.load(data);
log.info("processMessage: workQueueSize: {}, message = {}", workQueue.getStats().getQueueLength(), message);
log.debug("processMessage: workQueueSize: {}, message = {}", workQueue.getStats().getQueueLength(), message);
// force flush to ZK after each message because there is no fallback if workQueue items
// are removed from workQueue but fail to be written to ZK
clusterState = processQueueItem(message, clusterState, zkStateWriter, false, null);
@ -182,8 +182,7 @@ public class Overseer implements Closeable {
head = stateUpdateQueue.peek(true);
} catch (KeeperException e) {
if (e.code() == KeeperException.Code.SESSIONEXPIRED) {
log.warn(
"Solr cannot talk to ZK, exiting Overseer main queue loop", e);
log.warn("Solr cannot talk to ZK, exiting Overseer main queue loop", e);
return;
}
log.error("Exception in Overseer main queue loop", e);
@ -198,7 +197,7 @@ public class Overseer implements Closeable {
while (head != null) {
byte[] data = head;
final ZkNodeProps message = ZkNodeProps.load(data);
log.info("processMessage: queueSize: {}, message = {} current state version: {}", stateUpdateQueue.getStats().getQueueLength(), message, clusterState.getZkClusterStateVersion());
log.debug("processMessage: queueSize: {}, message = {} current state version: {}", stateUpdateQueue.getStats().getQueueLength(), message, clusterState.getZkClusterStateVersion());
// we can batch here because workQueue is our fallback in case a ZK write failed
clusterState = processQueueItem(message, clusterState, zkStateWriter, true, new ZkStateWriter.ZkWriteCallback() {
@Override
@ -297,7 +296,7 @@ public class Overseer implements Closeable {
String id = (String) m.get("id");
if(overseerCollectionConfigSetProcessor.getId().equals(id)){
try {
log.info("I'm exiting , but I'm still the leader");
log.warn("I'm exiting, but I'm still the leader");
zkClient.delete(path,stat.getVersion(),true);
} catch (KeeperException.BadVersionException e) {
//no problem ignore it some other Overseer has already taken over
@ -306,7 +305,7 @@ public class Overseer implements Closeable {
}
} else{
log.info("somebody else has already taken up the overseer position");
log.debug("somebody else has already taken up the overseer position");
}
} finally {
//if I am not shutting down, Then I need to rejoin election
@ -406,9 +405,7 @@ public class Overseer implements Closeable {
if (e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.error("", e);
return LeaderStatus.DONT_KNOW;
} else if (e.code() == KeeperException.Code.SESSIONEXPIRED) {
log.info("", e);
} else {
} else if (e.code() != KeeperException.Code.SESSIONEXPIRED) {
log.warn("", e);
}
} catch (InterruptedException e) {
@ -546,7 +543,7 @@ public class Overseer implements Closeable {
}
public synchronized void close() {
if (closed) return;
if (closed || id == null) return;
log.info("Overseer (id=" + id + ") closing");
doClose();

View File

@ -98,7 +98,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
this.waitAfterExpiration = config.getAutoReplicaFailoverWaitAfterExpiration();
int badNodeExpiration = config.getAutoReplicaFailoverBadNodeExpiration();
log.info(
log.debug(
"Starting "
+ this.getClass().getSimpleName()
+ " autoReplicaFailoverWorkLoopDelay={} autoReplicaFailoverWaitAfterExpiration={} autoReplicaFailoverBadNodeExpiration={}",

View File

@ -212,7 +212,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
@Override
@SuppressWarnings("unchecked")
public SolrResponse processMessage(ZkNodeProps message, String operation) {
log.info("OverseerCollectionMessageHandler.processMessage : "+ operation + " , "+ message.toString());
log.debug("OverseerCollectionMessageHandler.processMessage : "+ operation + " , "+ message.toString());
NamedList results = new NamedList();
try {
@ -480,7 +480,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
void commit(NamedList results, String slice, Replica parentShardLeader) {
log.info("Calling soft commit to make sub shard updates visible");
log.debug("Calling soft commit to make sub shard updates visible");
String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
// HttpShardHandler is hard coded to send a QueryRequest hence we go direct
// and we force open a searcher so that we have documents to show upon switching states
@ -537,7 +537,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
void waitForNewShard(String collectionName, String sliceName) throws KeeperException, InterruptedException {
log.info("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
RTimer timer = new RTimer();
int retryCount = 320;
while (retryCount-- > 0) {
@ -548,7 +548,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
Slice slice = collection.getSlice(sliceName);
if (slice != null) {
log.info("Waited for {}ms for slice {} of collection {} to be available",
log.debug("Waited for {}ms for slice {} of collection {} to be available",
timer.getTime(), sliceName, collectionName);
return;
}
@ -808,7 +808,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
if (configName != null) {
String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
log.info("creating collections conf node {} ", collDir);
log.debug("creating collections conf node {} ", collDir);
byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
if (zkStateReader.getZkClient().exists(collDir, true)) {
zkStateReader.getZkClient().setData(collDir, data, true);

View File

@ -144,7 +144,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
@Override
public void run() {
log.info("Process current queue of overseer operations");
log.debug("Process current queue of overseer operations");
LeaderStatus isLeader = amILeader();
while (isLeader == LeaderStatus.DONT_KNOW) {
log.debug("am_i_leader unclear {}", isLeader);
@ -290,7 +290,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
Thread.currentThread().interrupt();
continue;
}
log.info(messageHandler.getName() + ": Get the message id:" + head.getId() + " message:" + message.toString());
log.debug(messageHandler.getName() + ": Get the message id:" + head.getId() + " message:" + message.toString());
Runner runner = new Runner(messageHandler, message,
operation, head, lock);
tpe.execute(runner);
@ -393,9 +393,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
if (e.code() == KeeperException.Code.CONNECTIONLOSS) {
log.error("", e);
return LeaderStatus.DONT_KNOW;
} else if (e.code() == KeeperException.Code.SESSIONEXPIRED) {
log.info("", e);
} else {
} else if (e.code() != KeeperException.Code.SESSIONEXPIRED) {
log.warn("", e);
}
} catch (InterruptedException e) {
@ -486,7 +484,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
log.debug("Marked task [{}] as completed.", head.getId());
printTrackingMaps();
log.info(messageHandler.getName() + ": Message id:" + head.getId() +
log.debug(messageHandler.getName() + ": Message id:" + head.getId() +
" complete, response:" + response.getResponse().toString());
success = true;
} catch (KeeperException e) {

View File

@ -792,7 +792,7 @@ public class ZkController {
if (!SolrZkClient.containsChroot(zkHost)) {
return true;
}
log.info("zkHost includes chroot");
log.trace("zkHost includes chroot");
String chrootPath = zkHost.substring(zkHost.indexOf("/"), zkHost.length());
SolrZkClient tmpClient = new SolrZkClient(zkHost.substring(0,
@ -870,10 +870,8 @@ public class ZkController {
props.put(ZkStateReader.CORE_NAME_PROP, coreName);
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
if (log.isInfoEnabled()) {
log.info("Register replica - core:" + coreName + " address:" + baseUrl + " collection:"
+ cloudDesc.getCollectionName() + " shard:" + shardId);
}
log.debug("Register replica - core:{} address:{} collection:{} shard:{}",
coreName, baseUrl, cloudDesc.getCollectionName(), shardId);
ZkNodeProps leaderProps = new ZkNodeProps(props);
@ -900,7 +898,7 @@ public class ZkController {
String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
log.info("We are " + ourUrl + " and leader is " + leaderUrl);
log.debug("We are " + ourUrl + " and leader is " + leaderUrl);
boolean isLeader = leaderUrl.equals(ourUrl);
try (SolrCore core = cc.getCore(desc.getName())) {
@ -926,7 +924,7 @@ public class ZkController {
// TODO: public as recovering in the mean time?
// TODO: in the future we could do peersync in parallel with recoverFromLog
} else {
log.info("No LogReplay needed for core=" + core.getName() + " baseURL=" + baseUrl);
log.debug("No LogReplay needed for core={} baseURL={}", core.getName(), baseUrl);
}
}
}
@ -1148,11 +1146,11 @@ public class ZkController {
try {
String collection = cd.getCloudDescriptor().getCollectionName();
log.info("publishing state={}", state.toString());
log.debug("publishing state={}", state.toString());
// System.out.println(Thread.currentThread().getStackTrace()[3]);
Integer numShards = cd.getCloudDescriptor().getNumShards();
if (numShards == null) { // XXX sys prop hack
log.info("numShards not found on descriptor - reading it from system property");
log.debug("numShards not found on descriptor - reading it from system property");
numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP);
}
@ -1278,12 +1276,12 @@ public class ZkController {
public void createCollectionZkNode(CloudDescriptor cd) {
String collection = cd.getCollectionName();
log.info("Check for collection zkNode:" + collection);
log.debug("Check for collection zkNode:" + collection);
String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
try {
if (!zkClient.exists(collectionPath, true)) {
log.info("Creating collection in ZooKeeper:" + collection);
log.debug("Creating collection in ZooKeeper:" + collection);
try {
Map<String, Object> collectionProps = new HashMap<>();
@ -1335,7 +1333,7 @@ public class ZkController {
}
}
} else {
log.info("Collection zkNode exists");
log.debug("Collection zkNode exists");
}
} catch (KeeperException e) {
@ -1356,7 +1354,7 @@ public class ZkController {
Map<String, Object> collectionProps) throws KeeperException,
InterruptedException {
// check for configName
log.info("Looking for collection configName");
log.debug("Looking for collection configName");
List<String> configNames = null;
int retry = 1;
int retryLimt = 6;
@ -1417,7 +1415,7 @@ public class ZkController {
private void waitForCoreNodeName(CoreDescriptor descriptor) {
int retryCount = 320;
log.info("look for our core node name");
log.debug("look for our core node name");
while (retryCount-- > 0) {
Map<String, Slice> slicesMap = zkStateReader.getClusterState()
.getSlicesMap(descriptor.getCloudDescriptor().getCollectionName());
@ -1450,7 +1448,7 @@ public class ZkController {
}
private void waitForShardId(CoreDescriptor cd) {
log.info("waiting to find shard id in clusterstate for " + cd.getName());
log.debug("waiting to find shard id in clusterstate for " + cd.getName());
int retryCount = 320;
while (retryCount-- > 0) {
final String shardId = zkStateReader.getClusterState().getShardId(cd.getCollectionName(), getNodeName(), cd.getName());
@ -1499,7 +1497,7 @@ public class ZkController {
publish(cd, Replica.State.DOWN, false, true);
String collectionName = cd.getCloudDescriptor().getCollectionName();
DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
log.info(collection == null ?
log.debug(collection == null ?
"Collection {} not visible yet, but flagging it so a watch is registered when it becomes visible" :
"Registering watch for collection {}",
collectionName);
@ -1612,7 +1610,7 @@ public class ZkController {
}
if (lirState != null) {
log.info("Replica " + myCoreNodeName +
log.debug("Replica " + myCoreNodeName +
" is already in leader-initiated recovery, so not waiting for leader to see down state.");
} else {
@ -1680,9 +1678,7 @@ public class ZkController {
public static void linkConfSet(SolrZkClient zkClient, String collection, String confSetName) throws KeeperException, InterruptedException {
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
if (log.isInfoEnabled()) {
log.info("Load collection config from:" + path);
}
log.debug("Load collection config from:" + path);
byte[] data;
try {
data = zkClient.getData(path, null, null, true);
@ -2118,7 +2114,7 @@ public class ZkController {
zkClient.makePath(znodePath, znodeData, retryOnConnLoss);
}
}
log.info("Wrote {} to {}", state.toString(), znodePath);
log.debug("Wrote {} to {}", state.toString(), znodePath);
} catch (Exception exc) {
if (exc instanceof SolrException) {
throw (SolrException) exc;
@ -2214,7 +2210,7 @@ public class ZkController {
if (listener != null) {
synchronized (reconnectListeners) {
reconnectListeners.add(listener);
log.info("Added new OnReconnect listener "+listener);
log.debug("Added new OnReconnect listener "+listener);
}
}
}
@ -2229,7 +2225,7 @@ public class ZkController {
wasRemoved = reconnectListeners.remove(listener);
}
if (wasRemoved) {
log.info("Removed OnReconnect listener "+listener);
log.debug("Removed OnReconnect listener "+listener);
} else {
log.warn("Was asked to remove OnReconnect listener "+listener+
", but remove operation did not find it in the list of registered listeners.");
@ -2273,7 +2269,7 @@ public class ZkController {
} catch (KeeperException.NodeExistsException nee) {
try {
Stat stat = zkClient.exists(resourceLocation, null, true);
log.info("failed to set data version in zk is {} and expected version is {} ", stat.getVersion(), znodeVersion);
log.debug("failed to set data version in zk is {} and expected version is {} ", stat.getVersion(), znodeVersion);
} catch (Exception e1) {
log.warn("could not get stat");
}
@ -2337,11 +2333,11 @@ public class ZkController {
return;
}
if (listeners.remove(listener)) {
log.info("removed listener for config directory [{}]", confDir);
log.debug("removed listener for config directory [{}]", confDir);
}
if (listeners.isEmpty()) {
// no more listeners for this confDir, remove it from the map
log.info("No more listeners for config directory [{}]", confDir);
log.debug("No more listeners for config directory [{}]", confDir);
confDirectoryListeners.remove(confDir);
}
}
@ -2378,7 +2374,7 @@ public class ZkController {
assert Thread.holdsLock(confDirectoryListeners) : "confDirListeners lock not held by thread";
Set<Runnable> confDirListeners = confDirectoryListeners.get(confDir);
if (confDirListeners == null) {
log.info("watch zkdir {}" , confDir);
log.debug("watch zkdir {}" , confDir);
confDirListeners = new HashSet<>();
confDirectoryListeners.put(confDir, confDirListeners);
setConfWatcher(confDir, new WatcherImpl(confDir), null);
@ -2416,10 +2412,10 @@ public class ZkController {
resetWatcher = fireEventListeners(zkDir);
} finally {
if (Event.EventType.None.equals(event.getType())) {
log.info("A node got unwatched for {}", zkDir);
log.debug("A node got unwatched for {}", zkDir);
} else {
if (resetWatcher) setConfWatcher(zkDir, this, stat);
else log.info("A node got unwatched for {}", zkDir);
else log.debug("A node got unwatched for {}", zkDir);
}
}
}
@ -2429,7 +2425,7 @@ public class ZkController {
synchronized (confDirectoryListeners) {
// if this is not among directories to be watched then don't set the watcher anymore
if (!confDirectoryListeners.containsKey(zkDir)) {
log.info("Watcher on {} is removed ", zkDir);
log.debug("Watcher on {} is removed ", zkDir);
return false;
}
final Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
@ -2437,7 +2433,7 @@ public class ZkController {
final Set<Runnable> listenersCopy = new HashSet<>(listeners);
// run these in a separate thread because this can be long running
new Thread(() -> {
log.info("Running listeners for {}", zkDir);
log.debug("Running listeners for {}", zkDir);
for (final Runnable listener : listenersCopy) {
try {
listener.run();
@ -2526,9 +2522,9 @@ public class ZkController {
Overseer.getStateUpdateQueue(getZkClient()).offer(Utils.toJSON(m));
} catch (InterruptedException e) {
Thread.interrupted();
log.info("Publish node as down was interrupted.");
log.debug("Publish node as down was interrupted.");
} catch (Exception e) {
log.info("Could not publish node as down: " + e.getMessage());
log.warn("Could not publish node as down: " + e.getMessage());
}
}
}

View File

@ -50,7 +50,7 @@ public class ClusterStateMutator {
public ZkWriteCommand createCollection(ClusterState clusterState, ZkNodeProps message) {
String cName = message.getStr(NAME);
log.info("building a new cName: " + cName);
log.debug("building a new cName: " + cName);
if (clusterState.hasCollection(cName)) {
log.warn("Collection {} already exists. exit", cName);
return ZkStateWriter.NO_OP;

View File

@ -135,8 +135,8 @@ public class ReplicaMutator {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection/slice/replica " +
collectionName + "/" + sliceName + "/" + replicaName + " no action taken.");
}
log.info("Setting property " + property + " with value: " + propVal +
" for collection: " + collectionName + ". Full message: " + message);
log.info("Setting property {} with value {} for collection {}", property, propVal, collectionName);
log.debug("Full message: {}", message);
if (StringUtils.equalsIgnoreCase(replica.getStr(property), propVal)) return ZkStateWriter.NO_OP; // already the value we're going to set
// OK, there's no way we won't change the cluster state now
@ -184,13 +184,11 @@ public class ReplicaMutator {
collectionName + "/" + sliceName + "/" + replicaName + " no action taken.");
}
log.info("Deleting property " + property + " for collection: " + collectionName +
" slice " + sliceName + " replica " + replicaName + ". Full message: " + message);
log.info("Deleting property {} for collection: {} slice: {} replica: {}", property, collectionName, sliceName, replicaName);
log.debug("Full message: {}", message);
String curProp = replica.getStr(property);
if (curProp == null) return ZkStateWriter.NO_OP; // not there anyway, nothing to do.
log.info("Deleting property " + property + " for collection: " + collectionName +
" slice " + sliceName + " replica " + replicaName + ". Full message: " + message);
Slice slice = collection.getSlice(sliceName);
DocCollection newCollection = SliceMutator.updateReplica(collection,
slice, replicaName, unsetProperty(replica, property));
@ -209,7 +207,7 @@ public class ReplicaMutator {
final String cName = message.getStr(ZkStateReader.COLLECTION_PROP);
if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
Integer numShards = message.getInt(ZkStateReader.NUM_SHARDS_PROP, null);
log.info("Update state numShards={} message={}", numShards, message);
log.debug("Update state numShards={} message={}", numShards, message);
List<String> shardNames = new ArrayList<>();
@ -239,7 +237,7 @@ public class ReplicaMutator {
coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(collection,
message.getStr(ZkStateReader.NODE_NAME_PROP), message.getStr(ZkStateReader.CORE_NAME_PROP));
if (coreNodeName != null) {
log.info("node=" + coreNodeName + " is already registered");
log.debug("node=" + coreNodeName + " is already registered");
} else {
// if coreNodeName is null, auto assign one
coreNodeName = Assign.assignNode(collection);
@ -253,7 +251,7 @@ public class ReplicaMutator {
//get shardId from ClusterState
sliceName = ClusterStateMutator.getAssignedId(collection, coreNodeName);
if (sliceName != null) {
log.info("shard=" + sliceName + " is already registered");
log.debug("shard=" + sliceName + " is already registered");
}
}
if (sliceName == null) {
@ -261,7 +259,7 @@ public class ReplicaMutator {
if (collectionExists) {
// use existing numShards
numShards = collection.getSlices().size();
log.info("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards);
log.debug("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards);
}
sliceName = Assign.assignShard(collection, numShards);
log.info("Assigning new node to shard shard=" + sliceName);

View File

@ -221,17 +221,17 @@ public class ZkStateWriter {
if (c == null) {
// let's clean up the collections path for this collection
log.info("going to delete_collection {}", path);
log.debug("going to delete_collection {}", path);
reader.getZkClient().clean("/collections/" + name);
} else if (c.getStateFormat() > 1) {
byte[] data = Utils.toJSON(singletonMap(c.getName(), c));
if (reader.getZkClient().exists(path, true)) {
log.info("going to update_collection {} version: {}", path, c.getZNodeVersion());
log.debug("going to update_collection {} version: {}", path, c.getZNodeVersion());
Stat stat = reader.getZkClient().setData(path, data, c.getZNodeVersion(), true);
DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), stat.getVersion(), path);
clusterState = clusterState.copyWith(name, newCollection);
} else {
log.info("going to create_collection {}", path);
log.debug("going to create_collection {}", path);
reader.getZkClient().create(path, data, CreateMode.PERSISTENT, true);
DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), 0, path);
clusterState = clusterState.copyWith(name, newCollection);

View File

@ -219,7 +219,7 @@ public class CoreDescriptor {
cloudDesc = null;
}
log.info("Created CoreDescriptor: " + coreProperties);
log.debug("Created CoreDescriptor: " + coreProperties);
}
/**

View File

@ -105,15 +105,13 @@ public class ConnectionManager implements Watcher {
@Override
public void process(WatchedEvent event) {
if (event.getState() == AuthFailed || event.getState() == Disconnected || event.getState() == Expired) {
log.warn("Watcher " + this + " name:" + name + " got event " + event
+ " path:" + event.getPath() + " type:" + event.getType());
} else if (log.isInfoEnabled()) {
log.info("Watcher " + this + " name:" + name + " got event " + event
+ " path:" + event.getPath() + " type:" + event.getType());
log.warn("Watcher {} name: {} got event {} path: {} type: {}", this, name, event, event.getPath(), event.getType());
} else {
log.debug("Watcher {} name: {} got event {} path: {} type: {}", this, name, event, event.getPath(), event.getType());
}
if (isClosed) {
log.info("Client->ZooKeeper status change trigger but we are already closed");
log.debug("Client->ZooKeeper status change trigger but we are already closed");
return;
}
@ -216,7 +214,7 @@ public class ConnectionManager implements Watcher {
public synchronized void waitForConnected(long waitForConnection)
throws TimeoutException {
log.info("Waiting for client to connect to ZooKeeper");
log.debug("Waiting for client to connect to ZooKeeper");
long expire = System.nanoTime() + TimeUnit.NANOSECONDS.convert(waitForConnection, TimeUnit.MILLISECONDS);
long left = 1;
while (!connected && left > 0) {

View File

@ -209,7 +209,7 @@ public class SolrZkClient implements Closeable {
log.warn("VM param zkCredentialsProvider does not point to a class implementing ZkCredentialsProvider and with a non-arg constructor", t);
}
}
log.info("Using default ZkCredentialsProvider");
log.debug("Using default ZkCredentialsProvider");
return new DefaultZkCredentialsProvider();
}
@ -225,7 +225,7 @@ public class SolrZkClient implements Closeable {
log.warn("VM param zkACLProvider does not point to a class implementing ZkACLProvider and with a non-arg constructor", t);
}
}
log.info("Using default ZkACLProvider");
log.debug("Using default ZkACLProvider");
return new DefaultZkACLProvider();
}
@ -478,9 +478,7 @@ public class SolrZkClient implements Closeable {
*/
public void makePath(String path, byte[] data, CreateMode createMode,
Watcher watcher, boolean failOnExists, boolean retryOnConnLoss) throws KeeperException, InterruptedException {
if (log.isInfoEnabled()) {
log.info("makePath: " + path);
}
log.debug("makePath: {}", path);
boolean retry = true;
if (path.startsWith("/")) {
@ -564,10 +562,7 @@ public class SolrZkClient implements Closeable {
*/
public Stat setData(String path, File file, boolean retryOnConnLoss) throws IOException,
KeeperException, InterruptedException {
if (log.isInfoEnabled()) {
log.info("Write to ZooKeepeer " + file.getAbsolutePath() + " to " + path);
}
log.debug("Write to ZooKeeper: {} to {}", file.getAbsolutePath(), path);
byte[] data = FileUtils.readFileToByteArray(file);
return setData(path, data, retryOnConnLoss);
}
@ -768,7 +763,7 @@ public class SolrZkClient implements Closeable {
ZkMaintenanceUtils.traverseZkTree(this, root, ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, path -> {
try {
setACL(path, getZkACLProvider().getACLsToAdd(path), true);
log.info("Updated ACL on " + path);
log.debug("Updated ACL on {}", path);
} catch (NoNodeException e) {
// If a node was deleted, don't bother trying to set ACLs on it.
return;

View File

@ -288,7 +288,7 @@ public class ZkStateReader implements Closeable {
synchronized (getUpdateLock()) {
if (clusterState == null) {
LOG.info("ClusterState watchers have not been initialized");
LOG.warn("ClusterState watchers have not been initialized");
return;
}
@ -296,20 +296,20 @@ public class ZkStateReader implements Closeable {
if (ref == null || legacyCollectionStates.containsKey(collection)) {
// We either don't know anything about this collection (maybe it's new?) or it's legacy.
// First update the legacy cluster state.
LOG.info("Checking legacy cluster state for collection {}", collection);
LOG.debug("Checking legacy cluster state for collection {}", collection);
refreshLegacyClusterState(null);
if (!legacyCollectionStates.containsKey(collection)) {
// No dice, see if a new collection just got created.
LazyCollectionRef tryLazyCollection = new LazyCollectionRef(collection);
if (tryLazyCollection.get() != null) {
// What do you know, it exists!
LOG.info("Adding lazily-loaded reference for collection {}", collection);
LOG.debug("Adding lazily-loaded reference for collection {}", collection);
lazyCollectionStates.putIfAbsent(collection, tryLazyCollection);
constructState(Collections.singleton(collection));
}
}
} else if (ref.isLazilyLoaded()) {
LOG.info("Refreshing lazily-loaded state for collection {}", collection);
LOG.debug("Refreshing lazily-loaded state for collection {}", collection);
if (ref.get() != null) {
return;
}
@ -317,7 +317,7 @@ public class ZkStateReader implements Closeable {
refreshLegacyClusterState(null);
} else if (watchedCollectionStates.containsKey(collection)) {
// Exists as a watched collection, force a refresh.
LOG.info("Forcing refresh of watched collection state for {}", collection);
LOG.debug("Forcing refresh of watched collection state for {}", collection);
DocCollection newState = fetchCollectionState(collection, null);
if (updateWatchedCollection(collection, newState)) {
constructState(Collections.singleton(collection));
@ -404,7 +404,7 @@ public class ZkStateReader implements Closeable {
final Stat stat = new Stat();
final byte[] data = zkClient.getData(ALIASES, thisWatch, stat, true);
ZkStateReader.this.aliases = ClusterState.load(data);
LOG.info("New alias definition is: " + ZkStateReader.this.aliases.toString());
LOG.debug("New alias definition is: " + ZkStateReader.this.aliases.toString());
}
} catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
LOG.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
@ -872,11 +872,11 @@ public class ZkStateReader implements Closeable {
try {
byte[] data = zkClient.getData(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, new Stat(), true);
this.clusterProperties = (Map<String, Object>) Utils.fromJSON(data);
LOG.info("Loaded cluster properties: {}", this.clusterProperties);
LOG.debug("Loaded cluster properties: {}", this.clusterProperties);
return;
} catch (KeeperException.NoNodeException e) {
this.clusterProperties = Collections.emptyMap();
LOG.info("Loaded empty cluster properties");
LOG.debug("Loaded empty cluster properties");
// set an exists watch, and if the node has been created since the last call,
// read the data again
if (zkClient.exists(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, true) == null)
@ -952,7 +952,7 @@ public class ZkStateReader implements Closeable {
if (!collectionWatches.containsKey(coll)) {
// This collection is no longer interesting, stop watching.
LOG.info("Uninteresting collection {}", coll);
LOG.debug("Uninteresting collection {}", coll);
return;
}
@ -999,7 +999,7 @@ public class ZkStateReader implements Closeable {
return;
}
int liveNodesSize = ZkStateReader.this.clusterState == null ? 0 : ZkStateReader.this.clusterState.getLiveNodes().size();
LOG.info("A cluster state change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodesSize);
LOG.debug("A cluster state change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodesSize);
refreshAndWatch();
}
@ -1032,7 +1032,7 @@ public class ZkStateReader implements Closeable {
if (EventType.None.equals(event.getType())) {
return;
}
LOG.info("A collections change: [{}], has occurred - updating...", event);
LOG.debug("A collections change: [{}], has occurred - updating...", event);
refreshAndWatch();
synchronized (getUpdateLock()) {
constructState(Collections.emptySet());
@ -1065,7 +1065,7 @@ public class ZkStateReader implements Closeable {
if (EventType.None.equals(event.getType())) {
return;
}
LOG.info("A live node change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodes.size());
LOG.debug("A live node change: [{}], has occurred - updating... (live nodes size: [{}])", event, liveNodes.size());
refreshAndWatch();
}
@ -1286,7 +1286,7 @@ public class ZkStateReader implements Closeable {
private boolean updateWatchedCollection(String coll, DocCollection newState) {
if (newState == null) {
LOG.info("Deleting data for [{}]", coll);
LOG.debug("Removing cached collection state for [{}]", coll);
watchedCollectionStates.remove(coll);
return true;
}
@ -1300,7 +1300,7 @@ public class ZkStateReader implements Closeable {
DocCollection oldState = watchedCollectionStates.get(coll);
if (oldState == null) {
if (watchedCollectionStates.putIfAbsent(coll, newState) == null) {
LOG.info("Add data for [{}] ver [{}]", coll, newState.getZNodeVersion());
LOG.debug("Add data for [{}] ver [{}]", coll, newState.getZNodeVersion());
updated = true;
break;
}
@ -1312,7 +1312,7 @@ public class ZkStateReader implements Closeable {
break;
}
if (watchedCollectionStates.replace(coll, oldState, newState)) {
LOG.info("Updating data for [{}] from [{}] to [{}]", coll, oldState.getZNodeVersion(), newState.getZNodeVersion());
LOG.debug("Updating data for [{}] from [{}] to [{}]", coll, oldState.getZNodeVersion(), newState.getZNodeVersion());
updated = true;
break;
}
@ -1322,7 +1322,7 @@ public class ZkStateReader implements Closeable {
// Resolve race with unregisterCore.
if (!collectionWatches.containsKey(coll)) {
watchedCollectionStates.remove(coll);
LOG.info("Removing uninteresting collection [{}]", coll);
LOG.debug("Removing uninteresting collection [{}]", coll);
}
return updated;