mirror of https://github.com/apache/lucene.git
SOLR-6673: MDC based logging of collection, shard, replica, core
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1668992 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f1ab009e1f
commit
f678ac5d9d
|
@ -384,6 +384,9 @@ Other Changes
|
||||||
|
|
||||||
* SOLR-6414: Update to Hadoop 2.6.0. (Mark Miller)
|
* SOLR-6414: Update to Hadoop 2.6.0. (Mark Miller)
|
||||||
|
|
||||||
|
* SOLR-6673: MDC based logging of collection, shard, replica, core
|
||||||
|
(Ishan Chattopadhyaya , Noble Paul)
|
||||||
|
|
||||||
================== 5.0.0 ==================
|
================== 5.0.0 ==================
|
||||||
|
|
||||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||||
|
|
|
@ -15,6 +15,7 @@ import org.apache.solr.common.util.RetryUtil;
|
||||||
import org.apache.solr.common.util.RetryUtil.RetryCmd;
|
import org.apache.solr.common.util.RetryUtil.RetryCmd;
|
||||||
import org.apache.solr.core.CoreContainer;
|
import org.apache.solr.core.CoreContainer;
|
||||||
import org.apache.solr.core.SolrCore;
|
import org.apache.solr.core.SolrCore;
|
||||||
|
import org.apache.solr.logging.MDCUtils;
|
||||||
import org.apache.solr.search.SolrIndexSearcher;
|
import org.apache.solr.search.SolrIndexSearcher;
|
||||||
import org.apache.solr.update.UpdateLog;
|
import org.apache.solr.update.UpdateLog;
|
||||||
import org.apache.solr.util.RefCounted;
|
import org.apache.solr.util.RefCounted;
|
||||||
|
@ -24,10 +25,12 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
import org.slf4j.MDC;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
@ -113,7 +116,9 @@ class ShardLeaderElectionContextBase extends ElectionContext {
|
||||||
this.zkClient = zkStateReader.getZkClient();
|
this.zkClient = zkStateReader.getZkClient();
|
||||||
this.shardId = shardId;
|
this.shardId = shardId;
|
||||||
this.collection = collection;
|
this.collection = collection;
|
||||||
|
|
||||||
|
Map previousMDCContext = MDC.getCopyOfContextMap();
|
||||||
|
MDCUtils.setMDC(collection, shardId, null, null);
|
||||||
try {
|
try {
|
||||||
new ZkCmdExecutor(zkStateReader.getZkClient().getZkClientTimeout())
|
new ZkCmdExecutor(zkStateReader.getZkClient().getZkClientTimeout())
|
||||||
.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection,
|
.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection,
|
||||||
|
@ -123,6 +128,8 @@ class ShardLeaderElectionContextBase extends ElectionContext {
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw new SolrException(ErrorCode.SERVER_ERROR, e);
|
throw new SolrException(ErrorCode.SERVER_ERROR, e);
|
||||||
|
} finally {
|
||||||
|
MDCUtils.cleanupMDC(previousMDCContext);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -53,6 +53,7 @@ import org.apache.solr.core.CoreContainer;
|
||||||
import org.apache.solr.core.CoreDescriptor;
|
import org.apache.solr.core.CoreDescriptor;
|
||||||
import org.apache.solr.core.SolrCore;
|
import org.apache.solr.core.SolrCore;
|
||||||
import org.apache.solr.handler.component.ShardHandler;
|
import org.apache.solr.handler.component.ShardHandler;
|
||||||
|
import org.apache.solr.logging.MDCUtils;
|
||||||
import org.apache.solr.update.UpdateLog;
|
import org.apache.solr.update.UpdateLog;
|
||||||
import org.apache.solr.update.UpdateShardHandler;
|
import org.apache.solr.update.UpdateShardHandler;
|
||||||
import org.apache.zookeeper.CreateMode;
|
import org.apache.zookeeper.CreateMode;
|
||||||
|
@ -66,6 +67,7 @@ import org.apache.zookeeper.Watcher;
|
||||||
import org.apache.zookeeper.data.Stat;
|
import org.apache.zookeeper.data.Stat;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
import org.slf4j.MDC;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
|
@ -826,17 +828,19 @@ public final class ZkController {
|
||||||
*/
|
*/
|
||||||
public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores, boolean afterExpiration) throws Exception {
|
public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores, boolean afterExpiration) throws Exception {
|
||||||
// pre register has published our down state
|
// pre register has published our down state
|
||||||
|
|
||||||
final String baseUrl = getBaseUrl();
|
final String baseUrl = getBaseUrl();
|
||||||
|
|
||||||
final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
|
final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
|
||||||
final String collection = cloudDesc.getCollectionName();
|
final String collection = cloudDesc.getCollectionName();
|
||||||
|
|
||||||
|
Map previousMDCContext = MDC.getCopyOfContextMap();
|
||||||
|
MDCUtils.setCollection(collection);
|
||||||
|
|
||||||
final String coreZkNodeName = desc.getCloudDescriptor().getCoreNodeName();
|
final String coreZkNodeName = desc.getCloudDescriptor().getCoreNodeName();
|
||||||
assert coreZkNodeName != null : "we should have a coreNodeName by now";
|
assert coreZkNodeName != null : "we should have a coreNodeName by now";
|
||||||
|
|
||||||
String shardId = cloudDesc.getShardId();
|
String shardId = cloudDesc.getShardId();
|
||||||
|
MDCUtils.setShard(shardId);
|
||||||
Map<String,Object> props = new HashMap<>();
|
Map<String,Object> props = new HashMap<>();
|
||||||
// we only put a subset of props into the leader node
|
// we only put a subset of props into the leader node
|
||||||
props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
|
props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
|
||||||
|
@ -852,67 +856,71 @@ public final class ZkController {
|
||||||
ZkNodeProps leaderProps = new ZkNodeProps(props);
|
ZkNodeProps leaderProps = new ZkNodeProps(props);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
// If we're a preferred leader, insert ourselves at the head of the queue
|
try {
|
||||||
boolean joinAtHead = false;
|
// If we're a preferred leader, insert ourselves at the head of the queue
|
||||||
Replica replica = zkStateReader.getClusterState().getReplica(desc.getCloudDescriptor().getCollectionName(), coreZkNodeName);
|
boolean joinAtHead = false;
|
||||||
if (replica != null) {
|
Replica replica = zkStateReader.getClusterState().getReplica(desc.getCloudDescriptor().getCollectionName(), coreZkNodeName);
|
||||||
joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
|
if (replica != null) {
|
||||||
|
joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
|
||||||
|
}
|
||||||
|
joinElection(desc, afterExpiration, joinAtHead);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
// Restore the interrupted status
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
|
||||||
|
} catch (KeeperException | IOException e) {
|
||||||
|
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
|
||||||
}
|
}
|
||||||
joinElection(desc, afterExpiration, joinAtHead);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
// Restore the interrupted status
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
|
|
||||||
} catch (KeeperException | IOException e) {
|
|
||||||
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
// in this case, we want to wait for the leader as long as the leader might
|
// in this case, we want to wait for the leader as long as the leader might
|
||||||
// wait for a vote, at least - but also long enough that a large cluster has
|
// wait for a vote, at least - but also long enough that a large cluster has
|
||||||
// time to get its act together
|
// time to get its act together
|
||||||
String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
|
String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
|
||||||
|
|
||||||
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
|
|
||||||
log.info("We are " + ourUrl + " and leader is " + leaderUrl);
|
|
||||||
boolean isLeader = leaderUrl.equals(ourUrl);
|
|
||||||
|
|
||||||
try (SolrCore core = cc.getCore(desc.getName())) {
|
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
|
||||||
|
log.info("We are " + ourUrl + " and leader is " + leaderUrl);
|
||||||
|
boolean isLeader = leaderUrl.equals(ourUrl);
|
||||||
|
|
||||||
// recover from local transaction log and wait for it to complete before
|
try (SolrCore core = cc.getCore(desc.getName())) {
|
||||||
// going active
|
|
||||||
// TODO: should this be moved to another thread? To recoveryStrat?
|
|
||||||
// TODO: should this actually be done earlier, before (or as part of)
|
|
||||||
// leader election perhaps?
|
|
||||||
|
|
||||||
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
|
// recover from local transaction log and wait for it to complete before
|
||||||
if (!core.isReloaded() && ulog != null) {
|
// going active
|
||||||
// disable recovery in case shard is in construction state (for shard splits)
|
// TODO: should this be moved to another thread? To recoveryStrat?
|
||||||
Slice slice = getClusterState().getSlice(collection, shardId);
|
// TODO: should this actually be done earlier, before (or as part of)
|
||||||
if (!Slice.CONSTRUCTION.equals(slice.getState()) || !isLeader) {
|
// leader election perhaps?
|
||||||
Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
|
|
||||||
.getUpdateLog().recoverFromLog();
|
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
|
||||||
if (recoveryFuture != null) {
|
if (!core.isReloaded() && ulog != null) {
|
||||||
log.info("Replaying tlog for "+ourUrl+" during startup... NOTE: This can take a while.");
|
// disable recovery in case shard is in construction state (for shard splits)
|
||||||
recoveryFuture.get(); // NOTE: this could potentially block for
|
Slice slice = getClusterState().getSlice(collection, shardId);
|
||||||
// minutes or more!
|
if (!Slice.CONSTRUCTION.equals(slice.getState()) || !isLeader) {
|
||||||
// TODO: public as recovering in the mean time?
|
Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
|
||||||
// TODO: in the future we could do peersync in parallel with recoverFromLog
|
.getUpdateLog().recoverFromLog();
|
||||||
} else {
|
if (recoveryFuture != null) {
|
||||||
log.info("No LogReplay needed for core=" + core.getName() + " baseURL=" + baseUrl);
|
log.info("Replaying tlog for " + ourUrl + " during startup... NOTE: This can take a while.");
|
||||||
|
recoveryFuture.get(); // NOTE: this could potentially block for
|
||||||
|
// minutes or more!
|
||||||
|
// 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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
|
||||||
|
collection, coreZkNodeName, shardId, leaderProps, core, cc);
|
||||||
|
if (!didRecovery) {
|
||||||
|
publish(desc, ZkStateReader.ACTIVE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
|
|
||||||
collection, coreZkNodeName, shardId, leaderProps, core, cc);
|
|
||||||
if (!didRecovery) {
|
|
||||||
publish(desc, ZkStateReader.ACTIVE);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// make sure we have an update cluster state right away
|
||||||
|
zkStateReader.updateClusterState(true);
|
||||||
|
return shardId;
|
||||||
|
} finally {
|
||||||
|
MDCUtils.cleanupMDC(previousMDCContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
// make sure we have an update cluster state right away
|
|
||||||
zkStateReader.updateClusterState(true);
|
|
||||||
return shardId;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// timeoutms is the timeout for the first call to get the leader - there is then
|
// timeoutms is the timeout for the first call to get the leader - there is then
|
||||||
|
@ -1110,74 +1118,85 @@ public final class ZkController {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
String collection = cd.getCloudDescriptor().getCollectionName();
|
String collection = cd.getCloudDescriptor().getCollectionName();
|
||||||
log.info("publishing core={} state={} collection={}", cd.getName(), state, collection);
|
|
||||||
//System.out.println(Thread.currentThread().getStackTrace()[3]);
|
Map previousMDCContext = MDC.getCopyOfContextMap();
|
||||||
Integer numShards = cd.getCloudDescriptor().getNumShards();
|
MDCUtils.setCollection(collection);
|
||||||
if (numShards == null) { //XXX sys prop hack
|
|
||||||
log.info("numShards not found on descriptor - reading it from system property");
|
try {
|
||||||
numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP);
|
if (cd != null && cd.getName() != null)
|
||||||
}
|
MDCUtils.setCore(cd.getName());
|
||||||
|
log.info("publishing core={} state={} collection={}", cd.getName(), state, collection);
|
||||||
assert collection != null && collection.length() > 0;
|
//System.out.println(Thread.currentThread().getStackTrace()[3]);
|
||||||
|
Integer numShards = cd.getCloudDescriptor().getNumShards();
|
||||||
String shardId = cd.getCloudDescriptor().getShardId();
|
if (numShards == null) { //XXX sys prop hack
|
||||||
String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
|
log.info("numShards not found on descriptor - reading it from system property");
|
||||||
// If the leader initiated recovery, then verify that this replica has performed
|
numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP);
|
||||||
// recovery as requested before becoming active; don't even look at lirState if going down
|
}
|
||||||
if (!ZkStateReader.DOWN.equals(state)) {
|
|
||||||
String lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreNodeName);
|
assert collection != null && collection.length() > 0;
|
||||||
if (lirState != null) {
|
|
||||||
if (ZkStateReader.ACTIVE.equals(state)) {
|
String shardId = cd.getCloudDescriptor().getShardId();
|
||||||
// trying to become active, so leader-initiated state must be recovering
|
MDCUtils.setShard(shardId);
|
||||||
if (ZkStateReader.RECOVERING.equals(lirState)) {
|
String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
|
||||||
updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, ZkStateReader.ACTIVE, null, true);
|
// If the leader initiated recovery, then verify that this replica has performed
|
||||||
} else if (ZkStateReader.DOWN.equals(lirState)) {
|
// recovery as requested before becoming active; don't even look at lirState if going down
|
||||||
throw new SolrException(ErrorCode.INVALID_STATE,
|
if (!ZkStateReader.DOWN.equals(state)) {
|
||||||
"Cannot publish state of core '"+cd.getName()+"' as active without recovering first!");
|
String lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreNodeName);
|
||||||
}
|
if (lirState != null) {
|
||||||
} else if (ZkStateReader.RECOVERING.equals(state)) {
|
if (ZkStateReader.ACTIVE.equals(state)) {
|
||||||
// if it is currently DOWN, then trying to enter into recovering state is good
|
// trying to become active, so leader-initiated state must be recovering
|
||||||
if (ZkStateReader.DOWN.equals(lirState)) {
|
if (ZkStateReader.RECOVERING.equals(lirState)) {
|
||||||
updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, ZkStateReader.RECOVERING, null, true);
|
updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, ZkStateReader.ACTIVE, null, true);
|
||||||
|
} else if (ZkStateReader.DOWN.equals(lirState)) {
|
||||||
|
throw new SolrException(ErrorCode.INVALID_STATE,
|
||||||
|
"Cannot publish state of core '" + cd.getName() + "' as active without recovering first!");
|
||||||
|
}
|
||||||
|
} else if (ZkStateReader.RECOVERING.equals(state)) {
|
||||||
|
// if it is currently DOWN, then trying to enter into recovering state is good
|
||||||
|
if (ZkStateReader.DOWN.equals(lirState)) {
|
||||||
|
updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, ZkStateReader.RECOVERING, null, true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
Map<String, Object> props = new HashMap<>();
|
||||||
Map<String, Object> props = new HashMap<>();
|
props.put(Overseer.QUEUE_OPERATION, "state");
|
||||||
props.put(Overseer.QUEUE_OPERATION, "state");
|
props.put(ZkStateReader.STATE_PROP, state);
|
||||||
props.put(ZkStateReader.STATE_PROP, state);
|
props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
|
||||||
props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
|
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
|
||||||
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
|
props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles());
|
||||||
props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles());
|
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
|
||||||
props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
|
props.put(ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId());
|
||||||
props.put(ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId());
|
props.put(ZkStateReader.COLLECTION_PROP, collection);
|
||||||
props.put(ZkStateReader.COLLECTION_PROP, collection);
|
if (numShards != null) {
|
||||||
if (numShards != null) {
|
props.put(ZkStateReader.NUM_SHARDS_PROP, numShards.toString());
|
||||||
props.put(ZkStateReader.NUM_SHARDS_PROP, numShards.toString());
|
}
|
||||||
}
|
if (coreNodeName != null) {
|
||||||
if (coreNodeName != null) {
|
props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
||||||
props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
}
|
||||||
}
|
|
||||||
|
if (ClusterStateUtil.isAutoAddReplicas(getZkStateReader(), collection)) {
|
||||||
if (ClusterStateUtil.isAutoAddReplicas(getZkStateReader(), collection)) {
|
try (SolrCore core = cc.getCore(cd.getName())) {
|
||||||
try (SolrCore core = cc.getCore(cd.getName())) {
|
if (core != null && core.getDirectoryFactory().isSharedStorage()) {
|
||||||
if (core != null && core.getDirectoryFactory().isSharedStorage()) {
|
props.put("dataDir", core.getDataDir());
|
||||||
props.put("dataDir", core.getDataDir());
|
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
|
||||||
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
|
if (ulog != null) {
|
||||||
if (ulog != null) {
|
props.put("ulogDir", ulog.getLogDir());
|
||||||
props.put("ulogDir", ulog.getLogDir());
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ZkNodeProps m = new ZkNodeProps(props);
|
||||||
|
|
||||||
|
if (updateLastState) {
|
||||||
|
cd.getCloudDescriptor().lastPublished = state;
|
||||||
|
}
|
||||||
|
overseerJobQueue.offer(ZkStateReader.toJSON(m));
|
||||||
|
} finally {
|
||||||
|
MDCUtils.cleanupMDC(previousMDCContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
ZkNodeProps m = new ZkNodeProps(props);
|
|
||||||
|
|
||||||
if (updateLastState) {
|
|
||||||
cd.getCloudDescriptor().lastPublished = state;
|
|
||||||
}
|
|
||||||
overseerJobQueue.offer(ZkStateReader.toJSON(m));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
|
private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
|
||||||
|
|
|
@ -0,0 +1,69 @@
|
||||||
|
package org.apache.solr.logging;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
|
import org.slf4j.MDC;
|
||||||
|
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
public class MDCUtils {
|
||||||
|
public static void cleanupMDC(Map previousMDCContext) {
|
||||||
|
if (previousMDCContext != null)
|
||||||
|
MDC.setContextMap(previousMDCContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setMDC (String collection, String shard, String replica, String core) {
|
||||||
|
setCollection(collection);
|
||||||
|
setShard(shard);
|
||||||
|
setReplica(replica);
|
||||||
|
setCore(core);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setCollection(String collection) {
|
||||||
|
if (collection != null)
|
||||||
|
MDC.put(COLLECTION_PROP, collection);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setShard(String shard) {
|
||||||
|
if (shard != null)
|
||||||
|
MDC.put(SHARD_ID_PROP, shard);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setReplica(String replica) {
|
||||||
|
if (replica != null)
|
||||||
|
MDC.put(REPLICA_PROP, replica);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setCore(String core) {
|
||||||
|
if (core != null)
|
||||||
|
MDC.put(CORE_NAME_PROP, core);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void clearMDC() {
|
||||||
|
MDC.remove(COLLECTION_PROP);
|
||||||
|
MDC.remove(CORE_NAME_PROP);
|
||||||
|
MDC.remove(REPLICA_PROP);
|
||||||
|
MDC.remove(SHARD_ID_PROP);
|
||||||
|
}
|
||||||
|
}
|
|
@ -60,6 +60,7 @@ import org.apache.solr.core.SolrCore;
|
||||||
import org.apache.solr.core.SolrResourceLoader;
|
import org.apache.solr.core.SolrResourceLoader;
|
||||||
import org.apache.solr.core.SolrXmlConfig;
|
import org.apache.solr.core.SolrXmlConfig;
|
||||||
import org.apache.solr.handler.ContentStreamHandlerBase;
|
import org.apache.solr.handler.ContentStreamHandlerBase;
|
||||||
|
import org.apache.solr.logging.MDCUtils;
|
||||||
import org.apache.solr.request.SolrQueryRequest;
|
import org.apache.solr.request.SolrQueryRequest;
|
||||||
import org.apache.solr.request.SolrQueryRequestBase;
|
import org.apache.solr.request.SolrQueryRequestBase;
|
||||||
import org.apache.solr.request.SolrRequestHandler;
|
import org.apache.solr.request.SolrRequestHandler;
|
||||||
|
@ -73,7 +74,6 @@ import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
|
||||||
import org.apache.solr.util.RTimer;
|
import org.apache.solr.util.RTimer;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import javax.servlet.FilterChain;
|
import javax.servlet.FilterChain;
|
||||||
import javax.servlet.FilterConfig;
|
import javax.servlet.FilterConfig;
|
||||||
import javax.servlet.ServletException;
|
import javax.servlet.ServletException;
|
||||||
|
@ -81,6 +81,7 @@ import javax.servlet.ServletRequest;
|
||||||
import javax.servlet.ServletResponse;
|
import javax.servlet.ServletResponse;
|
||||||
import javax.servlet.http.HttpServletRequest;
|
import javax.servlet.http.HttpServletRequest;
|
||||||
import javax.servlet.http.HttpServletResponse;
|
import javax.servlet.http.HttpServletResponse;
|
||||||
|
|
||||||
import java.io.ByteArrayInputStream;
|
import java.io.ByteArrayInputStream;
|
||||||
import java.io.EOFException;
|
import java.io.EOFException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -219,6 +220,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain, boolean retry) throws IOException, ServletException {
|
public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain, boolean retry) throws IOException, ServletException {
|
||||||
|
MDCUtils.clearMDC();
|
||||||
|
|
||||||
if (abortErrorMessage != null) {
|
if (abortErrorMessage != null) {
|
||||||
sendError((HttpServletResponse) response, 500, abortErrorMessage);
|
sendError((HttpServletResponse) response, 500, abortErrorMessage);
|
||||||
|
@ -305,11 +307,14 @@ public class SolrDispatchFilter extends BaseSolrFilter {
|
||||||
|
|
||||||
if (core != null) {
|
if (core != null) {
|
||||||
path = path.substring( idx );
|
path = path.substring( idx );
|
||||||
|
MDCUtils.setCore(core.getName());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (core == null) {
|
if (core == null) {
|
||||||
if (!cores.isZooKeeperAware() ) {
|
if (!cores.isZooKeeperAware() ) {
|
||||||
core = cores.getCore("");
|
core = cores.getCore("");
|
||||||
|
if (core != null)
|
||||||
|
MDCUtils.setCore(core.getName());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -321,6 +326,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
|
||||||
if (core != null) {
|
if (core != null) {
|
||||||
// we found a core, update the path
|
// we found a core, update the path
|
||||||
path = path.substring( idx );
|
path = path.substring( idx );
|
||||||
|
MDCUtils.setCore(core.getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
// if we couldn't find it locally, look on other nodes
|
// if we couldn't find it locally, look on other nodes
|
||||||
|
@ -355,6 +361,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
|
||||||
// try the default core
|
// try the default core
|
||||||
if (core == null) {
|
if (core == null) {
|
||||||
core = cores.getCore("");
|
core = cores.getCore("");
|
||||||
|
MDCUtils.setCore(core.getName());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -11,13 +11,18 @@ import org.apache.log4j.spi.LoggingEvent;
|
||||||
import org.apache.log4j.spi.ThrowableInformation;
|
import org.apache.log4j.spi.ThrowableInformation;
|
||||||
import org.apache.solr.cloud.ZkController;
|
import org.apache.solr.cloud.ZkController;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.cloud.ClusterState;
|
|
||||||
import org.apache.solr.common.cloud.DocCollection;
|
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
import org.apache.solr.core.SolrCore;
|
import org.apache.solr.core.SolrCore;
|
||||||
|
import org.apache.solr.logging.MDCUtils;
|
||||||
import org.apache.solr.request.SolrQueryRequest;
|
import org.apache.solr.request.SolrQueryRequest;
|
||||||
import org.apache.solr.request.SolrRequestInfo;
|
import org.apache.solr.request.SolrRequestInfo;
|
||||||
|
import org.slf4j.MDC;
|
||||||
|
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
|
||||||
|
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
@ -107,7 +112,7 @@ public class SolrLogLayout extends Layout {
|
||||||
sb.append(" T");
|
sb.append(" T");
|
||||||
sb.append(th.getId());
|
sb.append(th.getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String format(LoggingEvent event) {
|
public String format(LoggingEvent event) {
|
||||||
return _format(event);
|
return _format(event);
|
||||||
|
@ -183,7 +188,9 @@ public class SolrLogLayout extends Layout {
|
||||||
// useful for sequencing when looking at multiple parts of a log file, but
|
// useful for sequencing when looking at multiple parts of a log file, but
|
||||||
// ms since start should be fine.
|
// ms since start should be fine.
|
||||||
appendThread(sb, event);
|
appendThread(sb, event);
|
||||||
|
|
||||||
|
appendMDC(sb);
|
||||||
|
|
||||||
if (info != null) {
|
if (info != null) {
|
||||||
sb.append(' ').append(info.shortId); // core
|
sb.append(' ').append(info.shortId); // core
|
||||||
}
|
}
|
||||||
|
@ -361,4 +368,17 @@ public class SolrLogLayout extends Layout {
|
||||||
public boolean ignoresThrowable() {
|
public boolean ignoresThrowable() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private void appendMDC(StringBuilder sb) {
|
||||||
|
sb.append(" [" + getMDCValueOrEmpty(COLLECTION_PROP) + "] ");
|
||||||
|
sb.append("[" + getMDCValueOrEmpty(SHARD_ID_PROP) + "] ");
|
||||||
|
sb.append("[" + getMDCValueOrEmpty(REPLICA_PROP) + "] ");
|
||||||
|
sb.append("[" + getMDCValueOrEmpty(CORE_NAME_PROP)+"] ");
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getMDCValueOrEmpty(String key) {
|
||||||
|
String val = MDC.get(key);
|
||||||
|
return val==null? "": val;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,7 +5,7 @@ log4j.rootLogger=INFO, file, CONSOLE
|
||||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||||
|
|
||||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x \u2013 %m%n
|
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x [%X{collection}] [%X{shard}] [%X{replica}] [%X{core}] \u2013 %m%n
|
||||||
|
|
||||||
#- size rotation with log cleanup.
|
#- size rotation with log cleanup.
|
||||||
log4j.appender.file=org.apache.log4j.RollingFileAppender
|
log4j.appender.file=org.apache.log4j.RollingFileAppender
|
||||||
|
|
|
@ -19,18 +19,21 @@ package org.apache.solr.common;
|
||||||
|
|
||||||
import java.io.CharArrayWriter;
|
import java.io.CharArrayWriter;
|
||||||
import java.io.PrintWriter;
|
import java.io.PrintWriter;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
import org.apache.solr.common.util.NamedList;
|
import org.apache.solr.common.util.NamedList;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.MDC;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public class SolrException extends RuntimeException {
|
public class SolrException extends RuntimeException {
|
||||||
|
|
||||||
|
final private Map mdcContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This list of valid HTTP Status error codes that Solr may return in
|
* This list of valid HTTP Status error codes that Solr may return in
|
||||||
* the case of a "Server Side" error.
|
* the case of a "Server Side" error.
|
||||||
|
@ -65,15 +68,18 @@ public class SolrException extends RuntimeException {
|
||||||
public SolrException(ErrorCode code, String msg) {
|
public SolrException(ErrorCode code, String msg) {
|
||||||
super(msg);
|
super(msg);
|
||||||
this.code = code.code;
|
this.code = code.code;
|
||||||
|
this.mdcContext = MDC.getCopyOfContextMap();
|
||||||
}
|
}
|
||||||
public SolrException(ErrorCode code, String msg, Throwable th) {
|
public SolrException(ErrorCode code, String msg, Throwable th) {
|
||||||
super(msg, th);
|
super(msg, th);
|
||||||
this.code = code.code;
|
this.code = code.code;
|
||||||
|
this.mdcContext = MDC.getCopyOfContextMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
public SolrException(ErrorCode code, Throwable th) {
|
public SolrException(ErrorCode code, Throwable th) {
|
||||||
super(th);
|
super(th);
|
||||||
this.code = code.code;
|
this.code = code.code;
|
||||||
|
this.mdcContext = MDC.getCopyOfContextMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -84,6 +90,7 @@ public class SolrException extends RuntimeException {
|
||||||
protected SolrException(int code, String msg, Throwable th) {
|
protected SolrException(int code, String msg, Throwable th) {
|
||||||
super(msg, th);
|
super(msg, th);
|
||||||
this.code = code;
|
this.code = code;
|
||||||
|
this.mdcContext = MDC.getCopyOfContextMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
int code=0;
|
int code=0;
|
||||||
|
@ -205,4 +212,34 @@ public class SolrException extends RuntimeException {
|
||||||
return t;
|
return t;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void logInfoWithMdc(Logger logger, String msg) {
|
||||||
|
Map previousMdcContext = MDC.getCopyOfContextMap();
|
||||||
|
MDC.setContextMap(mdcContext);
|
||||||
|
try {
|
||||||
|
logger.info(msg);
|
||||||
|
} finally{
|
||||||
|
MDC.setContextMap(previousMdcContext);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void logDebugWithMdc(Logger logger, String msg) {
|
||||||
|
Map previousMdcContext = MDC.getCopyOfContextMap();
|
||||||
|
MDC.setContextMap(mdcContext);
|
||||||
|
try {
|
||||||
|
logger.debug(msg);
|
||||||
|
} finally{
|
||||||
|
MDC.setContextMap(previousMdcContext);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void logWarnWithMdc(Logger logger, String msg) {
|
||||||
|
Map previousMdcContext = MDC.getCopyOfContextMap();
|
||||||
|
MDC.setContextMap(mdcContext);
|
||||||
|
try {
|
||||||
|
logger.warn(msg);
|
||||||
|
} finally{
|
||||||
|
MDC.setContextMap(previousMdcContext);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue