Logging: Drop Settings from some logging ctors (#33332)
Drops `Settings` from some logging ctors now that they are no longer needed. This should allow us to stop passing `Settings` around to quite as many places.
This commit is contained in:
parent
ea4eef8641
commit
f8b7a4dbc8
|
@ -49,16 +49,17 @@ public class Loggers {
|
|||
Setting.prefixKeySetting("logger.", (key) -> new Setting<>(key, Level.INFO.name(), Level::valueOf, Setting.Property.Dynamic,
|
||||
Setting.Property.NodeScope));
|
||||
|
||||
public static Logger getLogger(Class<?> clazz, Settings settings, ShardId shardId, String... prefixes) {
|
||||
return getLogger(clazz, settings, shardId.getIndex(), asArrayList(Integer.toString(shardId.id()), prefixes).toArray(new String[0]));
|
||||
public static Logger getLogger(Class<?> clazz, ShardId shardId, String... prefixes) {
|
||||
return getLogger(clazz, Settings.EMPTY,
|
||||
shardId.getIndex(), asArrayList(Integer.toString(shardId.id()), prefixes).toArray(new String[0]));
|
||||
}
|
||||
|
||||
/**
|
||||
* Just like {@link #getLogger(Class, org.elasticsearch.common.settings.Settings, ShardId, String...)} but String loggerName instead of
|
||||
* Just like {@link #getLogger(Class, ShardId, String...)} but String loggerName instead of
|
||||
* Class.
|
||||
*/
|
||||
public static Logger getLogger(String loggerName, Settings settings, ShardId shardId, String... prefixes) {
|
||||
return getLogger(loggerName, settings,
|
||||
public static Logger getLogger(String loggerName, ShardId shardId, String... prefixes) {
|
||||
return getLogger(loggerName, Settings.EMPTY,
|
||||
asArrayList(shardId.getIndexName(), Integer.toString(shardId.id()), prefixes).toArray(new String[0]));
|
||||
}
|
||||
|
||||
|
|
|
@ -71,7 +71,7 @@ class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler {
|
|||
this.config = indexSettings.getMergeSchedulerConfig();
|
||||
this.shardId = shardId;
|
||||
this.indexSettings = indexSettings.getSettings();
|
||||
this.logger = Loggers.getLogger(getClass(), this.indexSettings, shardId);
|
||||
this.logger = Loggers.getLogger(getClass(), shardId);
|
||||
refreshConfig();
|
||||
}
|
||||
|
||||
|
|
|
@ -134,7 +134,7 @@ public abstract class Engine implements Closeable {
|
|||
this.allocationId = engineConfig.getAllocationId();
|
||||
this.store = engineConfig.getStore();
|
||||
this.logger = Loggers.getLogger(Engine.class, // we use the engine class directly here to make sure all subclasses have the same logger name
|
||||
engineConfig.getIndexSettings().getSettings(), engineConfig.getShardId());
|
||||
engineConfig.getShardId());
|
||||
this.eventListener = engineConfig.getEventListener();
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ public abstract class AbstractIndexShardComponent implements IndexShardComponent
|
|||
protected AbstractIndexShardComponent(ShardId shardId, IndexSettings indexSettings) {
|
||||
this.shardId = shardId;
|
||||
this.indexSettings = indexSettings;
|
||||
this.logger = Loggers.getLogger(getClass(), this.indexSettings.getSettings(), shardId);
|
||||
this.logger = Loggers.getLogger(getClass(), shardId);
|
||||
this.deprecationLogger = new DeprecationLogger(logger);
|
||||
}
|
||||
|
||||
|
|
|
@ -165,7 +165,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
|
|||
final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING);
|
||||
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
|
||||
ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(dir, refreshInterval);
|
||||
this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", settings, shardId));
|
||||
this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId));
|
||||
this.shardLock = shardLock;
|
||||
this.onClose = onClose;
|
||||
|
||||
|
|
|
@ -173,10 +173,9 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
|||
final RemoteRecoveryTargetHandler recoveryTarget =
|
||||
new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService,
|
||||
request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime));
|
||||
handler = new RecoverySourceHandler(shard, recoveryTarget, request, recoverySettings.getChunkSize().bytesAsInt(), settings);
|
||||
handler = new RecoverySourceHandler(shard, recoveryTarget, request, recoverySettings.getChunkSize().bytesAsInt());
|
||||
return handler;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.elasticsearch.common.bytes.BytesArray;
|
|||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.util.CancellableThreads;
|
||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||
|
@ -116,13 +115,12 @@ public class RecoverySourceHandler {
|
|||
|
||||
public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget,
|
||||
final StartRecoveryRequest request,
|
||||
final int fileChunkSizeInBytes,
|
||||
final Settings nodeSettings) {
|
||||
final int fileChunkSizeInBytes) {
|
||||
this.shard = shard;
|
||||
this.recoveryTarget = recoveryTarget;
|
||||
this.request = request;
|
||||
this.shardId = this.request.shardId().id();
|
||||
this.logger = Loggers.getLogger(getClass(), nodeSettings, request.shardId(), "recover to " + request.targetNode().getName());
|
||||
this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName());
|
||||
this.chunkSizeInBytes = fileChunkSizeInBytes;
|
||||
this.response = new RecoveryResponse();
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
|||
this.cancellableThreads = new CancellableThreads();
|
||||
this.recoveryId = idGenerator.incrementAndGet();
|
||||
this.listener = listener;
|
||||
this.logger = Loggers.getLogger(getClass(), indexShard.indexSettings().getSettings(), indexShard.shardId());
|
||||
this.logger = Loggers.getLogger(getClass(), indexShard.shardId());
|
||||
this.indexShard = indexShard;
|
||||
this.sourceNode = sourceNode;
|
||||
this.shardId = indexShard.shardId();
|
||||
|
|
|
@ -108,7 +108,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
final StartRecoveryRequest request = getStartRecoveryRequest();
|
||||
Store store = newStore(createTempDir());
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY);
|
||||
recoverySettings.getChunkSize().bytesAsInt());
|
||||
Directory dir = store.directory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
|
||||
int numDocs = randomIntBetween(10, 100);
|
||||
|
@ -174,7 +174,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
when(shard.state()).thenReturn(IndexShardState.STARTED);
|
||||
final RecoveryTargetHandler recoveryTarget = mock(RecoveryTargetHandler.class);
|
||||
final RecoverySourceHandler handler =
|
||||
new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes, Settings.EMPTY);
|
||||
new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes);
|
||||
final List<Translog.Operation> operations = new ArrayList<>();
|
||||
final int initialNumberOfDocs = randomIntBetween(16, 64);
|
||||
for (int i = 0; i < initialNumberOfDocs; i++) {
|
||||
|
@ -281,7 +281,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
Store store = newStore(tempDir, false);
|
||||
AtomicBoolean failedEngine = new AtomicBoolean(false);
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) {
|
||||
recoverySettings.getChunkSize().bytesAsInt()) {
|
||||
@Override
|
||||
protected void failEngine(IOException cause) {
|
||||
assertFalse(failedEngine.get());
|
||||
|
@ -340,7 +340,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
Store store = newStore(tempDir, false);
|
||||
AtomicBoolean failedEngine = new AtomicBoolean(false);
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) {
|
||||
recoverySettings.getChunkSize().bytesAsInt()) {
|
||||
@Override
|
||||
protected void failEngine(IOException cause) {
|
||||
assertFalse(failedEngine.get());
|
||||
|
@ -405,11 +405,10 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
||||
final AtomicBoolean phase2Called = new AtomicBoolean();
|
||||
final RecoverySourceHandler handler = new RecoverySourceHandler(
|
||||
shard,
|
||||
mock(RecoveryTargetHandler.class),
|
||||
request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(),
|
||||
Settings.EMPTY) {
|
||||
shard,
|
||||
mock(RecoveryTargetHandler.class),
|
||||
request,
|
||||
recoverySettings.getChunkSize().bytesAsInt()) {
|
||||
|
||||
@Override
|
||||
public void phase1(final IndexCommit snapshot, final Supplier<Integer> translogOps) {
|
||||
|
|
|
@ -70,7 +70,6 @@ import org.elasticsearch.indices.recovery.RecoverySourceHandler;
|
|||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||
import org.elasticsearch.indices.recovery.StartRecoveryRequest;
|
||||
import org.elasticsearch.node.Node;
|
||||
import org.elasticsearch.repositories.IndexId;
|
||||
import org.elasticsearch.repositories.Repository;
|
||||
import org.elasticsearch.snapshots.Snapshot;
|
||||
|
@ -572,11 +571,10 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
final StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), targetAllocationId,
|
||||
pNode, rNode, snapshot, replica.routingEntry().primary(), 0, startingSeqNo);
|
||||
final RecoverySourceHandler recovery = new RecoverySourceHandler(
|
||||
primary,
|
||||
recoveryTarget,
|
||||
request,
|
||||
(int) ByteSizeUnit.MB.toBytes(1),
|
||||
Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), pNode.getName()).build());
|
||||
primary,
|
||||
recoveryTarget,
|
||||
request,
|
||||
(int) ByteSizeUnit.MB.toBytes(1));
|
||||
primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null,
|
||||
currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet());
|
||||
recovery.recoverToTarget();
|
||||
|
|
|
@ -101,7 +101,7 @@ public class MockFSIndexStore extends IndexStore {
|
|||
if (indexShard != null) {
|
||||
Boolean remove = shardSet.remove(indexShard);
|
||||
if (remove == Boolean.TRUE) {
|
||||
Logger logger = Loggers.getLogger(getClass(), indexShard.indexSettings().getSettings(), indexShard.shardId());
|
||||
Logger logger = Loggers.getLogger(getClass(), indexShard.shardId());
|
||||
MockFSDirectoryService.checkIndex(logger, indexShard.store(), indexShard.shardId());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue