mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-06-28 09:28:55 -04:00
Cleanup redundant logger fields (#101655)
Making loggers static that can be made static, removing redundant test loggers (we already have the correct per class logger in this.logger) and also removing some just unused loggers.
This commit is contained in:
parent
8485cd7e83
commit
995b4d3c69
91 changed files with 82 additions and 229 deletions
|
@ -40,7 +40,7 @@ import javax.inject.Inject;
|
|||
*/
|
||||
public class ExportElasticsearchBuildResourcesTask extends DefaultTask {
|
||||
|
||||
private final Logger logger = Logging.getLogger(ExportElasticsearchBuildResourcesTask.class);
|
||||
private static final Logger logger = Logging.getLogger(ExportElasticsearchBuildResourcesTask.class);
|
||||
|
||||
private final Set<String> resources = new HashSet<>();
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ import java.util.Map;
|
|||
|
||||
public class AzureBlobContainer extends AbstractBlobContainer {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(AzureBlobContainer.class);
|
||||
private static final Logger logger = LogManager.getLogger(AzureBlobContainer.class);
|
||||
private final AzureBlobStore blobStore;
|
||||
private final String keyPath;
|
||||
|
||||
|
|
|
@ -239,8 +239,8 @@ class AzureClientProvider extends AbstractLifecycleComponent {
|
|||
protected void doClose() throws IOException {}
|
||||
|
||||
private static final class SuccessfulRequestTracker implements HttpPipelinePolicy {
|
||||
private static final Logger logger = LogManager.getLogger(SuccessfulRequestTracker.class);
|
||||
private final BiConsumer<String, URL> onSuccessfulRequest;
|
||||
private final Logger logger = LogManager.getLogger(SuccessfulRequestTracker.class);
|
||||
|
||||
private SuccessfulRequestTracker(BiConsumer<String, URL> onSuccessfulRequest) {
|
||||
this.onSuccessfulRequest = onSuccessfulRequest;
|
||||
|
|
|
@ -50,7 +50,7 @@ class CancellableRateLimitedFluxIterator<T> implements Subscriber<T>, Iterator<T
|
|||
private final Condition condition;
|
||||
private final Consumer<T> cleaner;
|
||||
private final AtomicReference<Subscription> subscription = new AtomicReference<>();
|
||||
private final Logger logger = LogManager.getLogger(CancellableRateLimitedFluxIterator.class);
|
||||
private static final Logger logger = LogManager.getLogger(CancellableRateLimitedFluxIterator.class);
|
||||
private volatile Throwable error;
|
||||
private volatile boolean done;
|
||||
private int emittedElements;
|
||||
|
|
|
@ -37,7 +37,7 @@ import static org.elasticsearch.core.Strings.format;
|
|||
public class ReactorScheduledExecutorService extends AbstractExecutorService implements ScheduledExecutorService {
|
||||
private final ThreadPool threadPool;
|
||||
private final ExecutorService delegate;
|
||||
private final Logger logger = LogManager.getLogger(ReactorScheduledExecutorService.class);
|
||||
private static final Logger logger = LogManager.getLogger(ReactorScheduledExecutorService.class);
|
||||
|
||||
public ReactorScheduledExecutorService(ThreadPool threadPool, String executorName) {
|
||||
this.threadPool = threadPool;
|
||||
|
|
|
@ -33,7 +33,7 @@ class RetryingHttpInputStream extends InputStream {
|
|||
public static final int MAX_SUPPRESSED_EXCEPTIONS = 10;
|
||||
public static final long MAX_RANGE_VAL = Long.MAX_VALUE - 1;
|
||||
|
||||
private final Logger logger = LogManager.getLogger(RetryingHttpInputStream.class);
|
||||
private static final Logger logger = LogManager.getLogger(RetryingHttpInputStream.class);
|
||||
|
||||
private final String blobName;
|
||||
private final URI blobURI;
|
||||
|
|
|
@ -37,7 +37,7 @@ import java.util.Map;
|
|||
public class URLHttpClient implements Closeable {
|
||||
public static final int MAX_ERROR_MESSAGE_BODY_SIZE = 1024;
|
||||
private static final int MAX_CONNECTIONS = 50;
|
||||
private final Logger logger = LogManager.getLogger(URLHttpClient.class);
|
||||
private static final Logger logger = LogManager.getLogger(URLHttpClient.class);
|
||||
|
||||
private final CloseableHttpClient client;
|
||||
private final URLHttpClientSettings httpClientSettings;
|
||||
|
@ -142,7 +142,7 @@ public class URLHttpClient implements Closeable {
|
|||
};
|
||||
}
|
||||
|
||||
private void handleInvalidResponse(CloseableHttpResponse response) {
|
||||
private static void handleInvalidResponse(CloseableHttpResponse response) {
|
||||
int statusCode = response.getStatusLine().getStatusCode();
|
||||
String errorBody = parseBodyAsString(response, MAX_ERROR_MESSAGE_BODY_SIZE);
|
||||
throw new URLHttpClientException(statusCode, createErrorMessage(statusCode, errorBody));
|
||||
|
@ -156,7 +156,7 @@ public class URLHttpClient implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private String parseBodyAsString(CloseableHttpResponse response, int maxSize) {
|
||||
private static String parseBodyAsString(CloseableHttpResponse response, int maxSize) {
|
||||
String errorMessage = "";
|
||||
InputStream bodyContent = null;
|
||||
try {
|
||||
|
|
|
@ -38,7 +38,7 @@ import static org.hamcrest.Matchers.instanceOf;
|
|||
@ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, minNumDataNodes = 2)
|
||||
public class AwarenessAllocationIT extends ESIntegTestCase {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(AwarenessAllocationIT.class);
|
||||
private static final Logger logger = LogManager.getLogger(AwarenessAllocationIT.class);
|
||||
|
||||
@Override
|
||||
protected int numberOfReplicas() {
|
||||
|
|
|
@ -9,8 +9,6 @@
|
|||
package org.elasticsearch.cluster.allocation;
|
||||
|
||||
import org.apache.logging.log4j.Level;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
|
||||
|
@ -66,7 +64,6 @@ import static org.hamcrest.Matchers.hasSize;
|
|||
@LuceneTestCase.SuppressFileSystems(value = "WindowsFS")
|
||||
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
|
||||
public class ClusterRerouteIT extends ESIntegTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ClusterRerouteIT.class);
|
||||
|
||||
public void testRerouteWithCommands_disableAllocationSettings() throws Exception {
|
||||
Settings commonSettings = Settings.builder()
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.gateway;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
|
||||
|
@ -69,8 +67,6 @@ import static org.hamcrest.Matchers.notNullValue;
|
|||
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
|
||||
public class GatewayIndexStateIT extends ESIntegTestCase {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(GatewayIndexStateIT.class);
|
||||
|
||||
@Override
|
||||
protected boolean addMockInternalEngine() {
|
||||
// testRecoverBrokenIndexMetadata replies on the flushing on shutdown behavior which can be randomly disabled in MockInternalEngine.
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.indices.state;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.support.ActiveShardCount;
|
||||
|
@ -27,8 +25,6 @@ import static org.hamcrest.Matchers.notNullValue;
|
|||
|
||||
@ESIntegTestCase.ClusterScope(minNumDataNodes = 2)
|
||||
public class SimpleIndexStateIT extends ESIntegTestCase {
|
||||
private final Logger logger = LogManager.getLogger(SimpleIndexStateIT.class);
|
||||
|
||||
public void testSimpleOpenClose() {
|
||||
logger.info("--> creating test index");
|
||||
createIndex("test");
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.recovery;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||
|
@ -47,7 +45,6 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitC
|
|||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout;
|
||||
|
||||
public class RecoveryWhileUnderLoadIT extends ESIntegTestCase {
|
||||
private final Logger logger = LogManager.getLogger(RecoveryWhileUnderLoadIT.class);
|
||||
|
||||
public static final class RetentionLeaseSyncIntervalSettingPlugin extends Plugin {
|
||||
|
||||
|
|
|
@ -43,7 +43,7 @@ public class TransportVerifyShardBeforeCloseAction extends TransportReplicationA
|
|||
|
||||
public static final String NAME = CloseIndexAction.NAME + "[s]";
|
||||
public static final ActionType<ReplicationResponse> TYPE = new ActionType<>(NAME, ReplicationResponse::new);
|
||||
protected Logger logger = LogManager.getLogger(getClass());
|
||||
private static final Logger logger = LogManager.getLogger(TransportVerifyShardBeforeCloseAction.class);
|
||||
|
||||
@Inject
|
||||
public TransportVerifyShardBeforeCloseAction(
|
||||
|
|
|
@ -7,8 +7,6 @@
|
|||
*/
|
||||
package org.elasticsearch.action.admin.indices.readonly;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
|
@ -48,7 +46,6 @@ public class TransportVerifyShardIndexBlockAction extends TransportReplicationAc
|
|||
|
||||
public static final String NAME = AddIndexBlockAction.NAME + "[s]";
|
||||
public static final ActionType<ReplicationResponse> TYPE = new ActionType<>(NAME, ReplicationResponse::new);
|
||||
protected Logger logger = LogManager.getLogger(getClass());
|
||||
|
||||
@Inject
|
||||
public TransportVerifyShardIndexBlockAction(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.action.admin.indices.shrink;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
|
||||
|
@ -44,7 +42,6 @@ import java.util.Locale;
|
|||
* Main class to initiate resizing (shrink / split) an index into a new index
|
||||
*/
|
||||
public class TransportResizeAction extends TransportMasterNodeAction<ResizeRequest, ResizeResponse> {
|
||||
private static final Logger logger = LogManager.getLogger(TransportResizeAction.class);
|
||||
|
||||
private final MetadataCreateIndexService createIndexService;
|
||||
private final Client client;
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.util.function.BiConsumer;
|
|||
* Implements the low-level details of bulk request handling
|
||||
*/
|
||||
public final class BulkRequestHandler {
|
||||
private final Logger logger;
|
||||
private static final Logger logger = LogManager.getLogger(BulkRequestHandler.class);
|
||||
private final BiConsumer<BulkRequest, ActionListener<BulkResponse>> consumer;
|
||||
private final BulkProcessor.Listener listener;
|
||||
private final Semaphore semaphore;
|
||||
|
@ -36,7 +36,6 @@ public final class BulkRequestHandler {
|
|||
int concurrentRequests
|
||||
) {
|
||||
assert concurrentRequests >= 0;
|
||||
this.logger = LogManager.getLogger(getClass());
|
||||
this.consumer = consumer;
|
||||
this.listener = listener;
|
||||
this.concurrentRequests = concurrentRequests;
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.elasticsearch.transport.TransportService;
|
|||
import java.util.function.LongConsumer;
|
||||
|
||||
public abstract class PreVoteCollector {
|
||||
private final Logger logger = LogManager.getLogger(PreVoteCollector.class);
|
||||
private static final Logger logger = LogManager.getLogger(PreVoteCollector.class);
|
||||
|
||||
// Tuple for simple atomic updates. null until the first call to `update()`.
|
||||
protected volatile Tuple<DiscoveryNode, PreVoteResponse> state; // DiscoveryNode component is null if there is currently no known
|
||||
|
|
|
@ -36,7 +36,7 @@ import java.util.function.Function;
|
|||
*/
|
||||
public class FeatureFlag {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(FeatureFlag.class);
|
||||
private static final Logger logger = LogManager.getLogger(FeatureFlag.class);
|
||||
|
||||
private final String name;
|
||||
private final boolean enabled;
|
||||
|
|
|
@ -30,7 +30,7 @@ import static org.elasticsearch.core.Strings.format;
|
|||
*/
|
||||
class HttpTracer {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(HttpTracer.class);
|
||||
private static final Logger logger = LogManager.getLogger(HttpTracer.class);
|
||||
|
||||
private volatile String[] tracerLogInclude;
|
||||
private volatile String[] tracerLogExclude;
|
||||
|
|
|
@ -238,7 +238,6 @@ public class PrimaryReplicaSyncer {
|
|||
}
|
||||
|
||||
static class SnapshotSender extends AbstractRunnable implements ActionListener<ResyncReplicationResponse> {
|
||||
private final Logger logger;
|
||||
private final SyncAction syncAction;
|
||||
private final ResyncTask task; // to track progress
|
||||
private final String primaryAllocationId;
|
||||
|
@ -270,7 +269,6 @@ public class PrimaryReplicaSyncer {
|
|||
Executor executor,
|
||||
ActionListener<Void> listener
|
||||
) {
|
||||
this.logger = PrimaryReplicaSyncer.logger;
|
||||
this.syncAction = syncAction;
|
||||
this.task = task;
|
||||
this.shardId = shardId;
|
||||
|
@ -353,7 +351,7 @@ public class PrimaryReplicaSyncer {
|
|||
maxSeenAutoIdTimestamp,
|
||||
operations.toArray(EMPTY_ARRAY)
|
||||
);
|
||||
logger.trace(
|
||||
PrimaryReplicaSyncer.logger.trace(
|
||||
"{} sending batch of [{}][{}] (total sent: [{}], skipped: [{}])",
|
||||
shardId,
|
||||
operations.size(),
|
||||
|
@ -364,7 +362,12 @@ public class PrimaryReplicaSyncer {
|
|||
firstMessage.set(false);
|
||||
syncAction.sync(request, task, primaryAllocationId, primaryTerm, this);
|
||||
} else if (closed.compareAndSet(false, true)) {
|
||||
logger.trace("{} resync completed (total sent: [{}], skipped: [{}])", shardId, totalSentOps.get(), totalSkippedOps.get());
|
||||
PrimaryReplicaSyncer.logger.trace(
|
||||
"{} resync completed (total sent: [{}], skipped: [{}])",
|
||||
shardId,
|
||||
totalSentOps.get(),
|
||||
totalSkippedOps.get()
|
||||
);
|
||||
listener.onResponse(null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -335,7 +335,7 @@ public class IndicesService extends AbstractLifecycleComponent
|
|||
}
|
||||
});
|
||||
this.cleanInterval = INDICES_CACHE_CLEAN_INTERVAL_SETTING.get(settings);
|
||||
this.cacheCleaner = new CacheCleaner(indicesFieldDataCache, indicesRequestCache, logger, threadPool, this.cleanInterval);
|
||||
this.cacheCleaner = new CacheCleaner(indicesFieldDataCache, indicesRequestCache, threadPool, this.cleanInterval);
|
||||
this.metaStateService = metaStateService;
|
||||
this.engineFactoryProviders = engineFactoryProviders;
|
||||
|
||||
|
@ -1440,22 +1440,14 @@ public class IndicesService extends AbstractLifecycleComponent
|
|||
private static final class CacheCleaner implements Runnable, Releasable {
|
||||
|
||||
private final IndicesFieldDataCache cache;
|
||||
private final Logger logger;
|
||||
private final ThreadPool threadPool;
|
||||
private final TimeValue interval;
|
||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
private final IndicesRequestCache requestCache;
|
||||
|
||||
CacheCleaner(
|
||||
IndicesFieldDataCache cache,
|
||||
IndicesRequestCache requestCache,
|
||||
Logger logger,
|
||||
ThreadPool threadPool,
|
||||
TimeValue interval
|
||||
) {
|
||||
CacheCleaner(IndicesFieldDataCache cache, IndicesRequestCache requestCache, ThreadPool threadPool, TimeValue interval) {
|
||||
this.cache = cache;
|
||||
this.requestCache = requestCache;
|
||||
this.logger = logger;
|
||||
this.threadPool = threadPool;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
|
|
@ -67,7 +67,7 @@ public final class IndicesFieldDataCache implements RemovalListener<IndicesField
|
|||
}
|
||||
|
||||
public IndexFieldDataCache buildIndexFieldDataCache(IndexFieldDataCache.Listener listener, Index index, String fieldName) {
|
||||
return new IndexFieldCache(logger, cache, index, fieldName, indicesFieldDataCacheListener, listener);
|
||||
return new IndexFieldCache(cache, index, fieldName, indicesFieldDataCacheListener, listener);
|
||||
}
|
||||
|
||||
public Cache<Key, Accountable> getCache() {
|
||||
|
@ -107,14 +107,12 @@ public final class IndicesFieldDataCache implements RemovalListener<IndicesField
|
|||
* A specific cache instance for the relevant parameters of it (index, fieldNames, fieldType).
|
||||
*/
|
||||
static class IndexFieldCache implements IndexFieldDataCache, IndexReader.ClosedListener {
|
||||
private final Logger logger;
|
||||
final Index index;
|
||||
final String fieldName;
|
||||
private final Cache<Key, Accountable> cache;
|
||||
private final Listener[] listeners;
|
||||
|
||||
IndexFieldCache(Logger logger, final Cache<Key, Accountable> cache, Index index, String fieldName, Listener... listeners) {
|
||||
this.logger = logger;
|
||||
IndexFieldCache(final Cache<Key, Accountable> cache, Index index, String fieldName, Listener... listeners) {
|
||||
this.listeners = listeners;
|
||||
this.index = index;
|
||||
this.fieldName = fieldName;
|
||||
|
|
|
@ -54,7 +54,7 @@ import static org.elasticsearch.core.Strings.format;
|
|||
import static org.elasticsearch.indices.recovery.RecoverySettings.SNAPSHOT_RECOVERIES_SUPPORTED_VERSION;
|
||||
|
||||
public class ShardSnapshotsService {
|
||||
private final Logger logger = LogManager.getLogger(ShardSnapshotsService.class);
|
||||
private static final Logger logger = LogManager.getLogger(ShardSnapshotsService.class);
|
||||
|
||||
private final Client client;
|
||||
private final RepositoriesService repositoriesService;
|
||||
|
|
|
@ -35,7 +35,7 @@ import static org.elasticsearch.xcontent.XContentType.JSON;
|
|||
*/
|
||||
public class NamedComponentReader {
|
||||
|
||||
private Logger logger = LogManager.getLogger(NamedComponentReader.class);
|
||||
private static final Logger logger = LogManager.getLogger(NamedComponentReader.class);
|
||||
private static final String NAMED_COMPONENTS_FILE_NAME = "named_components.json";
|
||||
/**
|
||||
* a registry of known classes marked or indirectly marked (extending marked class) with @Extensible
|
||||
|
|
|
@ -93,7 +93,7 @@ public class MultiBucketConsumerService {
|
|||
* {@link Aggregator#buildAggregations} and {@link InternalAggregation#reduce}.
|
||||
*/
|
||||
public static class MultiBucketConsumer implements IntConsumer {
|
||||
private final Logger logger = LogManager.getLogger(MultiBucketConsumer.class);
|
||||
private static final Logger logger = LogManager.getLogger(MultiBucketConsumer.class);
|
||||
private final int limit;
|
||||
private final CircuitBreaker breaker;
|
||||
|
||||
|
|
|
@ -70,7 +70,7 @@ import static org.elasticsearch.search.aggregations.MultiBucketConsumerService.M
|
|||
|
||||
public final class CompositeAggregator extends BucketsAggregator implements SizedBucketAggregator {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(CompositeAggregator.class);
|
||||
private static final Logger logger = LogManager.getLogger(CompositeAggregator.class);
|
||||
private final int size;
|
||||
private final List<String> sourceNames;
|
||||
private final int[] reverseMuls;
|
||||
|
|
|
@ -48,7 +48,7 @@ final class TransportKeepAlive implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private final Logger logger = LogManager.getLogger(TransportKeepAlive.class);
|
||||
private static final Logger logger = LogManager.getLogger(TransportKeepAlive.class);
|
||||
private final CounterMetric successfulPings = new CounterMetric();
|
||||
private final CounterMetric failedPings = new CounterMetric();
|
||||
private final ConcurrentMap<TimeValue, ScheduledPing> pingIntervals = ConcurrentCollections.newConcurrentMap();
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.action.bulk;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
|
@ -52,7 +50,6 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
|||
public class BulkProcessor2Tests extends ESTestCase {
|
||||
|
||||
private ThreadPool threadPool;
|
||||
private final Logger logger = LogManager.getLogger(BulkProcessor2Tests.class);
|
||||
|
||||
@Before
|
||||
public void startThreadPool() {
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.action.bulk;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
|
@ -59,7 +57,6 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
|||
public class BulkProcessorTests extends ESTestCase {
|
||||
|
||||
private ThreadPool threadPool;
|
||||
private final Logger logger = LogManager.getLogger(BulkProcessorTests.class);
|
||||
|
||||
@Before
|
||||
public void startThreadPool() {
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -39,7 +37,6 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
|||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class AddIncrementallyTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(AddIncrementallyTests.class);
|
||||
|
||||
public void testAddNodesAndIndices() {
|
||||
Settings.Builder settings = Settings.builder();
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterInfo;
|
||||
|
@ -78,7 +76,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class AllocationCommandsTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(AllocationCommandsTests.class);
|
||||
|
||||
public void testMoveShardCommand() {
|
||||
AllocationService allocation = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -54,8 +52,6 @@ import static org.hamcrest.Matchers.sameInstance;
|
|||
|
||||
public class AwarenessAllocationTests extends ESAllocationTestCase {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(AwarenessAllocationTests.class);
|
||||
|
||||
public void testMoveShardOnceNewNodeWithAttributeAdded1() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
Settings.builder()
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -39,7 +37,6 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
|||
|
||||
public class BalanceConfigurationTests extends ESAllocationTestCase {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(BalanceConfigurationTests.class);
|
||||
// TODO maybe we can randomize these numbers somehow
|
||||
final int numberOfNodes = 25;
|
||||
final int numberOfIndices = 12;
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -38,7 +36,6 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ClusterRebalanceRoutingTests.class);
|
||||
|
||||
public void testAlways() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -31,7 +29,6 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ConcurrentRebalanceRoutingTests.class);
|
||||
|
||||
public void testClusterConcurrentRebalance() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -29,7 +27,6 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class ElectReplicaAsPrimaryDuringRelocationTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ElectReplicaAsPrimaryDuringRelocationTests.class);
|
||||
|
||||
public void testElectReplicaAsPrimaryDuringRelocation() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterInfo;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
|
@ -34,8 +32,6 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ExpectedShardSizeAllocationTests.class);
|
||||
|
||||
public void testInitializingHasExpectedSize() {
|
||||
final long byteSize = randomIntBetween(0, Integer.MAX_VALUE);
|
||||
final ClusterInfo clusterInfo = createClusterInfoWith(new ShardId("test", "_na_", 0), byteSize);
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteRequest;
|
||||
|
@ -55,7 +53,6 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class FailedNodeRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(FailedNodeRoutingTests.class);
|
||||
|
||||
public void testSimpleFailedNodeTest() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
|
@ -49,7 +47,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class FailedShardsRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(FailedShardsRoutingTests.class);
|
||||
|
||||
public void testFailedShardPrimaryRelocatingToAndFrom() {
|
||||
AllocationService allocation = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -42,7 +40,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class IndexBalanceTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(IndexBalanceTests.class);
|
||||
|
||||
public void testBalanceAllNodesStarted() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
|
@ -78,8 +76,6 @@ import static org.hamcrest.core.Is.is;
|
|||
|
||||
public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(NodeVersionAllocationDeciderTests.class);
|
||||
|
||||
public void testDoNotAllocateFromPrimary() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
Settings.builder()
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -28,7 +26,6 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class PreferPrimaryAllocationTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(PreferPrimaryAllocationTests.class);
|
||||
|
||||
public void testPreferPrimaryAllocationOverReplicas() {
|
||||
logger.info("create an allocation with 1 initial recoveries");
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -31,7 +29,6 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(PrimaryElectionRoutingTests.class);
|
||||
|
||||
public void testBackupElectionToPrimaryWhenPrimaryCanBeAllocatedToAnotherNode() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -29,8 +27,6 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class PrimaryNotRelocatedWhileBeingRecoveredTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(PrimaryNotRelocatedWhileBeingRecoveredTests.class);
|
||||
|
||||
public void testPrimaryNotRelocatedWhileBeingRecoveredFrom() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
Settings.builder()
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterInfo;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
|
@ -37,7 +35,6 @@ import static org.hamcrest.Matchers.hasSize;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class RebalanceAfterActiveTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(RebalanceAfterActiveTests.class);
|
||||
|
||||
public void testRebalanceOnlyAfterAllShardsAreActive() {
|
||||
final long[] sizes = new long[5];
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -31,7 +29,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class ReplicaAllocatedAfterPrimaryTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ReplicaAllocatedAfterPrimaryTests.class);
|
||||
|
||||
public void testBackupIsAllocatedAfterPrimary() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -29,8 +27,6 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class ShardVersioningTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ShardVersioningTests.class);
|
||||
|
||||
public void testSimple() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
Settings.builder()
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -31,7 +29,6 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class ShardsLimitAllocationTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ShardsLimitAllocationTests.class);
|
||||
|
||||
public void testIndexLevelShardsLimitAllocate() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -44,7 +42,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(SingleShardNoReplicasRoutingTests.class);
|
||||
|
||||
public void testSingleIndexStartedShard() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -31,7 +29,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class SingleShardOneReplicaRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(SingleShardOneReplicaRoutingTests.class);
|
||||
|
||||
public void testSingleIndexFirstStartPrimaryThenBackups() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -34,7 +32,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(TenShardsOneReplicaRoutingTests.class);
|
||||
|
||||
public void testSingleIndexFirstStartPrimaryThenBackups() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -59,7 +57,6 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class ThrottlingAllocationTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(ThrottlingAllocationTests.class);
|
||||
|
||||
public void testPrimaryRecoveryThrottling() {
|
||||
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.routing.allocation;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -31,7 +29,6 @@ import static org.hamcrest.Matchers.not;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
|
||||
private final Logger logger = LogManager.getLogger(UpdateNumberOfReplicasTests.class);
|
||||
|
||||
public void testUpdateNumberOfReplicas() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
|
|
|
@ -10,8 +10,6 @@ package org.elasticsearch.cluster.routing.allocation.decider;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -45,8 +43,6 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
|
||||
public class EnableAllocationTests extends ESAllocationTestCase {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(EnableAllocationTests.class);
|
||||
|
||||
public void testClusterEnableNone() {
|
||||
AllocationService strategy = createAllocationService(
|
||||
Settings.builder().put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NONE.name()).build()
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
|
||||
package org.elasticsearch.snapshots;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
|
@ -1557,8 +1555,6 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
|||
|
||||
private final class TestClusterNode {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(TestClusterNode.class);
|
||||
|
||||
private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(
|
||||
Stream.concat(ClusterModule.getNamedWriteables().stream(), NetworkModule.getNamedWriteables().stream()).toList()
|
||||
);
|
||||
|
|
|
@ -942,7 +942,7 @@ public class AbstractCoordinatorTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public final class ClusterNode {
|
||||
private final Logger logger = LogManager.getLogger(ClusterNode.class);
|
||||
private static final Logger logger = LogManager.getLogger(ClusterNode.class);
|
||||
|
||||
private final int nodeIndex;
|
||||
Coordinator coordinator;
|
||||
|
|
|
@ -44,7 +44,7 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
|
||||
public final class BackgroundIndexer implements AutoCloseable {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(getClass());
|
||||
private static final Logger logger = LogManager.getLogger(BackgroundIndexer.class);
|
||||
|
||||
final Thread[] writers;
|
||||
final Client client;
|
||||
|
|
|
@ -174,7 +174,7 @@ import static org.junit.Assert.fail;
|
|||
*/
|
||||
public final class InternalTestCluster extends TestCluster {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(getClass());
|
||||
private static final Logger logger = LogManager.getLogger(InternalTestCluster.class);
|
||||
|
||||
private static final Predicate<NodeAndClient> DATA_NODE_PREDICATE = nodeAndClient -> DiscoveryNode.canContainData(
|
||||
nodeAndClient.node.settings()
|
||||
|
|
|
@ -37,6 +37,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
*/
|
||||
public final class MockEngineSupport {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(Engine.class);
|
||||
|
||||
/**
|
||||
* Allows tests to wrap an index reader randomly with a given ratio. This
|
||||
* is disabled by default ie. {@code 0.0d} since reader wrapping is insanely
|
||||
|
@ -58,7 +60,6 @@ public final class MockEngineSupport {
|
|||
);
|
||||
|
||||
private final AtomicBoolean closing = new AtomicBoolean(false);
|
||||
private final Logger logger = LogManager.getLogger(Engine.class);
|
||||
private final ShardId shardId;
|
||||
private final InFlightSearchers inFlightSearchers;
|
||||
private final MockContext mockContext;
|
||||
|
@ -178,7 +179,7 @@ public final class MockEngineSupport {
|
|||
* early. - good news, stuff will fail all over the place if we don't
|
||||
* get this right here
|
||||
*/
|
||||
SearcherCloseable closeable = new SearcherCloseable(searcher, logger, inFlightSearchers);
|
||||
SearcherCloseable closeable = new SearcherCloseable(searcher, inFlightSearchers);
|
||||
return new Engine.Searcher(
|
||||
searcher.source(),
|
||||
reader,
|
||||
|
@ -222,12 +223,10 @@ public final class MockEngineSupport {
|
|||
private RuntimeException firstReleaseStack;
|
||||
private final Object lock = new Object();
|
||||
private final int initialRefCount;
|
||||
private final Logger logger;
|
||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
|
||||
SearcherCloseable(final Engine.Searcher searcher, Logger logger, InFlightSearchers inFlightSearchers) {
|
||||
SearcherCloseable(final Engine.Searcher searcher, InFlightSearchers inFlightSearchers) {
|
||||
this.searcher = searcher;
|
||||
this.logger = logger;
|
||||
initialRefCount = searcher.getIndexReader().getRefCount();
|
||||
this.inFlightSearchers = inFlightSearchers;
|
||||
assert initialRefCount > 0
|
||||
|
|
|
@ -38,7 +38,7 @@ import static org.elasticsearch.test.ESTestCase.copyWriteable;
|
|||
|
||||
public abstract class DisruptableMockTransport extends MockTransport {
|
||||
private final DiscoveryNode localNode;
|
||||
private final Logger logger = LogManager.getLogger(DisruptableMockTransport.class);
|
||||
private static final Logger logger = LogManager.getLogger(DisruptableMockTransport.class);
|
||||
private final DeterministicTaskQueue deterministicTaskQueue;
|
||||
private final List<Runnable> blackholedRequests = new ArrayList<>();
|
||||
private final Set<String> blockedActions = new HashSet<>();
|
||||
|
|
|
@ -28,7 +28,7 @@ import static org.elasticsearch.core.Strings.format;
|
|||
* is still running and AsyncTaskIndexService if task results already stored there.
|
||||
*/
|
||||
public class AsyncResultsService<Task extends AsyncTask, Response extends AsyncResponse<Response>> {
|
||||
private final Logger logger = LogManager.getLogger(AsyncResultsService.class);
|
||||
private static final Logger logger = LogManager.getLogger(AsyncResultsService.class);
|
||||
private final Class<? extends Task> asyncTaskClass;
|
||||
private final TaskManager taskManager;
|
||||
private final ClusterService clusterService;
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.function.Consumer;
|
|||
* is still running and AsyncTaskIndexService if task results already stored there.
|
||||
*/
|
||||
public class DeleteAsyncResultsService {
|
||||
private final Logger logger = LogManager.getLogger(DeleteAsyncResultsService.class);
|
||||
private static final Logger logger = LogManager.getLogger(DeleteAsyncResultsService.class);
|
||||
private final TaskManager taskManager;
|
||||
private final AsyncTaskIndexService<? extends AsyncResponse<?>> store;
|
||||
|
||||
|
|
|
@ -102,7 +102,7 @@ public final class DocumentSubsetBitsetCache implements IndexReader.ClosedListen
|
|||
|
||||
private static final BitSet NULL_MARKER = new FixedBitSet(0);
|
||||
|
||||
private final Logger logger;
|
||||
private static final Logger logger = LogManager.getLogger(DocumentSubsetBitsetCache.class);
|
||||
|
||||
/**
|
||||
* When a {@link BitSet} is evicted from {@link #bitsetCache}, we need to also remove it from {@link #keysByIndex}.
|
||||
|
@ -131,8 +131,6 @@ public final class DocumentSubsetBitsetCache implements IndexReader.ClosedListen
|
|||
* it is sometimes necessary to run an asynchronous task to synchronize the internal state.
|
||||
*/
|
||||
protected DocumentSubsetBitsetCache(Settings settings, ExecutorService cleanupExecutor) {
|
||||
this.logger = LogManager.getLogger(getClass());
|
||||
|
||||
final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock();
|
||||
this.cacheEvictionLock = new ReleasableLock(readWriteLock.writeLock());
|
||||
this.cacheModificationLock = new ReleasableLock(readWriteLock.readLock());
|
||||
|
|
|
@ -34,7 +34,7 @@ public final class ApplicationPermission {
|
|||
|
||||
public static final ApplicationPermission NONE = new ApplicationPermission(Collections.emptyList());
|
||||
|
||||
private final Logger logger;
|
||||
private static final Logger logger = LogManager.getLogger(ApplicationPermission.class);
|
||||
private final List<PermissionEntry> permissions;
|
||||
|
||||
/**
|
||||
|
@ -43,7 +43,6 @@ public final class ApplicationPermission {
|
|||
* applied. The resources are treated as a wildcard {@link Automatons#pattern}.
|
||||
*/
|
||||
ApplicationPermission(List<Tuple<ApplicationPrivilege, Set<String>>> privilegesAndResources) {
|
||||
this.logger = LogManager.getLogger(getClass());
|
||||
Map<ApplicationPrivilege, PermissionEntry> permissionsByPrivilege = new HashMap<>();
|
||||
privilegesAndResources.forEach(tup -> permissionsByPrivilege.compute(tup.v1(), (appPriv, existing) -> {
|
||||
final Set<String> resourceNames = tup.v2();
|
||||
|
|
|
@ -14,8 +14,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
|||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.logging.LogManager;
|
||||
import org.elasticsearch.logging.Logger;
|
||||
import org.elasticsearch.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.xcontent.ParseField;
|
||||
import org.elasticsearch.xcontent.ToXContentObject;
|
||||
|
@ -56,8 +54,6 @@ public class QueryRule implements Writeable, ToXContentObject {
|
|||
private final List<QueryRuleCriteria> criteria;
|
||||
private final Map<String, Object> actions;
|
||||
|
||||
private final Logger logger = LogManager.getLogger(QueryRule.class);
|
||||
|
||||
public enum QueryRuleType {
|
||||
PINNED;
|
||||
|
||||
|
|
|
@ -25,8 +25,6 @@ import org.elasticsearch.index.query.QueryRewriteContext;
|
|||
import org.elasticsearch.index.query.SearchExecutionContext;
|
||||
import org.elasticsearch.license.LicenseUtils;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.logging.LogManager;
|
||||
import org.elasticsearch.logging.Logger;
|
||||
import org.elasticsearch.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.xcontent.ParseField;
|
||||
import org.elasticsearch.xcontent.XContentBuilder;
|
||||
|
@ -69,8 +67,6 @@ public class RuleQueryBuilder extends AbstractQueryBuilder<RuleQueryBuilder> {
|
|||
private final List<Item> pinnedDocs;
|
||||
private final Supplier<List<Item>> pinnedDocsSupplier;
|
||||
|
||||
private final Logger logger = LogManager.getLogger(RuleQueryBuilder.class);
|
||||
|
||||
@Override
|
||||
public TransportVersion getMinimalSupportedVersion() {
|
||||
return TransportVersions.V_8_500_040;
|
||||
|
|
|
@ -12,8 +12,6 @@ import com.fasterxml.jackson.core.JsonProcessingException;
|
|||
|
||||
import org.elasticsearch.common.ValidationException;
|
||||
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
|
||||
import org.elasticsearch.logging.LogManager;
|
||||
import org.elasticsearch.logging.Logger;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.script.TemplateScript;
|
||||
|
@ -35,8 +33,6 @@ public class SearchApplicationTemplateService {
|
|||
private final ScriptService scriptService;
|
||||
private final NamedXContentRegistry xContentRegistry;
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SearchApplicationTemplateService.class);
|
||||
|
||||
public SearchApplicationTemplateService(ScriptService scriptService, NamedXContentRegistry xContentRegistry) {
|
||||
this.scriptService = scriptService;
|
||||
this.xContentRegistry = xContentRegistry;
|
||||
|
|
|
@ -47,7 +47,7 @@ import static org.elasticsearch.xpack.eql.execution.search.RuntimeUtils.prepareR
|
|||
|
||||
public class SampleIterator implements Executable {
|
||||
|
||||
private final Logger log = LogManager.getLogger(SampleIterator.class);
|
||||
private static final Logger log = LogManager.getLogger(SampleIterator.class);
|
||||
|
||||
private final QueryClient client;
|
||||
private final List<SampleCriterion> criteria;
|
||||
|
|
|
@ -32,7 +32,7 @@ public class SequenceMatcher {
|
|||
private static final String CB_INFLIGHT_LABEL = "sequence_inflight";
|
||||
private static final String CB_COMPLETED_LABEL = "sequence_completed";
|
||||
|
||||
private final Logger log = LogManager.getLogger(SequenceMatcher.class);
|
||||
private static final Logger log = LogManager.getLogger(SequenceMatcher.class);
|
||||
|
||||
static class Stats {
|
||||
|
||||
|
|
|
@ -77,7 +77,7 @@ public class TumblingWindow implements Executable {
|
|||
*/
|
||||
private static final int MISSING_EVENTS_SEQUENCES_CHECK_BATCH_SIZE = 1000;
|
||||
|
||||
private final Logger log = LogManager.getLogger(TumblingWindow.class);
|
||||
private static final Logger log = LogManager.getLogger(TumblingWindow.class);
|
||||
|
||||
/**
|
||||
* Simple cache for removing duplicate strings (such as index name or common keys).
|
||||
|
|
|
@ -7,8 +7,6 @@
|
|||
|
||||
package org.elasticsearch.xpack.idp;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
|
@ -68,7 +66,6 @@ public class IdentityProviderPlugin extends Plugin implements ActionPlugin {
|
|||
|
||||
private static final Setting<Boolean> ENABLED_SETTING = Setting.boolSetting("xpack.idp.enabled", false, Setting.Property.NodeScope);
|
||||
|
||||
private final Logger logger = LogManager.getLogger(IdentityProviderPlugin.class);
|
||||
private boolean enabled;
|
||||
private Settings settings;
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ public class TransportDeleteSamlServiceProviderAction extends HandledTransportAc
|
|||
DeleteSamlServiceProviderRequest,
|
||||
DeleteSamlServiceProviderResponse> {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(TransportDeleteSamlServiceProviderAction.class);
|
||||
private static final Logger logger = LogManager.getLogger(TransportDeleteSamlServiceProviderAction.class);
|
||||
private final SamlServiceProviderIndex index;
|
||||
|
||||
@Inject
|
||||
|
|
|
@ -35,7 +35,7 @@ public class TransportPutSamlServiceProviderAction extends HandledTransportActio
|
|||
PutSamlServiceProviderRequest,
|
||||
PutSamlServiceProviderResponse> {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(TransportPutSamlServiceProviderAction.class);
|
||||
private static final Logger logger = LogManager.getLogger(TransportPutSamlServiceProviderAction.class);
|
||||
private final SamlServiceProviderIndex index;
|
||||
private final SamlIdentityProvider identityProvider;
|
||||
private final Clock clock;
|
||||
|
|
|
@ -38,7 +38,7 @@ public class TransportSamlInitiateSingleSignOnAction extends HandledTransportAct
|
|||
SamlInitiateSingleSignOnRequest,
|
||||
SamlInitiateSingleSignOnResponse> {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(TransportSamlInitiateSingleSignOnAction.class);
|
||||
private static final Logger logger = LogManager.getLogger(TransportSamlInitiateSingleSignOnAction.class);
|
||||
|
||||
private final SecurityContext securityContext;
|
||||
private final SamlIdentityProvider identityProvider;
|
||||
|
|
|
@ -63,7 +63,7 @@ public class ApplicationActionsResolver extends AbstractLifecycleComponent {
|
|||
Setting.Property.NodeScope
|
||||
);
|
||||
|
||||
private final Logger logger = LogManager.getLogger(ApplicationActionsResolver.class);
|
||||
private static final Logger logger = LogManager.getLogger(ApplicationActionsResolver.class);
|
||||
|
||||
private final ServiceProviderDefaults defaults;
|
||||
private final Client client;
|
||||
|
|
|
@ -62,7 +62,7 @@ public class UserPrivilegeResolver {
|
|||
}
|
||||
}
|
||||
|
||||
private final Logger logger = LogManager.getLogger(UserPrivilegeResolver.class);
|
||||
private static final Logger logger = LogManager.getLogger(UserPrivilegeResolver.class);
|
||||
private final Client client;
|
||||
private final SecurityContext securityContext;
|
||||
private final ApplicationActionsResolver actionsResolver;
|
||||
|
|
|
@ -60,7 +60,7 @@ public class SamlAuthnRequestValidator {
|
|||
|
||||
private final SamlFactory samlFactory;
|
||||
private final SamlIdentityProvider idp;
|
||||
private final Logger logger = LogManager.getLogger(SamlAuthnRequestValidator.class);
|
||||
private static final Logger logger = LogManager.getLogger(SamlAuthnRequestValidator.class);
|
||||
private static final String[] XSD_FILES = new String[] {
|
||||
"/org/elasticsearch/xpack/idp/saml/support/saml-schema-protocol-2.0.xsd",
|
||||
"/org/elasticsearch/xpack/idp/saml/support/saml-schema-assertion-2.0.xsd",
|
||||
|
|
|
@ -53,7 +53,7 @@ import static org.opensaml.saml.saml2.core.NameIDType.TRANSIENT;
|
|||
*/
|
||||
public class SuccessfulAuthenticationResponseMessageBuilder {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SuccessfulAuthenticationResponseMessageBuilder.class);
|
||||
private static final Logger logger = LogManager.getLogger(SuccessfulAuthenticationResponseMessageBuilder.class);
|
||||
|
||||
private final Clock clock;
|
||||
private final SamlIdentityProvider idp;
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.util.stream.Stream;
|
|||
*/
|
||||
public class SamlIdentityProvider {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SamlIdentityProvider.class);
|
||||
private static final Logger logger = LogManager.getLogger(SamlIdentityProvider.class);
|
||||
|
||||
private final String entityId;
|
||||
private final Map<String, URL> ssoEndpoints;
|
||||
|
|
|
@ -34,7 +34,7 @@ public class SamlMetadataGenerator {
|
|||
|
||||
private final SamlFactory samlFactory;
|
||||
private final SamlIdentityProvider idp;
|
||||
private final Logger logger = LogManager.getLogger(SamlMetadataGenerator.class);
|
||||
private static final Logger logger = LogManager.getLogger(SamlMetadataGenerator.class);
|
||||
|
||||
public SamlMetadataGenerator(SamlFactory samlFactory, SamlIdentityProvider idp) {
|
||||
this.samlFactory = samlFactory;
|
||||
|
|
|
@ -65,7 +65,7 @@ import java.util.stream.Stream;
|
|||
*/
|
||||
public class SamlServiceProviderIndex implements Closeable {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SamlServiceProviderIndex.class);
|
||||
private static final Logger logger = LogManager.getLogger(SamlServiceProviderIndex.class);
|
||||
|
||||
private final Client client;
|
||||
private final ClusterService clusterService;
|
||||
|
|
|
@ -71,7 +71,7 @@ public class MlMemoryTracker implements LocalNodeMasterListener {
|
|||
private static final Duration RECENT_UPDATE_THRESHOLD = Duration.ofMinutes(1);
|
||||
private static final Duration DEFAULT_AUTOSCALING_CHECK_INTERVAL = Duration.ofMinutes(5);
|
||||
|
||||
private final Logger logger = LogManager.getLogger(MlMemoryTracker.class);
|
||||
private static final Logger logger = LogManager.getLogger(MlMemoryTracker.class);
|
||||
private final Map<String, Long> memoryRequirementByAnomalyDetectorJob = new ConcurrentHashMap<>();
|
||||
private final Map<String, Long> memoryRequirementByDataFrameAnalyticsJob = new ConcurrentHashMap<>();
|
||||
private final Map<String, Map<String, Long>> memoryRequirementByTaskName;
|
||||
|
|
|
@ -8,8 +8,6 @@
|
|||
package org.elasticsearch.xpack.ssl;
|
||||
|
||||
import org.apache.logging.log4j.Level;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.util.Strings;
|
||||
import org.elasticsearch.common.settings.MockSecureSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -35,7 +33,6 @@ import java.util.concurrent.CountDownLatch;
|
|||
* by verifying if the cluster goes back to normal.
|
||||
*/
|
||||
public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
||||
private final Logger LOGGER = LogManager.getLogger(SSLReloadDuringStartupIntegTests.class);
|
||||
|
||||
private final String goodKeyStoreFilePath = "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks";
|
||||
private final String badKeyStoreFilePath = "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.jks";
|
||||
|
@ -100,18 +97,18 @@ public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
|||
final Path badKeystorePath = getDataPath(badKeyStoreFilePath); // stop a node, and apply this bad keystore
|
||||
final Path goodKeystorePath = getDataPath(goodKeyStoreFilePath); // start the node, and apply this good keystore
|
||||
assertTrue(Files.exists(nodeKeystorePath));
|
||||
LOGGER.trace("Stopping node [{}] in {}-node cluster {}...", nodeName, nodeNames.length, nodeNames);
|
||||
logger.trace("Stopping node [{}] in {}-node cluster {}...", nodeName, nodeNames.length, nodeNames);
|
||||
final long stopNanos = System.nanoTime();
|
||||
internalCluster().restartNode(nodeName, new RestartCallback() {
|
||||
@Override
|
||||
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||
LOGGER.debug("Node [{}] stopped in {}ms.", nodeName, TimeValue.timeValueNanos(System.nanoTime() - stopNanos).millisFrac());
|
||||
logger.debug("Node [{}] stopped in {}ms.", nodeName, TimeValue.timeValueNanos(System.nanoTime() - stopNanos).millisFrac());
|
||||
atomicCopyIfPossible(badKeystorePath, nodeKeystorePath);
|
||||
final Thread fixKeystoreThread = new Thread(() -> {
|
||||
waitUntilNodeStartupIsReadyToBegin(beforeKeystoreFix); // SYNC: Cert update & ES restart
|
||||
try {
|
||||
atomicCopyIfPossible(goodKeystorePath, nodeKeystorePath);
|
||||
LOGGER.trace("Waiting for ES restart...");
|
||||
logger.trace("Waiting for ES restart...");
|
||||
afterKeystoreFix.countDown(); // SYNC: Cert update & ES restart
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
|
@ -123,30 +120,30 @@ public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
|||
return super.onNodeStopped(nodeName); // ASSUME: RestartCallback will do ES start next
|
||||
}
|
||||
});
|
||||
LOGGER.trace("Waiting for keystore fix...");
|
||||
timed(LOGGER, Level.DEBUG, "Awaited {}ms. Verifying the cluster...", () -> {
|
||||
logger.trace("Waiting for keystore fix...");
|
||||
timed(Level.DEBUG, "Awaited {}ms. Verifying the cluster...", () -> {
|
||||
try {
|
||||
afterKeystoreFix.await(); // SYNC: Verify cluster after cert update
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
timed(LOGGER, Level.TRACE, "Ensure cluster size consistency took {}ms.", this::ensureClusterSizeConsistency);
|
||||
timed(LOGGER, Level.TRACE, "Ensure fully connected cluster took {}ms.", this::ensureFullyConnectedCluster);
|
||||
timed(Level.TRACE, "Ensure cluster size consistency took {}ms.", this::ensureClusterSizeConsistency);
|
||||
timed(Level.TRACE, "Ensure fully connected cluster took {}ms.", this::ensureFullyConnectedCluster);
|
||||
}
|
||||
|
||||
private void waitUntilNodeStartupIsReadyToBegin(final CountDownLatch beforeKeystoreFix) {
|
||||
LOGGER.trace("Waiting for ES start to begin...");
|
||||
logger.trace("Waiting for ES start to begin...");
|
||||
beforeKeystoreFix.countDown(); // SYNC: Cert update & ES restart
|
||||
final long sleepMillis = randomLongBetween(1L, 2000L); // intended sleepMillis
|
||||
timed(LOGGER, Level.DEBUG, "Awaited {}ms. Sleeping " + sleepMillis + "ms before fixing...", () -> {
|
||||
timed(Level.DEBUG, "Awaited {}ms. Sleeping " + sleepMillis + "ms before fixing...", () -> {
|
||||
try {
|
||||
beforeKeystoreFix.await(); // SYNC: Cert update & ES restart
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
});
|
||||
timed(LOGGER, Level.DEBUG, "Slept {}ms, intended " + sleepMillis + "ms. Fixing can start now...", () -> {
|
||||
timed(Level.DEBUG, "Slept {}ms, intended " + sleepMillis + "ms. Fixing can start now...", () -> {
|
||||
try {
|
||||
Thread.sleep(sleepMillis); // Simulate cert update delay relative to ES start
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -156,9 +153,9 @@ public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
|||
}
|
||||
|
||||
private void waitUntilFixKeystoreIsReadyToBegin(final CountDownLatch beforeKeystoreFix) {
|
||||
LOGGER.trace("Waiting for keystore fix to begin...");
|
||||
logger.trace("Waiting for keystore fix to begin...");
|
||||
beforeKeystoreFix.countDown(); // SYNC: Cert update & ES restart
|
||||
timed(LOGGER, Level.DEBUG, "Awaited {}ms. Node can start now...", () -> {
|
||||
timed(Level.DEBUG, "Awaited {}ms. Node can start now...", () -> {
|
||||
try {
|
||||
beforeKeystoreFix.await(); // SYNC: Cert update & ES restart
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -167,8 +164,7 @@ public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
|||
});
|
||||
}
|
||||
|
||||
static void timed(final Logger LOGGER, final Level level, final String message, final Runnable runnable) {
|
||||
assert LOGGER != null;
|
||||
private void timed(final Level level, final String message, final Runnable runnable) {
|
||||
assert level != null;
|
||||
assert Strings.isEmpty(message) == false;
|
||||
assert message.contains("{}ms") : "Message must contain {}ms";
|
||||
|
@ -178,7 +174,7 @@ public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
|||
try {
|
||||
runnable.run();
|
||||
} finally {
|
||||
LOGGER.log(level, message, TimeValue.timeValueNanos(System.nanoTime() - startNanos).millisFrac());
|
||||
logger.log(level, message, TimeValue.timeValueNanos(System.nanoTime() - startNanos).millisFrac());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -191,16 +187,16 @@ public class SSLReloadDuringStartupIntegTests extends SecurityIntegTestCase {
|
|||
* @throws IOException Cannot create temp file, or copy source file to temp file, or non-atomic move temp file to target file.
|
||||
*/
|
||||
private void atomicCopyIfPossible(Path source, Path target) throws IOException {
|
||||
LOGGER.trace("Copying [{}] to [{}]", source, target);
|
||||
logger.trace("Copying [{}] to [{}]", source, target);
|
||||
Path tmp = createTempFile();
|
||||
LOGGER.trace("Created temporary file [{}]", tmp);
|
||||
logger.trace("Created temporary file [{}]", tmp);
|
||||
Files.copy(source, tmp, StandardCopyOption.REPLACE_EXISTING);
|
||||
try {
|
||||
Files.move(tmp, target, StandardCopyOption.REPLACE_EXISTING, StandardCopyOption.ATOMIC_MOVE);
|
||||
LOGGER.debug("Atomic move succeeded from [{}] to [{}]", tmp, target);
|
||||
logger.debug("Atomic move succeeded from [{}] to [{}]", tmp, target);
|
||||
} catch (AtomicMoveNotSupportedException e) {
|
||||
Files.move(tmp, target, StandardCopyOption.REPLACE_EXISTING);
|
||||
LOGGER.debug("Non-atomic move succeeded from [{}] to [{}]", tmp, target);
|
||||
logger.debug("Non-atomic move succeeded from [{}] to [{}]", tmp, target);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,7 +41,7 @@ import static org.elasticsearch.xpack.core.security.authc.support.DelegatedAutho
|
|||
public class DelegatedAuthorizationSupport {
|
||||
|
||||
private final RealmUserLookup lookup;
|
||||
private final Logger logger;
|
||||
private static final Logger logger = LogManager.getLogger(DelegatedAuthorizationSupport.class);
|
||||
private final XPackLicenseState licenseState;
|
||||
|
||||
/**
|
||||
|
@ -74,7 +74,6 @@ public class DelegatedAuthorizationSupport {
|
|||
final List<Realm> resolvedLookupRealms = resolveRealms(allRealms, lookupRealms);
|
||||
checkForRealmChains(resolvedLookupRealms, settings);
|
||||
this.lookup = new RealmUserLookup(resolvedLookupRealms, threadContext);
|
||||
this.logger = LogManager.getLogger(getClass());
|
||||
this.licenseState = licenseState;
|
||||
}
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ public class SecondaryAuthenticator {
|
|||
*/
|
||||
public static final String SECONDARY_AUTH_HEADER_NAME = "es-secondary-authorization";
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SecondaryAuthenticator.class);
|
||||
private static final Logger logger = LogManager.getLogger(SecondaryAuthenticator.class);
|
||||
private final SecurityContext securityContext;
|
||||
private final AuthenticationService authenticationService;
|
||||
private final AuditTrailService auditTrailService;
|
||||
|
|
|
@ -40,7 +40,7 @@ public final class SetSecurityUserProcessor extends AbstractProcessor {
|
|||
|
||||
public static final String TYPE = "set_security_user";
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SetSecurityUserProcessor.class);
|
||||
private static final Logger logger = LogManager.getLogger(SetSecurityUserProcessor.class);
|
||||
|
||||
private final SecurityContext securityContext;
|
||||
private final Settings settings;
|
||||
|
|
|
@ -19,7 +19,7 @@ import org.elasticsearch.xpack.security.rest.action.SecurityBaseRestHandler;
|
|||
|
||||
abstract class NativeRoleBaseRestHandler extends SecurityBaseRestHandler {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(NativeRoleBaseRestHandler.class);
|
||||
private static final Logger logger = LogManager.getLogger(NativeRoleBaseRestHandler.class);
|
||||
|
||||
NativeRoleBaseRestHandler(Settings settings, XPackLicenseState licenseState) {
|
||||
super(settings, licenseState);
|
||||
|
|
|
@ -19,7 +19,7 @@ import org.elasticsearch.xpack.security.rest.action.SecurityBaseRestHandler;
|
|||
|
||||
abstract class NativeUserBaseRestHandler extends SecurityBaseRestHandler {
|
||||
|
||||
private final Logger logger = LogManager.getLogger(NativeUserBaseRestHandler.class);
|
||||
private static final Logger logger = LogManager.getLogger(NativeUserBaseRestHandler.class);
|
||||
|
||||
NativeUserBaseRestHandler(Settings settings, XPackLicenseState licenseState) {
|
||||
super(settings, licenseState);
|
||||
|
|
|
@ -52,7 +52,7 @@ public class SecuritySystemIndices {
|
|||
public static final String SECURITY_PROFILE_ALIAS = ".security-profile";
|
||||
public static final Version VERSION_SECURITY_PROFILE_ORIGIN = Version.V_8_3_0;
|
||||
|
||||
private final Logger logger = LogManager.getLogger(SecuritySystemIndices.class);
|
||||
private static final Logger logger = LogManager.getLogger(SecuritySystemIndices.class);
|
||||
|
||||
private final SystemIndexDescriptor mainDescriptor;
|
||||
private final SystemIndexDescriptor tokenDescriptor;
|
||||
|
|
|
@ -37,7 +37,7 @@ import static org.elasticsearch.core.Strings.format;
|
|||
import static org.elasticsearch.indices.recovery.RecoverySettings.SEQ_NO_SNAPSHOT_RECOVERIES_SUPPORTED_VERSION;
|
||||
|
||||
public class SnapshotsRecoveryPlannerService implements RecoveryPlannerService {
|
||||
private final Logger logger = LogManager.getLogger(SnapshotsRecoveryPlannerService.class);
|
||||
private static final Logger logger = LogManager.getLogger(SnapshotsRecoveryPlannerService.class);
|
||||
|
||||
private final ShardSnapshotsService shardSnapshotsService;
|
||||
private final BooleanSupplier isLicenseActive;
|
||||
|
|
|
@ -6,8 +6,6 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.watch;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.action.DocWriteRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
|
@ -158,7 +156,6 @@ public class WatchTests extends ESTestCase {
|
|||
private TextTemplateEngine templateEngine;
|
||||
private HtmlSanitizer htmlSanitizer;
|
||||
private XPackLicenseState licenseState;
|
||||
private Logger logger;
|
||||
private Settings settings = Settings.EMPTY;
|
||||
private WatcherSearchTemplateService searchTemplateService;
|
||||
|
||||
|
@ -172,7 +169,6 @@ public class WatchTests extends ESTestCase {
|
|||
templateEngine = mock(TextTemplateEngine.class);
|
||||
htmlSanitizer = mock(HtmlSanitizer.class);
|
||||
licenseState = mock(XPackLicenseState.class);
|
||||
logger = LogManager.getLogger(WatchTests.class);
|
||||
searchTemplateService = mock(WatcherSearchTemplateService.class);
|
||||
}
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue