SOLR-9906: SolrjNamedThreadFactory is deprecated in favor of SolrNamedThreadFactory. DefaultSolrThreadFactory is removed from solr-core in favor of SolrNamedThreadFactory in solrj package and all solr-core classes now use SolrNamedThreadFactory

This commit is contained in:
Shalin Shekhar Mangar 2020-04-13 08:16:35 +05:30
parent 4dece1ae17
commit 13f19f6555
87 changed files with 224 additions and 235 deletions

View File

@ -13,7 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
@defaultMessage Spawns threads with vague names; use a custom thread factory (Lucene's NamedThreadFactory, Solr's DefaultSolrThreadFactory) and name threads so that you can tell (by its name) which executor it is associated with
@defaultMessage Spawns threads with vague names; use a custom thread factory (Lucene's NamedThreadFactory, Solr's SolrNamedThreadFactory) and name threads so that you can tell (by its name) which executor it is associated with
java.util.concurrent.Executors#newFixedThreadPool(int)
java.util.concurrent.Executors#newSingleThreadExecutor()
java.util.concurrent.Executors#newCachedThreadPool()

View File

@ -13,7 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
@defaultMessage Spawns threads with vague names; use a custom thread factory (Lucene's NamedThreadFactory, Solr's DefaultSolrThreadFactory) and name threads so that you can tell (by its name) which executor it is associated with
@defaultMessage Spawns threads with vague names; use a custom thread factory (Lucene's NamedThreadFactory, Solr's SolrNamedThreadFactory) and name threads so that you can tell (by its name) which executor it is associated with
java.util.concurrent.Executors#newFixedThreadPool(int)
java.util.concurrent.Executors#newSingleThreadExecutor()
java.util.concurrent.Executors#newCachedThreadPool()

View File

@ -148,6 +148,10 @@ Other Changes
* SOLR-14386: Update Jetty to 9.4.27 and dropwizard-metrics version to 4.1.5 (Erick Erickson)
* SOLR-9906: SolrjNamedThreadFactory is deprecated in favor of SolrNamedThreadFactory. DefaultSolrThreadFactory is
removed from solr-core in favor of SolrNamedThreadFactory in solrj package and all solr-core classes now use
SolrNamedThreadFactory. (Andras Salamon, shalin)
================== 8.5.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -48,7 +48,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.handler.AnalyticsHandler;
import org.apache.solr.handler.component.AnalyticsComponent;
import org.apache.solr.response.AnalyticsShardResponseWriter;
@ -142,7 +142,7 @@ public class AnalyticsShardRequestManager {
* @throws IOException if an exception occurs while sending requests.
*/
private void streamFromShards() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("SolrAnalyticsStream"));
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("SolrAnalyticsStream"));
List<Future<SolrException>> futures = new ArrayList<>();
List<AnalyticsShardRequester> openers = new ArrayList<>();
for (String replicaUrl : replicaUrls) {

View File

@ -33,7 +33,7 @@ import io.prometheus.client.Collector;
import io.prometheus.client.Histogram;
import org.apache.solr.prometheus.exporter.SolrExporter;
import org.apache.solr.prometheus.scraper.Async;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -51,7 +51,7 @@ public class SchedulerMetricsCollector implements Closeable {
private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(
1,
new DefaultSolrThreadFactory("scheduled-metrics-collector"));
new SolrNamedThreadFactory("scheduled-metrics-collector"));
private final Executor executor;

View File

@ -39,7 +39,7 @@ import org.apache.solr.prometheus.collector.SchedulerMetricsCollector;
import org.apache.solr.prometheus.scraper.SolrCloudScraper;
import org.apache.solr.prometheus.scraper.SolrScraper;
import org.apache.solr.prometheus.scraper.SolrStandaloneScraper;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -105,11 +105,11 @@ public class SolrExporter {
this.metricCollectorExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(
numberThreads,
new DefaultSolrThreadFactory("solr-exporter-collectors"));
new SolrNamedThreadFactory("solr-exporter-collectors"));
this.requestExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(
numberThreads,
new DefaultSolrThreadFactory("solr-exporter-requests"));
new SolrNamedThreadFactory("solr-exporter-requests"));
this.solrScraper = createScraper(scrapeConfiguration, metricsConfiguration.getSettings());
this.metricsCollector = new MetricsCollectorFactory(metricCollectorExecutor, scrapeInterval, solrScraper, metricsConfiguration).create();

View File

@ -42,7 +42,7 @@ import org.apache.solr.prometheus.exporter.MetricsConfiguration;
import org.apache.solr.prometheus.exporter.PrometheusExporterSettings;
import org.apache.solr.prometheus.exporter.SolrClientFactory;
import org.apache.solr.prometheus.utils.Helpers;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -82,7 +82,7 @@ public class SolrCloudScraperTest extends PrometheusExporterTestBase {
@Before
public void setUp() throws Exception {
super.setUp();
executor = ExecutorUtil.newMDCAwareFixedThreadPool(25, new DefaultSolrThreadFactory("solr-cloud-scraper-tests"));
executor = ExecutorUtil.newMDCAwareFixedThreadPool(25, new SolrNamedThreadFactory("solr-cloud-scraper-tests"));
configuration = Helpers.loadConfiguration("conf/prometheus-solr-exporter-scraper-test-config.xml");
solrCloudScraper = createSolrCloudScraper();
}

View File

@ -34,7 +34,7 @@ import org.apache.solr.prometheus.PrometheusExporterTestBase;
import org.apache.solr.prometheus.collector.MetricSamples;
import org.apache.solr.prometheus.exporter.MetricsConfiguration;
import org.apache.solr.prometheus.utils.Helpers;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RestTestBase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -64,7 +64,7 @@ public class SolrStandaloneScraperTest extends RestTestBase {
true,
null);
executor = ExecutorUtil.newMDCAwareFixedThreadPool(25, new DefaultSolrThreadFactory("solr-cloud-scraper-tests"));
executor = ExecutorUtil.newMDCAwareFixedThreadPool(25, new SolrNamedThreadFactory("solr-cloud-scraper-tests"));
configuration = Helpers.loadConfiguration("conf/prometheus-solr-exporter-scraper-test-config.xml");
solrClient = getHttpSolrClient(restTestHarness.getAdminURL());

View File

@ -49,7 +49,7 @@ import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.cloud.SocketProxy;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.servlet.SolrDispatchFilter;
@ -616,7 +616,7 @@ public class JettySolrRunner {
SolrDispatchFilter sdf = getSolrDispatchFilter();
ExecutorService customThreadPool = null;
if (sdf != null) {
customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jettyShutDown"));
customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("jettyShutDown"));
sdf.closeOnDestroy(false);
// customThreadPool.submit(() -> {

View File

@ -45,7 +45,7 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
@ -196,7 +196,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
this.tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, MAX_PARALLEL_TASKS, 0L, TimeUnit.MILLISECONDS,
new SynchronousQueue<Runnable>(),
new DefaultSolrThreadFactory("OverseerThreadFactory"));
new SolrNamedThreadFactory("OverseerThreadFactory"));
try {
while (!this.isClosed) {
try {

View File

@ -96,7 +96,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.URLUtil;
@ -609,7 +609,7 @@ public class ZkController implements Closeable {
log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
}
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("preCloseThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("preCloseThreadPool"));
try {
synchronized (collectionToTerms) {
@ -629,7 +629,7 @@ public class ZkController implements Closeable {
if (!this.isClosed)
preClose();
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
customThreadPool.submit(() -> Collections.singleton(overseerElector.getContext()).parallelStream().forEach(IOUtils::closeQuietly));

View File

@ -84,7 +84,7 @@ import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.handler.component.ShardResponse;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.CreateMode;
@ -174,7 +174,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
final private LockTree lockTree = new LockTree();
ExecutorService tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, 10, 0L, TimeUnit.MILLISECONDS,
new SynchronousQueue<>(),
new DefaultSolrThreadFactory("OverseerCollectionMessageHandlerThreadFactory"));
new SolrNamedThreadFactory("OverseerCollectionMessageHandlerThreadFactory"));
protected static final Random RANDOM;
static {

View File

@ -57,7 +57,7 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -140,10 +140,10 @@ public class ScheduledTriggers implements Closeable {
public ScheduledTriggers(SolrResourceLoader loader, SolrCloudManager cloudManager) {
scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(DEFAULT_TRIGGER_CORE_POOL_SIZE,
new DefaultSolrThreadFactory("ScheduledTrigger"));
new SolrNamedThreadFactory("ScheduledTrigger"));
scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("AutoscalingActionExecutor"));
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
this.loader = loader;

View File

@ -100,7 +100,7 @@ import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.metrics.SolrMetricsContext;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.MockSearchableSolrClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -261,7 +261,7 @@ public class SimCloudManager implements SolrCloudManager {
this.clusterStateProvider = new SimClusterStateProvider(liveNodesSet, this);
this.nodeStateProvider = new SimNodeStateProvider(liveNodesSet, this.stateManager, this.clusterStateProvider, null);
this.queueFactory = new GenericDistributedQueueFactory(stateManager);
this.simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
this.simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new SolrNamedThreadFactory("simCloudManagerPool"));
this.autoScalingHandler = new AutoScalingHandler(this, loader);
@ -609,7 +609,7 @@ public class SimCloudManager implements SolrCloudManager {
} catch (Exception e) {
// ignore
}
simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new SolrNamedThreadFactory("simCloudManagerPool"));
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this);
triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");

View File

@ -46,7 +46,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.IdUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -232,7 +232,7 @@ public class SimDistribStateManager implements DistribStateManager {
public SimDistribStateManager(Node root) {
this.id = IdUtils.timeRandomId();
this.root = root != null ? root : createNewRootNode();
watchersPool = ExecutorUtil.newMDCAwareFixedThreadPool(10, new DefaultSolrThreadFactory("sim-watchers"));
watchersPool = ExecutorUtil.newMDCAwareFixedThreadPool(10, new SolrNamedThreadFactory("sim-watchers"));
String bufferSize = System.getProperty("jute.maxbuffer", Integer.toString(0xffffff));
juteMaxbuffer = Integer.parseInt(bufferSize);
}

View File

@ -77,7 +77,7 @@ import org.apache.solr.common.cloud.Replica.State;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.DirectoryFactory.DirContext;
import org.apache.solr.core.backup.repository.BackupRepository;
@ -119,7 +119,6 @@ import org.apache.solr.security.PublicKeyHandler;
import org.apache.solr.security.SecurityPluginHolder;
import org.apache.solr.update.SolrCoreState;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.OrderedExecutor;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.stats.MetricUtils;
@ -183,7 +182,7 @@ public class CoreContainer {
private volatile UpdateShardHandler updateShardHandler;
private volatile ExecutorService coreContainerWorkExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(
new DefaultSolrThreadFactory("coreContainerWorkExecutor"));
new SolrNamedThreadFactory("coreContainerWorkExecutor"));
private final OrderedExecutor replayUpdatesExecutor;
@ -329,7 +328,7 @@ public class CoreContainer {
cfg.getReplayUpdatesThreads(),
ExecutorUtil.newMDCAwareCachedThreadPool(
cfg.getReplayUpdatesThreads(),
new DefaultSolrThreadFactory("replayUpdatesExecutor")));
new SolrNamedThreadFactory("replayUpdatesExecutor")));
}
private synchronized void initializeAuthorizationPlugin(Map<String, Object> authorizationConf) {
@ -749,7 +748,7 @@ public class CoreContainer {
ExecutorService coreLoadExecutor = MetricUtils.instrumentedExecutorService(
ExecutorUtil.newMDCAwareFixedThreadPool(
cfg.getCoreLoadThreadCount(isZooKeeperAware()),
new DefaultSolrThreadFactory("coreLoadExecutor")), null,
new SolrNamedThreadFactory("coreLoadExecutor")), null,
metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoBean.Group.node)),
SolrMetricManager.mkName("coreLoadExecutor", SolrInfoBean.Category.CONTAINER.toString(), "threadPool"));
final List<Future<SolrCore>> futures = new ArrayList<>();
@ -932,7 +931,7 @@ public class CoreContainer {
log.info("Shutting down CoreContainer instance=" + System.identityHashCode(this));
ExecutorUtil.shutdownAndAwaitTermination(coreContainerAsyncTaskExecutor);
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
isShutDown = true;
try {

View File

@ -157,7 +157,7 @@ import org.apache.solr.update.processor.RunUpdateProcessorFactory;
import org.apache.solr.update.processor.UpdateRequestProcessorChain;
import org.apache.solr.update.processor.UpdateRequestProcessorChain.ProcessorInfo;
import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.IOFunction;
import org.apache.solr.util.NumberUtils;
import org.apache.solr.util.PropertiesInputStream;
@ -1837,7 +1837,7 @@ public final class SolrCore implements SolrInfoBean, Closeable {
private final LinkedList<RefCounted<SolrIndexSearcher>> _realtimeSearchers = new LinkedList<>();
final ExecutorService searcherExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(
new DefaultSolrThreadFactory("searcherExecutor"));
new SolrNamedThreadFactory("searcherExecutor"));
private int onDeckSearchers; // number of searchers preparing
// Lock ordering: one can acquire the openSearcherLock and then the searcherLock, but not vice-versa.
private Object searcherLock = new Object(); // the sync object for the searcher

View File

@ -21,7 +21,7 @@ import org.apache.http.annotation.Experimental;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -130,7 +130,7 @@ class SolrCores {
}
ExecutorService coreCloseExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(Integer.MAX_VALUE,
new DefaultSolrThreadFactory("coreCloseExecutor"));
new SolrNamedThreadFactory("coreCloseExecutor"));
try {
for (SolrCore core : coreList) {
coreCloseExecutor.submit(() -> {

View File

@ -39,7 +39,7 @@ import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -51,7 +51,7 @@ public class ZkContainer {
private SolrZkServer zkServer;
private ExecutorService coreZkRegister = ExecutorUtil.newMDCAwareCachedThreadPool(
new DefaultSolrThreadFactory("coreZkRegister") );
new SolrNamedThreadFactory("coreZkRegister") );
// see ZkController.zkRunOnly
private boolean zkRunOnly = Boolean.getBoolean("zkRunOnly"); // expert

View File

@ -46,7 +46,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.CdcrUpdateLog;
@ -133,7 +133,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
if (leaderStateManager.amILeader() && processStateManager.getState().equals(CdcrParams.ProcessState.STARTED)) {
if (replicatorStates.size() > 0) {
this.bootstrapExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(replicatorStates.size(),
new SolrjNamedThreadFactory("cdcr-bootstrap-status"));
new SolrNamedThreadFactory("cdcr-bootstrap-status"));
}
this.initLogReaders();
this.scheduler.start();

View File

@ -18,7 +18,7 @@ package org.apache.solr.handler;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -63,8 +63,8 @@ class CdcrReplicatorScheduler {
void start() {
if (!isStarted) {
scheduler = Executors.newSingleThreadScheduledExecutor(new DefaultSolrThreadFactory("cdcr-scheduler"));
replicatorsPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new DefaultSolrThreadFactory("cdcr-replicator"));
scheduler = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-scheduler"));
replicatorsPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("cdcr-replicator"));
// the scheduler thread is executed every second and submits one replication task
// per available state in the queue

View File

@ -70,7 +70,7 @@ import org.apache.solr.update.SolrCoreState;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.VersionInfo;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -407,7 +407,7 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
DocCollection docCollection = cstate.getCollectionOrNull(collection);
Collection<Slice> shards = docCollection == null? null : docCollection.getActiveSlices();
ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new DefaultSolrThreadFactory("parallelCdcrExecutor"));
ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("parallelCdcrExecutor"));
long checkpoint = Long.MAX_VALUE;
try {

View File

@ -37,7 +37,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -88,7 +88,7 @@ class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
public void stateUpdate() {
// If I am not the leader, I need to synchronise periodically my update log with my leader.
if (!leaderStateManager.amILeader()) {
scheduler = Executors.newSingleThreadScheduledExecutor(new DefaultSolrThreadFactory("cdcr-update-log-synchronizer"));
scheduler = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-update-log-synchronizer"));
scheduler.scheduleWithFixedDelay(new UpdateLogSynchronisation(), 0, timeSchedule, TimeUnit.MILLISECONDS);
return;
}

View File

@ -97,7 +97,7 @@ import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.VersionInfo;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.FileUtils;
import org.apache.solr.util.PropertiesOutputStream;
import org.apache.solr.util.RTimer;
@ -493,7 +493,7 @@ public class IndexFetcher {
}
// Create the sync service
fsyncService = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("fsyncService"));
fsyncService = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("fsyncService"));
// use a synchronized list because the list is read by other threads (to show details)
filesDownloaded = Collections.synchronizedList(new ArrayList<Map<String, Object>>());
// if the generation of master is older than that of the slave , it means they are not compatible to be copied

View File

@ -99,7 +99,7 @@ import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.SolrIndexWriter;
import org.apache.solr.update.VersionInfo;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.NumberUtils;
import org.apache.solr.util.PropertiesInputStream;
import org.apache.solr.util.RefCounted;
@ -175,7 +175,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
private ReentrantLock indexFetchLock = new ReentrantLock();
private ExecutorService restoreExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(
new DefaultSolrThreadFactory("restoreExecutor"));
new SolrNamedThreadFactory("restoreExecutor"));
private volatile Future<Boolean> restoreFuture;
@ -1212,7 +1212,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
}
};
executorService = Executors.newSingleThreadScheduledExecutor(
new DefaultSolrThreadFactory("indexFetcher"));
new SolrNamedThreadFactory("indexFetcher"));
// Randomize initial delay, with a minimum of 1ms
long initialDelayNs = new Random().nextLong() % pollIntervalNs
+ TimeUnit.NANOSECONDS.convert(1, TimeUnit.MILLISECONDS);

View File

@ -77,7 +77,7 @@ import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.SchemaManager;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.PermissionNameProvider;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.SolrPluginUtils;
import org.apache.solr.util.plugin.SolrCoreAware;
@ -780,7 +780,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
// use an executor service to invoke schema zk version requests in parallel with a max wait time
int poolSize = Math.min(concurrentTasks.size(), 10);
ExecutorService parallelExecutor =
ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new DefaultSolrThreadFactory("solrHandlerExecutor"));
ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("solrHandlerExecutor"));
try {
List<Future<Boolean>> results =
parallelExecutor.invokeAll(concurrentTasks, maxWaitSecs, TimeUnit.SECONDS);

View File

@ -51,7 +51,7 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.PermissionNameProvider;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.stats.MetricUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -73,7 +73,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
private final CoreAdminHandlerApi coreAdminHandlerApi;
protected ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(50,
new DefaultSolrThreadFactory("parallelCoreAdminExecutor"));
new SolrNamedThreadFactory("parallelCoreAdminExecutor"));
protected static int MAX_TRACKED_REQUESTS = 100;
public static String RUNNING = "running";

View File

@ -91,7 +91,7 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.PermissionNameProvider;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.KeeperException;
import org.rrd4j.ConsolFun;
import org.rrd4j.DsType;
@ -229,7 +229,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
if (enable) {
collectService = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(1,
new DefaultSolrThreadFactory("MetricsHistoryHandler"));
new SolrNamedThreadFactory("MetricsHistoryHandler"));
collectService.setRemoveOnCancelPolicy(true);
collectService.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
collectService.scheduleWithFixedDelay(() -> collectMetrics(),

View File

@ -72,7 +72,7 @@ import org.apache.solr.metrics.SolrMetricsContext;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.security.HttpClientBuilderPlugin;
import org.apache.solr.update.UpdateShardHandlerConfig;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.stats.InstrumentedHttpListenerFactory;
import org.apache.solr.util.stats.MetricUtils;
import org.slf4j.Logger;
@ -299,7 +299,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
this.maximumPoolSize,
this.keepAliveTime, TimeUnit.SECONDS,
blockingQueue,
new DefaultSolrThreadFactory("httpShardExecutor"),
new SolrNamedThreadFactory("httpShardExecutor"),
// the Runnable added to this executor handles all exceptions so we disable stack trace collection as an optimization
// see SOLR-11880 for more details
false

View File

@ -35,7 +35,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.search.SolrIndexSearcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -51,7 +51,7 @@ public abstract class IterativeMergeStrategy implements MergeStrategy {
public void merge(ResponseBuilder rb, ShardRequest sreq) {
rb._responseDocs = new SolrDocumentList(); // Null pointers will occur otherwise.
rb.onePassDistributedQuery = true; // Turn off the second pass distributed.
executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("IterativeMergeStrategy"));
executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("IterativeMergeStrategy"));
httpClient = getHttpClient();
try {
process(rb, sreq);

View File

@ -47,7 +47,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.rrd4j.core.RrdBackend;
import org.rrd4j.core.RrdBackendFactory;
import org.slf4j.Logger;
@ -104,7 +104,7 @@ public class SolrRrdBackendFactory extends RrdBackendFactory implements SolrClos
log.debug("Created " + hashCode());
this.idPrefixLength = ID_PREFIX.length() + ID_SEP.length();
syncService = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(2,
new DefaultSolrThreadFactory("SolrRrdBackendFactory"));
new SolrNamedThreadFactory("SolrRrdBackendFactory"));
syncService.setRemoveOnCancelPolicy(true);
syncService.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
syncService.scheduleWithFixedDelay(() -> maybeSyncBackends(),

View File

@ -69,7 +69,7 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.rest.schema.FieldTypeXmlAdapter;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.FileUtils;
import org.apache.solr.util.RTimer;
import org.apache.zookeeper.CreateMode;
@ -236,7 +236,7 @@ public final class ManagedIndexSchema extends IndexSchema {
// use an executor service to invoke schema zk version requests in parallel with a max wait time
int poolSize = Math.min(concurrentTasks.size(), 10);
ExecutorService parallelExecutor =
ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new DefaultSolrThreadFactory("managedSchemaExecutor"));
ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("managedSchemaExecutor"));
try {
List<Future<Integer>> results =
parallelExecutor.invokeAll(concurrentTasks, maxWaitSecs, TimeUnit.SECONDS);

View File

@ -42,7 +42,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.core.SolrInfoBean;
import org.apache.solr.metrics.SolrMetricsContext;
import org.apache.solr.security.AuditEvent.EventType;
@ -117,7 +117,7 @@ public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfo
pluginConfig.remove(PARAM_NUM_THREADS);
if (async) {
queue = new ArrayBlockingQueue<>(blockingQueueSize);
executorService = ExecutorUtil.newMDCAwareFixedThreadPool(numThreads, new SolrjNamedThreadFactory("audit"));
executorService = ExecutorUtil.newMDCAwareFixedThreadPool(numThreads, new SolrNamedThreadFactory("audit"));
executorService.submit(this);
}
pluginConfig.remove("class");

View File

@ -31,7 +31,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -59,7 +59,7 @@ public final class CommitTracker implements Runnable {
private long tLogFileSizeUpperBound;
private final ScheduledExecutorService scheduler =
Executors.newScheduledThreadPool(1, new DefaultSolrThreadFactory("commitScheduler"));
Executors.newScheduledThreadPool(1, new SolrNamedThreadFactory("commitScheduler"));
private ScheduledFuture pending;
// state

View File

@ -73,7 +73,7 @@ import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.update.processor.UpdateRequestProcessor;
import org.apache.solr.update.processor.UpdateRequestProcessorChain;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.OrderedExecutor;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.RefCounted;
@ -2096,7 +2096,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
ThreadPoolExecutor recoveryExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0,
Integer.MAX_VALUE, 1, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
new DefaultSolrThreadFactory("recoveryExecutor"));
new SolrNamedThreadFactory("recoveryExecutor"));
public static void deleteFile(File file) {

View File

@ -35,7 +35,7 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.core.SolrInfoBean;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.metrics.SolrMetricsContext;
@ -64,7 +64,7 @@ public class UpdateShardHandler implements SolrInfoBean {
private ExecutorService updateExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
60L, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrjNamedThreadFactory("updateExecutor"),
new SolrNamedThreadFactory("updateExecutor"),
// the Runnable added to this executor handles all exceptions so we disable stack trace collection as an optimization
// see SOLR-11880 for more details
false);
@ -136,7 +136,7 @@ public class UpdateShardHandler implements SolrInfoBean {
queryParams.add(DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM);
updateOnlyClient.setQueryParams(queryParams);
ThreadFactory recoveryThreadFactory = new SolrjNamedThreadFactory("recoveryExecutor");
ThreadFactory recoveryThreadFactory = new SolrNamedThreadFactory("recoveryExecutor");
if (cfg != null && cfg.getMaxRecoveryThreads() > 0) {
log.debug("Creating recoveryExecutor with pool size {}", cfg.getMaxRecoveryThreads());
recoveryExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(cfg.getMaxRecoveryThreads(), recoveryThreadFactory);

View File

@ -50,7 +50,7 @@ import org.apache.solr.update.AddUpdateCommand;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.DeleteUpdateCommand;
import org.apache.solr.util.DateMathParser;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -259,7 +259,7 @@ public final class DocExpirationUpdateProcessorFactory
private void initDeleteExpiredDocsScheduler(SolrCore core) {
executor = new ScheduledThreadPoolExecutor
(1, new DefaultSolrThreadFactory("autoExpireDocs"),
(1, new SolrNamedThreadFactory("autoExpireDocs"),
new RejectedExecutionHandler() {
public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
log.warn("Skipping execution of '{}' using '{}'", r, e);

View File

@ -73,6 +73,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.JavaBinCodec;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.noggit.CharArr;
import org.noggit.JSONWriter;
@ -399,9 +400,9 @@ public class ExportTool extends SolrCLI.ToolBase {
DocCollection coll = stateProvider.getCollection(this.coll);
Map<String, Slice> m = coll.getSlicesMap();
producerThreadpool = ExecutorUtil.newMDCAwareFixedThreadPool(m.size(),
new DefaultSolrThreadFactory("solrcli-exporter-producers"));
new SolrNamedThreadFactory("solrcli-exporter-producers"));
consumerThreadpool = ExecutorUtil.newMDCAwareFixedThreadPool(1,
new DefaultSolrThreadFactory("solrcli-exporter-consumer"));
new SolrNamedThreadFactory("solrcli-exporter-consumer"));
sink.start();
CountDownLatch consumerlatch = new CountDownLatch(1);
try {

View File

@ -81,7 +81,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TestInjection.Hook;
import org.junit.BeforeClass;
@ -709,7 +709,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
try {
executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
new DefaultSolrThreadFactory("testExecutor"));
new SolrNamedThreadFactory("testExecutor"));
int cnt = 3;
// create the cores

View File

@ -27,7 +27,7 @@ import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ConnectionManager;
import org.apache.solr.common.cloud.DefaultConnectionStrategy;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.EventType;
@ -113,7 +113,7 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
@Test
public void testReconnectWhenZkDisappeared() throws Exception {
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new DefaultSolrThreadFactory("connectionManagerTest"));
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("connectionManagerTest"));
// setup a SolrZkClient to do some getBaseUrlForNodeName testing
Path zkDir = createTempDir("zkData");

View File

@ -28,7 +28,7 @@ import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.junit.After;
@ -41,7 +41,7 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
protected ZkTestServer zkServer;
protected SolrZkClient zkClient;
protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("dqtest-"));
protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("dqtest-"));
@Before
@Override

View File

@ -35,7 +35,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.SessionExpiredException;
@ -420,7 +420,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
@Test
public void testStressElection() throws Exception {
final ScheduledExecutorService scheduler = Executors
.newScheduledThreadPool(15, new DefaultSolrThreadFactory("stressElection"));
.newScheduledThreadPool(15, new SolrNamedThreadFactory("stressElection"));
final List<ClientThread> threads = Collections
.synchronizedList(new ArrayList<ClientThread>());

View File

@ -69,7 +69,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloudConfig;
@ -324,7 +324,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
public void tearDown() throws Exception {
testDone = true;
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
for (ZkController zkController : zkControllers) {
customThreadPool.submit( () -> zkController.close());
@ -352,7 +352,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
for (Overseer overseer : overseers) {

View File

@ -57,7 +57,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TestInjection;
@ -87,7 +87,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
ThreadPoolExecutor executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0,
Integer.MAX_VALUE, 5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
new DefaultSolrThreadFactory("testExecutor"));
new SolrNamedThreadFactory("testExecutor"));
CompletionService<Object> completionService;
Set<Future<Object>> pending;

View File

@ -53,7 +53,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.TestInjection;
import org.junit.AfterClass;
import org.junit.Before;
@ -114,7 +114,7 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
NUM_THREADS = atLeast(3);
EXEC_SERVICE = ExecutorUtil.newMDCAwareFixedThreadPool
(NUM_THREADS, new DefaultSolrThreadFactory(DEBUG_LABEL));
(NUM_THREADS, new SolrNamedThreadFactory(DEBUG_LABEL));
// at least 2, but don't go crazy on nightly/test.multiplier with "atLeast()"
final int numShards = TEST_NIGHTLY ? 5 : 2;

View File

@ -30,7 +30,7 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.CreateMode;
@ -176,7 +176,7 @@ public class TestStressLiveNodes extends SolrCloudTestCase {
}
try {
final ExecutorService executorService = ExecutorUtil.newMDCAwareFixedThreadPool
(thrashers.size()+1, new DefaultSolrThreadFactory("test_live_nodes_thrasher_iter"+iter));
(thrashers.size()+1, new SolrNamedThreadFactory("test_live_nodes_thrasher_iter"+iter));
executorService.invokeAll(thrashers);
executorService.shutdown();

View File

@ -33,7 +33,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.CollectionStatePredicate;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.cloud.SolrCloudTestCase.clusterShape;
@ -74,7 +74,7 @@ public class TestWaitForStateWithJettyShutdowns extends SolrTestCaseJ4 {
public void testWaitForStateBeforeShutDown() throws Exception {
final String col_name = "test_col";
final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool
(1, new DefaultSolrThreadFactory("background_executor"));
(1, new SolrNamedThreadFactory("background_executor"));
final MiniSolrCloudCluster cluster = new MiniSolrCloudCluster
(1, createTempDir(), buildJettyConfig("/solr"));
try {

View File

@ -34,7 +34,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
@ -334,7 +334,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
int numReplicas = atLeast(3);
ThreadPoolExecutor executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
5, TimeUnit.SECONDS, new SynchronousQueue<>(),
new DefaultSolrThreadFactory("testExecutor"));
new SolrNamedThreadFactory("testExecutor"));
try {
// create the cores
createCollectionInOneInstance(adminClient, jetty.getNodeName(), executor, "multiunload", 2, numReplicas);
@ -344,7 +344,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE, 5,
TimeUnit.SECONDS, new SynchronousQueue<>(),
new DefaultSolrThreadFactory("testExecutor"));
new SolrNamedThreadFactory("testExecutor"));
try {
for (int j = 0; j < numReplicas; j++) {
final int freezeJ = j;

View File

@ -42,7 +42,7 @@ 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.common.util.ExecutorUtil;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -239,7 +239,7 @@ public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
final AtomicInteger numFailure = new AtomicInteger(0);
final CountDownLatch latch = new CountDownLatch(numThreads);
ExecutorService es = ExecutorUtil.newMDCAwareFixedThreadPool(numThreads, new DefaultSolrThreadFactory("testAsyncIdRaceCondition"));
ExecutorService es = ExecutorUtil.newMDCAwareFixedThreadPool(numThreads, new SolrNamedThreadFactory("testAsyncIdRaceCondition"));
try {
for (int i = 0; i < numThreads; i++) {
es.submit(new Runnable() {

View File

@ -27,7 +27,7 @@ import java.util.function.Predicate;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.After;
import org.junit.Test;
@ -36,7 +36,7 @@ import org.junit.Test;
*/
public class TestSimDistributedQueue extends SolrTestCaseJ4 {
private static final Charset UTF8 = Charset.forName("UTF-8");
protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("sdqtest-"));
protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("sdqtest-"));
@Test
// commented 20-July-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 05-Jul-2018

View File

@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -219,7 +219,7 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
AtomicInteger numDocs = new AtomicInteger(0);
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new DefaultSolrThreadFactory("cdcr-test-update-scheduler"));
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-test-update-scheduler"));
executor.scheduleWithFixedDelay(new UpdateThread(numDocs), 10, 10, TimeUnit.MILLISECONDS);
// Restart the slave node to trigger Replication strategy

View File

@ -28,7 +28,7 @@ import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.update.SolrCoreState;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.junit.Test;
@ -187,7 +187,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
final int LOOP = 100;
final int MT = 16;
ExecutorService service = ExecutorUtil.newMDCAwareFixedThreadPool(MT, new DefaultSolrThreadFactory("refCountMT"));
ExecutorService service = ExecutorUtil.newMDCAwareFixedThreadPool(MT, new SolrNamedThreadFactory("refCountMT"));
List<Callable<Integer>> callees = new ArrayList<>(MT);
final CoreContainer cores = h.getCoreContainer();
for (int i = 0; i < MT; ++i) {
@ -275,7 +275,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
@Test
public void testReloadLeak() throws Exception {
final ExecutorService executor =
ExecutorUtil.newMDCAwareFixedThreadPool(1, new DefaultSolrThreadFactory("testReloadLeak"));
ExecutorUtil.newMDCAwareFixedThreadPool(1, new SolrNamedThreadFactory("testReloadLeak"));
// Continuously open new searcher while core is not closed, and reload core to try to reproduce searcher leak.
// While in practice we never continuously open new searchers, this is trying to make up for the fact that opening

View File

@ -55,7 +55,7 @@ import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.security.AuditEvent.EventType;
import org.apache.solr.security.AuditEvent.RequestType;
import org.apache.solr.security.AuditLoggerPlugin.JSONAuditEventFormatter;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -168,7 +168,7 @@ public class AuditLoggerIntegrationTest extends SolrCloudAuthTestCase {
// Now shutdown cluster while 1 event still in process
// Do this in a background thread because it blocks...
final Thread shutdownThread = new DefaultSolrThreadFactory("shutdown")
final Thread shutdownThread = new SolrNamedThreadFactory("shutdown")
.newThread(() -> { try {
log.info("START Shutting down Cluster.");
harness.shutdownCluster();
@ -498,7 +498,7 @@ public class AuditLoggerIntegrationTest extends SolrCloudAuthTestCase {
AuditTestHarness() throws IOException {
receiver = new CallbackReceiver();
callbackPort = receiver.getPort();
receiverThread = new DefaultSolrThreadFactory("auditTestCallback").newThread(receiver);
receiverThread = new SolrNamedThreadFactory("auditTestCallback").newThread(receiver);
receiverThread.start();
}

View File

@ -66,7 +66,7 @@ import org.apache.solr.handler.loader.XMLLoader;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RefCounted;
import org.junit.After;
import org.junit.AfterClass;
@ -114,8 +114,8 @@ public class AddBlockUpdateTest extends SolrTestCaseJ4 {
inputFactory = XMLInputFactory.newInstance();
exe = // Executors.newSingleThreadExecutor();
rarely() ? ExecutorUtil.newMDCAwareFixedThreadPool(atLeast(2), new DefaultSolrThreadFactory("AddBlockUpdateTest")) : ExecutorUtil
.newMDCAwareCachedThreadPool(new DefaultSolrThreadFactory("AddBlockUpdateTest"));
rarely() ? ExecutorUtil.newMDCAwareFixedThreadPool(atLeast(2), new SolrNamedThreadFactory("AddBlockUpdateTest")) : ExecutorUtil
.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("AddBlockUpdateTest"));
counter.set(0);
initCore("solrconfig.xml", "schema15.xml");

View File

@ -59,7 +59,7 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.index.NoMergePolicyFactory;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
@ -296,7 +296,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
// Reordering needs to happen using parallel threads
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
// re-order the updates for NONLEADER 0
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
@ -347,7 +347,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
// Reordering needs to happen using parallel threads
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
// re-order the updates by swapping the last two
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
@ -750,7 +750,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
// Reordering needs to happen using parallel threads, since some of these updates will
// be blocking calls, waiting for some previous updates to arrive on which it depends.
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
// re-order the updates for NONLEADER 0
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
@ -818,7 +818,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
// Reordering needs to happen using parallel threads
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
// re-order the updates for NONLEADER 0
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
@ -890,7 +890,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
// Reordering needs to happen using parallel threads
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
// re-order the last two updates for NONLEADER 0
List<UpdateRequest> reorderedUpdates = new ArrayList<>(updates);
Collections.swap(reorderedUpdates, 2, 3);
@ -972,7 +972,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
"Waiting for dependant update to timeout", 1, 6000);
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
for (UpdateRequest update : updates) {
AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, cloudClient,
random().nextLong());
@ -1045,7 +1045,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
shardToJetty.get(SHARD1).get(1).jetty.getDebugFilter().addDelay("Waiting for dependant update to timeout", 4, 5998); // the delete update
threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
for (UpdateRequest update : updates) {
AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, cloudClient,
random().nextLong());
@ -1298,7 +1298,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
"Waiting for dependant update to timeout", 2, 8000);
ExecutorService threadpool =
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new DefaultSolrThreadFactory(getTestName()));
ExecutorUtil.newMDCAwareFixedThreadPool(updates.size() + 1, new SolrNamedThreadFactory(getTestName()));
for (UpdateRequest update : updates) {
AsyncUpdateWithRandomCommit task = new AsyncUpdateWithRandomCommit(update, cloudClient,
random().nextLong());

View File

@ -56,7 +56,7 @@ import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.update.UpdateCommand;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.Ignore;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
@ -273,7 +273,7 @@ public abstract class RoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
try (CloudSolrClient solrClient = getCloudSolrClient(cluster)) {
try {
exec = ExecutorUtil.newMDCAwareFixedThreadPool(1 + random().nextInt(2),
new DefaultSolrThreadFactory(getSaferTestName()));
new SolrNamedThreadFactory(getSaferTestName()));
List<Future<UpdateResponse>> futures = new ArrayList<>(solrInputDocuments.length);
for (SolrInputDocument solrInputDocument : solrInputDocuments) {
String col = collections.get(random().nextInt(collections.size()));

View File

@ -30,7 +30,7 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.Before;
import org.junit.BeforeClass;
@ -445,7 +445,7 @@ public class TestDocBasedVersionConstraints extends SolrTestCaseJ4 {
public void testConcurrentAdds() throws Exception {
final int NUM_DOCS = atLeast(50);
final int MAX_CONCURENT = atLeast(10);
ExecutorService runner = ExecutorUtil.newMDCAwareFixedThreadPool(MAX_CONCURENT, new DefaultSolrThreadFactory("TestDocBasedVersionConstraints"));
ExecutorService runner = ExecutorUtil.newMDCAwareFixedThreadPool(MAX_CONCURENT, new SolrNamedThreadFactory("TestDocBasedVersionConstraints"));
// runner = Executors.newFixedThreadPool(1); // to test single threaded
try {
for (int id = 0; id < NUM_DOCS; id++) {

View File

@ -81,7 +81,7 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.Hash;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
@ -105,7 +105,7 @@ public abstract class BaseCloudSolrClient extends SolrClient {
private final RequestReplicaListTransformerGenerator requestRLTGenerator;
boolean parallelUpdates; //TODO final
private ExecutorService threadPool = ExecutorUtil
.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory(
.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory(
"CloudSolrClient ThreadPool"));
private String idField = ID;
public static final String STATE_VERSION = "_stateVer_";

View File

@ -40,7 +40,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.eclipse.jetty.client.api.Response;
import org.eclipse.jetty.client.util.InputStreamResponseListener;
import org.eclipse.jetty.http.HttpStatus;
@ -160,7 +160,7 @@ public class ConcurrentUpdateHttp2SolrClient extends SolrClient {
this.scheduler = builder.executorService;
this.shutdownExecutor = false;
} else {
this.scheduler = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("concurrentUpdateScheduler"));
this.scheduler = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("concurrentUpdateScheduler"));
this.shutdownExecutor = true;
}

View File

@ -55,7 +55,7 @@ import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -144,7 +144,7 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
this.scheduler = builder.executorService;
this.shutdownExecutor = false;
} else {
this.scheduler = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("concurrentUpdateScheduler"));
this.scheduler = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("concurrentUpdateScheduler"));
this.shutdownExecutor = true;
}

View File

@ -68,7 +68,7 @@ import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.client.HttpClientTransport;
import org.eclipse.jetty.client.ProtocolHandlers;
@ -177,7 +177,7 @@ public class Http2SolrClient extends SolrClient {
BlockingArrayQueue<Runnable> queue = new BlockingArrayQueue<>(256, 256);
ThreadPoolExecutor httpClientExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(32,
256, 60, TimeUnit.SECONDS, queue, new SolrjNamedThreadFactory("h2sc"));
256, 60, TimeUnit.SECONDS, queue, new SolrNamedThreadFactory("h2sc"));
SslContextFactory.Client sslContextFactory;
boolean ssl;

View File

@ -82,7 +82,7 @@ import org.apache.solr.common.util.Base64;
import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -303,7 +303,7 @@ public class HttpSolrClient extends BaseHttpSolrClient {
public HttpUriRequestResponse httpUriRequest(final SolrRequest request, final ResponseParser processor) throws SolrServerException, IOException {
HttpUriRequestResponse mrr = new HttpUriRequestResponse();
final HttpRequestBase method = createMethod(request, null);
ExecutorService pool = ExecutorUtil.newMDCAwareFixedThreadPool(1, new SolrjNamedThreadFactory("httpUriRequest"));
ExecutorService pool = ExecutorUtil.newMDCAwareFixedThreadPool(1, new SolrNamedThreadFactory("httpUriRequest"));
try {
MDC.put("HttpSolrClient.url", baseUrl);
mrr.future = pool.submit(() -> executeMethod(method, request.getUserPrincipal(), processor, isV2ApiRequest(request)));

View File

@ -52,7 +52,7 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.MDC;
import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
@ -442,7 +442,7 @@ public abstract class LBSolrClient extends SolrClient {
synchronized (this) {
if (aliveCheckExecutor == null) {
aliveCheckExecutor = Executors.newSingleThreadScheduledExecutor(
new SolrjNamedThreadFactory("aliveCheckExecutor"));
new SolrNamedThreadFactory("aliveCheckExecutor"));
aliveCheckExecutor.scheduleAtFixedRate(
getAliveCheckRunner(new WeakReference<>(this)),
this.interval, this.interval, TimeUnit.MILLISECONDS);

View File

@ -50,7 +50,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.client.solrj.io.stream.metrics.Metric;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.SORT;
@ -517,7 +517,7 @@ public class GatherNodesStream extends TupleStream implements Expressible {
ExecutorService threadPool = null;
try {
threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(4, new SolrjNamedThreadFactory("GatherNodesStream"));
threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(4, new SolrNamedThreadFactory("GatherNodesStream"));
Map<String, Node> roots = new HashMap();

View File

@ -49,7 +49,7 @@ import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.SORT;
@ -300,7 +300,7 @@ public class ShortestPathStream extends TupleStream implements Expressible {
try {
threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(threads, new SolrjNamedThreadFactory("ShortestPathStream"));
threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(threads, new SolrNamedThreadFactory("ShortestPathStream"));
//Breadth first search
TRAVERSE:

View File

@ -54,7 +54,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.DISTRIB;
import static org.apache.solr.common.params.CommonParams.SORT;
@ -395,7 +395,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("CloudSolrStream"));
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("CloudSolrStream"));
try {
List<Future<TupleWrapper>> futures = new ArrayList();
for (TupleStream solrStream : solrStreams) {

View File

@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.io.comp.StreamComparator;
import org.apache.solr.client.solrj.io.stream.expr.Explanation;
import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.client.solrj.io.stream.expr.Expressible;
import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
@ -206,7 +206,7 @@ public class DaemonStream extends TupleStream implements Expressible {
}
this.closed = false;
this.streamRunner = new StreamRunner(runInterval, id);
ExecutorService service = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("DaemonStream-" + id));
ExecutorService service = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("DaemonStream-" + id));
try {
service.submit(this.streamRunner);
}

View File

@ -51,7 +51,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.DISTRIB;
import static org.apache.solr.common.params.CommonParams.ROWS;
@ -347,7 +347,7 @@ public class DeepRandomStream extends TupleStream implements Expressible {
}
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("DeepRandomStream"));
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("DeepRandomStream"));
try {
List<Future<TupleWrapper>> futures = new ArrayList();
for (TupleStream solrStream : solrStreams) {

View File

@ -37,7 +37,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParamete
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -138,7 +138,7 @@ public class ExecutorStream extends TupleStream implements Expressible {
}
public void open() throws IOException {
executorService = ExecutorUtil.newMDCAwareFixedThreadPool(threads, new SolrjNamedThreadFactory("ExecutorStream"));
executorService = ExecutorUtil.newMDCAwareFixedThreadPool(threads, new SolrNamedThreadFactory("ExecutorStream"));
stream.open();
}

View File

@ -57,7 +57,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.DISTRIB;
import static org.apache.solr.common.params.CommonParams.ID;
@ -248,7 +248,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
}
this.cloudSolrClient = this.cache.getCloudSolrClient(zkHost);
this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("FeaturesSelectionStream"));
this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("FeaturesSelectionStream"));
}
public List<TupleStream> children() {

View File

@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
public class ParallelListStream extends TupleStream implements Expressible {
@ -137,7 +137,7 @@ public class ParallelListStream extends TupleStream implements Expressible {
}
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("ParallelListStream"));
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("ParallelListStream"));
try {
List<Future<StreamIndex>> futures = new ArrayList();
int i=0;

View File

@ -49,7 +49,7 @@ import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.DISTRIB;
@ -233,7 +233,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
isCloseCache = false;
}
this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("SignificantTermsStream"));
this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("SignificantTermsStream"));
}
public List<TupleStream> children() {

View File

@ -58,7 +58,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.DISTRIB;
import static org.apache.solr.common.params.CommonParams.ID;
@ -328,7 +328,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
}
this.cloudSolrClient = this.cache.getCloudSolrClient(zkHost);
this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("TextLogitSolrStream"));
this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TextLogitSolrStream"));
}
public List<TupleStream> children() {

View File

@ -58,7 +58,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import static org.apache.solr.common.params.CommonParams.DISTRIB;
import static org.apache.solr.common.params.CommonParams.ID;
@ -311,7 +311,7 @@ public class TopicStream extends CloudSolrStream implements Expressible {
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("TopicStream"));
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TopicStream"));
try {
List<Future<TupleWrapper>> futures = new ArrayList();
for (TupleStream solrStream : solrStreams) {

View File

@ -46,7 +46,7 @@ import org.apache.solr.common.StringUtils;
import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoAuthException;
@ -83,9 +83,9 @@ public class SolrZkClient implements Closeable {
private ZkCmdExecutor zkCmdExecutor;
private final ExecutorService zkCallbackExecutor =
ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("zkCallback"));
ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("zkCallback"));
private final ExecutorService zkConnManagerCallbackExecutor =
ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("zkConnectionManagerCallback"));
ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("zkConnectionManagerCallback"));
private volatile boolean isClosed = false;
private ZkClientConnectionStrategy zkClientConnectionStrategy;

View File

@ -57,7 +57,7 @@ import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
@ -220,7 +220,7 @@ public class ZkStateReader implements SolrCloseable {
/**
* Used to submit notifications to Collection Properties watchers in order
**/
private final ExecutorService collectionPropsNotifications = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("collectionPropsNotifications"));
private final ExecutorService collectionPropsNotifications = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("collectionPropsNotifications"));
private static final long LAZY_CACHE_TIME = TimeUnit.NANOSECONDS.convert(STATE_UPDATE_DELAY, TimeUnit.MILLISECONDS);

View File

@ -114,7 +114,7 @@ public class ExecutorUtil {
* Create a cached thread pool using a named thread factory
*/
public static ExecutorService newMDCAwareCachedThreadPool(String name) {
return newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory(name));
return newMDCAwareCachedThreadPool(new SolrNamedThreadFactory(name));
}
/**

View File

@ -14,36 +14,39 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.util;
package org.apache.solr.common.util;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
public class DefaultSolrThreadFactory implements ThreadFactory {
/**
* A {@link ThreadFactory} that names threads with the given prefix.
*/
public class SolrNamedThreadFactory implements ThreadFactory {
private static final AtomicInteger poolNumber = new AtomicInteger(1);
private final ThreadGroup group;
private final AtomicInteger threadNumber = new AtomicInteger(1);
private final String prefix;
public DefaultSolrThreadFactory(String namePrefix) {
public SolrNamedThreadFactory(String namePrefix) {
SecurityManager s = System.getSecurityManager();
group = (s != null)? s.getThreadGroup() :
Thread.currentThread().getThreadGroup();
Thread.currentThread().getThreadGroup();
prefix = namePrefix + "-" +
poolNumber.getAndIncrement() +
"-thread-";
poolNumber.getAndIncrement() +
"-thread-";
}
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(group, r,
prefix + threadNumber.getAndIncrement(),
0);
prefix + threadNumber.getAndIncrement(),
0);
t.setDaemon(false);
if (t.getPriority() != Thread.NORM_PRIORITY)
t.setPriority(Thread.NORM_PRIORITY);
return t;
}
}
}

View File

@ -19,31 +19,14 @@ package org.apache.solr.common.util;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
public class SolrjNamedThreadFactory implements ThreadFactory {
private static final AtomicInteger poolNumber = new AtomicInteger(1);
private final ThreadGroup group;
private final AtomicInteger threadNumber = new AtomicInteger(1);
private final String prefix;
public SolrjNamedThreadFactory(String namePrefix) {
SecurityManager s = System.getSecurityManager();
group = (s != null)? s.getThreadGroup() :
Thread.currentThread().getThreadGroup();
prefix = namePrefix + "-" +
poolNumber.getAndIncrement() +
"-thread-";
}
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(group, r,
prefix + threadNumber.getAndIncrement(),
0);
t.setDaemon(false);
if (t.getPriority() != Thread.NORM_PRIORITY)
t.setPriority(Thread.NORM_PRIORITY);
return t;
}
/**
* Deprecated to rename to {@link SolrNamedThreadFactory}.
*
* @deprecated use {@link SolrNamedThreadFactory} instead.
*/
@Deprecated
public class SolrjNamedThreadFactory extends SolrNamedThreadFactory {
public SolrjNamedThreadFactory(String namePrefix) {
super(namePrefix);
}
}

View File

@ -29,7 +29,7 @@ import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettyConfig;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.eclipse.jetty.client.api.Response;
import org.eclipse.jetty.servlet.ServletHolder;
import org.junit.BeforeClass;
@ -72,7 +72,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
concurrentClient.blockUntilFinished();
int poolSize = 5;
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrjNamedThreadFactory("testCUSS"));
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("testCUSS"));
int numDocs = 100;
int numRunnables = 5;
@ -161,7 +161,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
concurrentClient.deleteByQuery("collection1", "*:*");
int poolSize = 5;
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrjNamedThreadFactory("testCUSS"));
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("testCUSS"));
for (int r=0; r < numRunnables; r++)
threadPool.execute(new ConcurrentUpdateSolrClientTest.SendDocsRunnable(String.valueOf(r), numDocs, concurrentClient, "collection1"));

View File

@ -26,7 +26,7 @@ import org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.eclipse.jetty.servlet.ServletHolder;
import org.junit.BeforeClass;
import org.junit.Test;
@ -160,7 +160,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
concurrentClient.blockUntilFinished();
int poolSize = 5;
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrjNamedThreadFactory("testCUSS"));
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("testCUSS"));
int numDocs = 100;
int numRunnables = 5;
@ -242,7 +242,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
concurrentClient.deleteByQuery("collection1", "*:*");
int poolSize = 5;
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrjNamedThreadFactory("testCUSS"));
ExecutorService threadPool = ExecutorUtil.newMDCAwareFixedThreadPool(poolSize, new SolrNamedThreadFactory("testCUSS"));
for (int r=0; r < numRunnables; r++)
threadPool.execute(new SendDocsRunnable(String.valueOf(r), numDocs, concurrentClient, "collection1"));

View File

@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -126,7 +126,7 @@ public class HttpSolrClientConPoolTest extends SolrJettyTestBase {
final HttpSolrClient client1 ;
int threadCount = atLeast(2);
final ExecutorService threads = ExecutorUtil.newMDCAwareFixedThreadPool(threadCount,
new SolrjNamedThreadFactory(getClass().getSimpleName()+"TestScheduler"));
new SolrNamedThreadFactory(getClass().getSimpleName()+"TestScheduler"));
CloseableHttpClient httpClient = HttpClientUtil.createClient(new ModifiableSolrParams(), pool);
try{
final LBHttpSolrClient roundRobin = new LBHttpSolrClient.Builder().

View File

@ -64,8 +64,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.eclipse.jetty.servlet.ServletHolder;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -104,7 +103,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
Integer.MAX_VALUE,
15, TimeUnit.SECONDS, // terminate idle threads after 15 sec
new SynchronousQueue<>(), // directly hand off tasks
new DefaultSolrThreadFactory("BaseDistributedSearchTestCase"),
new SolrNamedThreadFactory("BaseDistributedSearchTestCase"),
false
);
@ -413,7 +412,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
}
protected void destroyServers() throws Exception {
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
customThreadPool.submit(() -> Collections.singleton(controlClient).parallelStream().forEach(c -> {
IOUtils.closeQuietly(c);

View File

@ -113,7 +113,7 @@ import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.ContentStreamBase;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.Utils;
import org.apache.solr.common.util.XML;
@ -268,7 +268,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
testExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
15L, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new SolrjNamedThreadFactory("testExecutor"),
new SolrNamedThreadFactory("testExecutor"),
true);
// set solr.install.dir needed by some test configs outside of the test sandbox (!)

View File

@ -82,7 +82,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
@ -416,7 +416,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
// expect sliceCount active shards, but no active replicas
SolrCloudTestCase.activeClusterShape(sliceCount, 0));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
int numOtherReplicas = numJettys - getPullReplicaCount() * sliceCount;
@ -522,7 +522,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("createReplicaRequests"));
customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("createReplicaRequests"));
for (CollectionAdminRequest r : createReplicaRequests) {
customThreadPool.submit(() -> {
@ -541,7 +541,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
customThreadPool = ExecutorUtil
.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("createPullReplicaRequests"));
.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("createPullReplicaRequests"));
for (CollectionAdminRequest r : createPullReplicaRequests) {
customThreadPool.submit(() -> {
CollectionAdminResponse response;
@ -1714,7 +1714,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
@Override
protected void destroyServers() throws Exception {
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("closeThreadPool"));
customThreadPool.submit(() -> Collections.singleton(commonCloudSolrClient).parallelStream().forEach(c -> {
IOUtils.closeQuietly(c);

View File

@ -43,7 +43,7 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
@ -592,7 +592,7 @@ public class ChaosMonkey {
Integer.MAX_VALUE,
15, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new DefaultSolrThreadFactory("ChaosMonkey"),
new SolrNamedThreadFactory("ChaosMonkey"),
false);
for (JettySolrRunner jetty : jettys) {
executor.submit(() -> {
@ -613,7 +613,7 @@ public class ChaosMonkey {
Integer.MAX_VALUE,
15, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new DefaultSolrThreadFactory("ChaosMonkey"),
new SolrNamedThreadFactory("ChaosMonkey"),
false);
for (JettySolrRunner jetty : jettys) {
executor.submit(() -> {

View File

@ -67,7 +67,7 @@ import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeOut;
@ -302,7 +302,7 @@ public class MiniSolrCloudCluster {
startups.add(() -> startJettySolrRunner(newNodeName(), jettyConfig.context, jettyConfig));
}
final ExecutorService executorLauncher = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-launcher"));
final ExecutorService executorLauncher = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("jetty-launcher"));
Collection<Future<JettySolrRunner>> futures = executorLauncher.invokeAll(startups);
ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
Exception startupError = checkForExceptions("Error starting up MiniSolrCloudCluster", futures);
@ -610,7 +610,7 @@ public class MiniSolrCloudCluster {
shutdowns.add(() -> stopJettySolrRunner(jetty));
}
jettys.clear();
final ExecutorService executorCloser = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-closer"));
final ExecutorService executorCloser = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("jetty-closer"));
Collection<Future<JettySolrRunner>> futures = executorCloser.invokeAll(shutdowns);
ExecutorUtil.shutdownAndAwaitTermination(executorCloser);
Exception shutdownError = checkForExceptions("Error shutting down MiniSolrCloudCluster", futures);