diff --git a/libs/core/src/main/java/org/elasticsearch/core/AbstractRefCounted.java b/libs/core/src/main/java/org/elasticsearch/core/AbstractRefCounted.java index cd296e5d5f31b..1ad8724dce2bf 100644 --- a/libs/core/src/main/java/org/elasticsearch/core/AbstractRefCounted.java +++ b/libs/core/src/main/java/org/elasticsearch/core/AbstractRefCounted.java @@ -11,17 +11,13 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * A basic RefCounted implementation that is initialized with a - * ref count of 1 and calls {@link #closeInternal()} once it reaches - * a 0 ref count + * A basic {@link RefCounted} implementation that is initialized with a ref count of 1 and calls {@link #closeInternal()} once it reaches + * a 0 ref count. */ public abstract class AbstractRefCounted implements RefCounted { - private final AtomicInteger refCount = new AtomicInteger(1); - private final String name; + public static final String ALREADY_CLOSED_MESSAGE = "already closed, can't increment ref count"; - public AbstractRefCounted(String name) { - this.name = name; - } + private final AtomicInteger refCount = new AtomicInteger(1); @Override public final void incRef() { @@ -63,14 +59,16 @@ public final boolean decRef() { } /** - * Called whenever the ref count is incremented or decremented. Can be implemented by implementations to a record of access to the - * instance for debugging purposes. + * Called whenever the ref count is incremented or decremented. Can be overridden to record access to the instance for debugging + * purposes. */ protected void touch() { } protected void alreadyClosed() { - throw new IllegalStateException(name + " is already closed can't increment refCount current count [" + refCount.get() + "]"); + final int currentRefCount = refCount.get(); + assert currentRefCount == 0 : currentRefCount; + throw new IllegalStateException(ALREADY_CLOSED_MESSAGE); } /** @@ -80,15 +78,21 @@ public int refCount() { return this.refCount.get(); } - - /** gets the name of this instance */ - public String getName() { - return name; - } - /** * Method that is invoked once the reference count reaches zero. * Implementations of this method must handle all exceptions and may not throw any exceptions. */ protected abstract void closeInternal(); + + /** + * Construct an {@link AbstractRefCounted} which runs the given {@link Runnable} when all references are released. + */ + public static AbstractRefCounted of(Runnable onClose) { + return new AbstractRefCounted() { + @Override + protected void closeInternal() { + onClose.run(); + } + }; + } } diff --git a/libs/core/src/test/java/org/elasticsearch/common/util/concurrent/RefCountedTests.java b/libs/core/src/test/java/org/elasticsearch/common/util/concurrent/RefCountedTests.java index e30e4eb4f3301..8579ebddee87c 100644 --- a/libs/core/src/test/java/org/elasticsearch/common/util/concurrent/RefCountedTests.java +++ b/libs/core/src/test/java/org/elasticsearch/common/util/concurrent/RefCountedTests.java @@ -11,7 +11,6 @@ import org.elasticsearch.test.ESTestCase; import org.hamcrest.Matchers; -import java.io.IOException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -20,7 +19,8 @@ import static org.hamcrest.Matchers.is; public class RefCountedTests extends ESTestCase { - public void testRefCount() throws IOException { + + public void testRefCount() { MyRefCounted counted = new MyRefCounted(); int incs = randomIntBetween(1, 100); @@ -56,12 +56,9 @@ public void testRefCount() throws IOException { counted.decRef(); assertFalse(counted.tryIncRef()); - try { - counted.incRef(); - fail(" expected exception"); - } catch (IllegalStateException ex) { - assertThat(ex.getMessage(), equalTo("test is already closed can't increment refCount current count [0]")); - } + assertThat( + expectThrows(IllegalStateException.class, counted::incRef).getMessage(), + equalTo(AbstractRefCounted.ALREADY_CLOSED_MESSAGE)); try { counted.ensureOpen(); @@ -77,29 +74,26 @@ public void testMultiThreaded() throws InterruptedException { final CountDownLatch latch = new CountDownLatch(1); final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); for (int i = 0; i < threads.length; i++) { - threads[i] = new Thread() { - @Override - public void run() { - try { - latch.await(); - for (int j = 0; j < 10000; j++) { - counted.incRef(); - try { - counted.ensureOpen(); - } finally { - counted.decRef(); - } + threads[i] = new Thread(() -> { + try { + latch.await(); + for (int j = 0; j < 10000; j++) { + counted.incRef(); + try { + counted.ensureOpen(); + } finally { + counted.decRef(); } - } catch (Exception e) { - exceptions.add(e); } + } catch (Exception e) { + exceptions.add(e); } - }; + }); threads[i].start(); } latch.countDown(); - for (int i = 0; i < threads.length; i++) { - threads[i].join(); + for (Thread thread : threads) { + thread.join(); } counted.decRef(); try { @@ -110,17 +104,12 @@ public void run() { } assertThat(counted.refCount(), is(0)); assertThat(exceptions, Matchers.emptyIterable()); - } - private final class MyRefCounted extends AbstractRefCounted { + private static final class MyRefCounted extends AbstractRefCounted { private final AtomicBoolean closed = new AtomicBoolean(false); - MyRefCounted() { - super("test"); - } - @Override protected void closeInternal() { this.closed.set(true); diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/Page.java b/libs/nio/src/main/java/org/elasticsearch/nio/Page.java index dafe68cfea744..50a1c0207eca5 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/Page.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/Page.java @@ -65,7 +65,6 @@ private static class RefCountedCloseable extends AbstractRefCounted { private final Releasable closeable; private RefCountedCloseable(Releasable closeable) { - super("byte array page"); this.closeable = closeable; } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/SharedGroupFactory.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/SharedGroupFactory.java index d9ccd0eb7b158..686d46e02a7e9 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/SharedGroupFactory.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/SharedGroupFactory.java @@ -85,11 +85,9 @@ private SharedGroup getGenericGroup() { private static class RefCountedGroup extends AbstractRefCounted { - public static final String NAME = "ref-counted-event-loop-group"; private final EventLoopGroup eventLoopGroup; private RefCountedGroup(EventLoopGroup eventLoopGroup) { - super(NAME); this.eventLoopGroup = eventLoopGroup; } diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/AmazonEc2Reference.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/AmazonEc2Reference.java index f6c65e326b79a..6000fd7b6731e 100644 --- a/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/AmazonEc2Reference.java +++ b/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/AmazonEc2Reference.java @@ -22,7 +22,6 @@ public class AmazonEc2Reference extends AbstractRefCounted implements Releasable private final AmazonEC2 client; AmazonEc2Reference(AmazonEC2 client) { - super("AWS_EC2_CLIENT"); this.client = client; } diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/AmazonS3Reference.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/AmazonS3Reference.java index 4aa085268d660..cf48dcf1d04cf 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/AmazonS3Reference.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/AmazonS3Reference.java @@ -24,7 +24,6 @@ public class AmazonS3Reference extends AbstractRefCounted implements Releasable private final AmazonS3 client; AmazonS3Reference(AmazonS3 client) { - super("AWS_S3_CLIENT"); this.client = client; } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioGroupFactory.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioGroupFactory.java index e51e1a832adec..500e840224b49 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioGroupFactory.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioGroupFactory.java @@ -86,11 +86,9 @@ private void onException(Exception exception) { private static class RefCountedNioGroup extends AbstractRefCounted implements NioGroup { - public static final String NAME = "ref-counted-nio-group"; private final NioSelectorGroup nioGroup; private RefCountedNioGroup(NioSelectorGroup nioGroup) { - super(NAME); this.nioGroup = nioGroup; } diff --git a/server/src/main/java/org/elasticsearch/common/bytes/ReleasableBytesReference.java b/server/src/main/java/org/elasticsearch/common/bytes/ReleasableBytesReference.java index 472b9080f33b8..db075f1d8e8f5 100644 --- a/server/src/main/java/org/elasticsearch/common/bytes/ReleasableBytesReference.java +++ b/server/src/main/java/org/elasticsearch/common/bytes/ReleasableBytesReference.java @@ -216,7 +216,6 @@ private static final class RefCountedReleasable extends AbstractRefCounted { private final Releasable releasable; RefCountedReleasable(Releasable releasable) { - super("bytes-reference"); this.releasable = releasable; } diff --git a/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java b/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java index 6d6895867f888..48c287951ed2f 100644 --- a/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java +++ b/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java @@ -178,7 +178,6 @@ private final class CachedItem extends AbstractRefCounted { private final CancellationChecks cancellationChecks = new CancellationChecks(); CachedItem(Key key) { - super("cached item"); this.key = key; incRef(); // start with a refcount of 2 so we're not closed while adding the first listener this.future.addListener(new ActionListener<>() { diff --git a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java index dc15232b184a5..bf5524b7dc8ef 100644 --- a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java @@ -81,12 +81,7 @@ public abstract class AbstractHttpServerTransport extends AbstractLifecycleCompo private final AtomicLong totalChannelsAccepted = new AtomicLong(); private final Set httpChannels = Collections.newSetFromMap(new ConcurrentHashMap<>()); private final PlainActionFuture allClientsClosedListener = PlainActionFuture.newFuture(); - private final RefCounted refCounted = new AbstractRefCounted("abstract-http-server-transport") { - @Override - protected void closeInternal() { - allClientsClosedListener.onResponse(null); - } - }; + private final RefCounted refCounted = AbstractRefCounted.of(() -> allClientsClosedListener.onResponse(null)); private final Set httpServerChannels = Collections.newSetFromMap(new ConcurrentHashMap<>()); private final HttpClientStatsTracker httpClientStatsTracker; diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 3bae95ebbeff5..7c7cdc8443355 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -136,13 +136,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref private final ShardLock shardLock; private final OnClose onClose; - private final AbstractRefCounted refCounter = new AbstractRefCounted("store") { - @Override - protected void closeInternal() { - // close us once we are done - Store.this.closeInternal(); - } - }; + private final AbstractRefCounted refCounter = AbstractRefCounted.of(this::closeInternal); // close us once we are done public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock) { this(shardId, indexSettings, directory, shardLock, OnClose.EMPTY); diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index bd602415568af..0fd9a23b95e50 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -308,24 +308,21 @@ public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, lon // avoid closing these resources while ongoing requests are still being processed, we use a // ref count which will only close them when both this service and all index services are // actually closed - indicesRefCount = new AbstractRefCounted("indices") { - @Override - protected void closeInternal() { - try { - IOUtils.close( - analysisRegistry, - indexingMemoryController, - indicesFieldDataCache, - cacheCleaner, - indicesRequestCache, - indicesQueryCache); - } catch (IOException e) { - throw new UncheckedIOException(e); - } finally { - closeLatch.countDown(); - } + indicesRefCount = AbstractRefCounted.of(() -> { + try { + IOUtils.close( + analysisRegistry, + indexingMemoryController, + indicesFieldDataCache, + cacheCleaner, + indicesRequestCache, + indicesQueryCache); + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + closeLatch.countDown(); } - }; + }); final String nodeName = Objects.requireNonNull(Node.NODE_NAME_SETTING.get(settings)); nodeWriteDanglingIndicesInfo = WRITE_DANGLING_INDICES_INFO_SETTING.get(settings); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java index 4331b6f88a073..cacd5955d8ff5 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java @@ -39,7 +39,6 @@ public class MultiFileWriter extends AbstractRefCounted implements Releasable { public MultiFileWriter(Store store, RecoveryState.Index indexState, String tempFilePrefix, Logger logger, Runnable ensureOpen) { - super("multi_file_writer"); this.store = store; this.indexState = indexState; this.tempFilePrefix = tempFilePrefix; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index fd962d229d94b..fce25d3ff2e25 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -95,7 +95,6 @@ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, SnapshotFilesProvider snapshotFilesProvider, PeerRecoveryTargetService.RecoveryListener listener) { - super("recovery_status"); this.cancellableThreads = new CancellableThreads(); this.recoveryId = idGenerator.incrementAndGet(); this.listener = listener; diff --git a/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java b/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java index 9e29a1553f66d..a835c89100067 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java @@ -66,12 +66,7 @@ public ReaderContext(ShardSearchContextId id, this.singleSession = singleSession; this.keepAlive = new AtomicLong(keepAliveInMillis); this.lastAccessTime = new AtomicLong(nowInMillis()); - this.refCounted = new AbstractRefCounted("reader_context") { - @Override - protected void closeInternal() { - doClose(); - } - }; + this.refCounted = AbstractRefCounted.of(this::doClose); } public void validate(TransportRequest request) { diff --git a/server/src/main/java/org/elasticsearch/transport/ClusterConnectionManager.java b/server/src/main/java/org/elasticsearch/transport/ClusterConnectionManager.java index 4b5da7dbbefde..77d321ae9ed3d 100644 --- a/server/src/main/java/org/elasticsearch/transport/ClusterConnectionManager.java +++ b/server/src/main/java/org/elasticsearch/transport/ClusterConnectionManager.java @@ -37,21 +37,8 @@ public class ClusterConnectionManager implements ConnectionManager { private final ConcurrentMap connectedNodes = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap> pendingConnections = ConcurrentCollections.newConcurrentMap(); - private final AbstractRefCounted connectingRefCounter = new AbstractRefCounted("connection manager") { - @Override - protected void closeInternal() { - Iterator> iterator = connectedNodes.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry next = iterator.next(); - try { - IOUtils.closeWhileHandlingException(next.getValue()); - } finally { - iterator.remove(); - } - } - closeLatch.countDown(); - } - }; + private final AbstractRefCounted connectingRefCounter = AbstractRefCounted.of(this::pendingConnectionsComplete); + private final Transport transport; private final ConnectionProfile defaultProfile; private final AtomicBoolean closing = new AtomicBoolean(false); @@ -237,6 +224,19 @@ private void internalClose(boolean waitForPendingConnections) { } } + private void pendingConnectionsComplete() { + final Iterator> iterator = connectedNodes.entrySet().iterator(); + while (iterator.hasNext()) { + final Map.Entry next = iterator.next(); + try { + IOUtils.closeWhileHandlingException(next.getValue()); + } finally { + iterator.remove(); + } + } + closeLatch.countDown(); + } + private void internalOpenConnection(DiscoveryNode node, ConnectionProfile connectionProfile, ActionListener listener) { transport.openConnection(node, connectionProfile, listener.map(connection -> { diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index b9a50865a28b3..506c6845991dc 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; @@ -344,7 +345,7 @@ public void onFailure(Exception e) { } catch (AlreadyClosedException ex) { throw ex; } catch (IllegalStateException ex) { - assertEquals("reader_context is already closed can't increment refCount current count [0]", ex.getMessage()); + assertEquals(AbstractRefCounted.ALREADY_CLOSED_MESSAGE, ex.getMessage()); } catch (SearchContextMissingException ex) { // that's fine } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java index 31ea88e8421f0..88f70d9849d57 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java @@ -265,7 +265,6 @@ private static final class LeakAwareRefCounted extends AbstractRefCounted { private final Releasable releasable; LeakAwareRefCounted(Releasable releasable) { - super("leak-aware-ref-counted"); this.releasable = releasable; leak = LeakTracker.INSTANCE.track(releasable); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index e8927f0d60a30..350584a46431c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -192,7 +192,6 @@ private static class RestoreSession extends AbstractRefCounted { private RestoreSession(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef, Scheduler.Cancellable timeoutTask) { - super("restore-session"); this.sessionUUID = sessionUUID; this.indexShard = indexShard; this.commitRef = commitRef; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/CacheFile.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/CacheFile.java index bc5f8a4825459..b70c55e7a2a54 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/CacheFile.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/CacheFile.java @@ -64,21 +64,7 @@ public interface ModificationListener { * for it. Once this instance has been evicted, all listeners notified and all {@link FileChannelReference} for it released, * it makes sure to delete the physical file backing this cache. */ - private final AbstractRefCounted refCounter = new AbstractRefCounted("CacheFile") { - @Override - protected void closeInternal() { - assert evicted.get(); - assert assertNoPendingListeners(); - try { - Files.deleteIfExists(file); - } catch (IOException e) { - // nothing to do but log failures here since closeInternal could be called from anywhere and must not throw - logger.warn(() -> new ParameterizedMessage("Failed to delete [{}]", file), e); - } finally { - listener.onCacheFileDelete(CacheFile.this); - } - } - }; + private final AbstractRefCounted refCounter = AbstractRefCounted.of(this::deleteFile); private final SparseFileTracker tracker; private final CacheKey cacheKey; @@ -115,7 +101,6 @@ private final class FileChannelReference extends AbstractRefCounted { private final FileChannel fileChannel; FileChannelReference() throws IOException { - super("FileChannel[" + file + "]"); this.fileChannel = FileChannel.open(file, OPEN_OPTIONS); refCounter.incRef(); } @@ -527,4 +512,17 @@ public SortedSet fsync() throws IOException { } return Collections.emptySortedSet(); } + + private void deleteFile() { + assert evicted.get(); + assert assertNoPendingListeners(); + try { + Files.deleteIfExists(file); + } catch (IOException e) { + // nothing to do but log failures here since closeInternal could be called from anywhere and must not throw + logger.warn(() -> new ParameterizedMessage("Failed to delete [{}]", file), e); + } finally { + listener.onCacheFileDelete(CacheFile.this); + } + } } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java index e37fb2b5b84a6..3ca5a09e0b0c6 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/FrozenCacheService.java @@ -662,7 +662,6 @@ class CacheFileRegion extends AbstractRefCounted { volatile int sharedBytesPos = -1; CacheFileRegion(RegionKey regionKey, long regionSize) { - super("CacheFileRegion"); this.regionKey = regionKey; assert regionSize > 0L; tracker = new SparseFileTracker("file", regionSize); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/SharedBytes.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/SharedBytes.java index 38c1e1c7b9b82..810247788540a 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/SharedBytes.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/shared/SharedBytes.java @@ -54,7 +54,6 @@ public class SharedBytes extends AbstractRefCounted { SharedBytes(int numRegions, long regionSize, NodeEnvironment environment, IntConsumer writeBytes, IntConsumer readBytes) throws IOException { - super("shared-bytes"); this.numRegions = numRegions; this.regionSize = regionSize; final long fileSize = numRegions * regionSize; @@ -158,7 +157,6 @@ public final class IO extends AbstractRefCounted { private final long pageStart; private IO(final int sharedBytesPos) { - super("shared-bytes-io"); this.sharedBytesPos = sharedBytesPos; pageStart = getPhysicalOffset(sharedBytesPos); }