Explorar o código

Simplify CacheFile acquire and release Methods (#64139)

Follow up to #63911 making these methods either work out
or throw an already closed exception and resulting possible simplifications.
Armin Braun %!s(int64=5) %!d(string=hai) anos
pai
achega
6ed9691ff9

+ 10 - 5
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java

@@ -128,7 +128,7 @@ public class CacheFile {
         return reference == null ? null : reference.fileChannel;
     }
 
-    public boolean acquire(final EvictionListener listener) throws IOException {
+    public void acquire(final EvictionListener listener) throws IOException {
         assert listener != null;
 
         ensureOpen();
@@ -150,12 +150,14 @@ public class CacheFile {
                     refCounter.decRef();
                 }
             }
+        } else {
+            assert evicted.get();
+            throwAlreadyEvicted();
         }
         assert invariant();
-        return success;
     }
 
-    public boolean release(final EvictionListener listener) {
+    public void release(final EvictionListener listener) {
         assert listener != null;
 
         boolean success = false;
@@ -179,7 +181,6 @@ public class CacheFile {
             }
         }
         assert invariant();
-        return success;
     }
 
     private boolean assertNoPendingListeners() {
@@ -244,10 +245,14 @@ public class CacheFile {
 
     private void ensureOpen() {
         if (evicted.get()) {
-            throw new AlreadyClosedException("Cache file is evicted");
+            throwAlreadyEvicted();
         }
     }
 
+    private static void throwAlreadyEvicted() {
+        throw new AlreadyClosedException("Cache file is evicted");
+    }
+
     @FunctionalInterface
     interface RangeAvailableHandler {
         int onRangeAvailable(FileChannel channel) throws IOException;

+ 6 - 18
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java

@@ -18,7 +18,6 @@ import org.apache.lucene.util.BytesRefIterator;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.blobstore.cache.BlobStoreCacheService;
 import org.elasticsearch.blobstore.cache.CachedBlob;
-import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.SuppressForbidden;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.collect.Tuple;
@@ -135,14 +134,6 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
         return Tuple.tuple(start, end);
     }
 
-    private CacheFile getCacheFileSafe() throws Exception {
-        final CacheFile cacheFile = cacheFileReference.get();
-        if (cacheFile == null) {
-            throw new AlreadyClosedException("Failed to acquire a non-evicted cache file");
-        }
-        return cacheFile;
-    }
-
     @Override
     protected void readInternal(ByteBuffer b) throws IOException {
         ensureContext(ctx -> ctx != CACHE_WARMING_CONTEXT);
@@ -163,7 +154,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
         logger.trace("readInternal: read [{}-{}] ([{}] bytes) from [{}]", position, position + length, length, this);
 
         try {
-            final CacheFile cacheFile = getCacheFileSafe();
+            final CacheFile cacheFile = cacheFileReference.get();
 
             // Can we serve the read directly from disk? If so, do so and don't worry about anything else.
 
@@ -447,7 +438,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
         assert assertRangeIsAlignedWithPart(partRange);
 
         try {
-            final CacheFile cacheFile = getCacheFileSafe();
+            final CacheFile cacheFile = cacheFileReference.get();
 
             final Tuple<Long, Long> range = cacheFile.getAbsentRangeWithin(partRange.v1(), partRange.v2());
             if (range == null) {
@@ -775,7 +766,6 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
             this.directory = directory;
         }
 
-        @Nullable
         CacheFile get() throws Exception {
             CacheFile currentCacheFile = cacheFile.get();
             if (currentCacheFile != null) {
@@ -788,13 +778,11 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
                 if (currentCacheFile != null) {
                     return currentCacheFile;
                 }
-                if (newCacheFile.acquire(this)) {
-                    final CacheFile previousCacheFile = cacheFile.getAndSet(newCacheFile);
-                    assert previousCacheFile == null;
-                    return newCacheFile;
-                }
+                newCacheFile.acquire(this);
+                final CacheFile previousCacheFile = cacheFile.getAndSet(newCacheFile);
+                assert previousCacheFile == null;
+                return newCacheFile;
             }
-            return null;
         }
 
         @Override

+ 8 - 14
x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CacheFileTests.java

@@ -39,21 +39,18 @@ public class CacheFileTests extends ESTestCase {
         assertThat("Cache file is not acquired: file does not exist", Files.exists(file), is(false));
 
         final TestEvictionListener listener = new TestEvictionListener();
-        boolean acquired = cacheFile.acquire(listener);
-        assertThat("Cache file has been acquired", acquired, is(true));
+        cacheFile.acquire(listener);
         assertThat("Cache file has been acquired: file should exists", Files.exists(file), is(true));
         assertThat("Cache file has been acquired: channel should exists", cacheFile.getChannel(), notNullValue());
         assertThat("Cache file has been acquired: channel is open", cacheFile.getChannel().isOpen(), is(true));
         assertThat("Cache file has been acquired: eviction listener is not executed", listener.isCalled(), is(false));
 
-        boolean released = cacheFile.release(listener);
-        assertThat("Cache file has been released", released, is(true));
+        cacheFile.release(listener);
         assertThat("Cache file has been released: eviction listener is not executed", listener.isCalled(), is(false));
         assertThat("Cache file has been released: channel does not exist", cacheFile.getChannel(), nullValue());
         assertThat("Cache file is not evicted: file still exists after release", Files.exists(file), is(true));
 
-        acquired = cacheFile.acquire(listener);
-        assertThat("Cache file is acquired again", acquired, is(true));
+        cacheFile.acquire(listener);
 
         FileChannel fileChannel = cacheFile.getChannel();
         assertThat("Channel should exists", fileChannel, notNullValue());
@@ -68,8 +65,7 @@ public class CacheFileTests extends ESTestCase {
         assertThat("Cache file is evicted but not fully released: channel is open", cacheFile.getChannel().isOpen(), is(true));
         assertThat("Channel didn't change after eviction", cacheFile.getChannel(), sameInstance(fileChannel));
 
-        released = cacheFile.release(listener);
-        assertTrue("Cache file is fully released", released);
+        cacheFile.release(listener);
         assertThat("Cache file evicted and fully released: channel does not exist", cacheFile.getChannel(), nullValue());
         assertThat("Cache file has been deleted", Files.exists(file), is(false));
     }
@@ -83,14 +79,12 @@ public class CacheFileTests extends ESTestCase {
 
         if (randomBoolean()) {
             final TestEvictionListener listener = new TestEvictionListener();
-            boolean acquired = cacheFile.acquire(listener);
-            assertTrue("Cache file is acquired", acquired);
+            cacheFile.acquire(listener);
 
             assertThat(cacheFile.getChannel(), notNullValue());
             assertThat(Files.exists(file), is(true));
 
-            boolean released = cacheFile.release(listener);
-            assertTrue("Cache file is released", released);
+            cacheFile.release(listener);
         }
 
         cacheFile.startEviction();
@@ -105,7 +99,7 @@ public class CacheFileTests extends ESTestCase {
         final List<TestEvictionListener> acquiredListeners = new ArrayList<>();
         for (int i = 0; i < randomIntBetween(1, 20); i++) {
             TestEvictionListener listener = new TestEvictionListener();
-            assertTrue(cacheFile.acquire(listener));
+            cacheFile.acquire(listener);
             assertThat(cacheFile.getChannel(), notNullValue());
             acquiredListeners.add(listener);
         }
@@ -135,7 +129,7 @@ public class CacheFileTests extends ESTestCase {
         final CacheFile cacheFile = new CacheFile("test", randomLongBetween(1, 100), file);
 
         final TestEvictionListener evictionListener = new TestEvictionListener();
-        assertTrue(cacheFile.acquire(evictionListener));
+        cacheFile.acquire(evictionListener);
         final long length = cacheFile.getLength();
         final DeterministicTaskQueue deterministicTaskQueue = new DeterministicTaskQueue(
             builder().put(NODE_NAME_SETTING.getKey(), getTestName()).build(),