浏览代码

Pass Directory instead of DirectoryService to Store (#33466)

Instead of passing DirectoryService which causes yet another dependency
on Store we can just pass in a Directory since we will just call
`DirectoryService#newDirectory()` on it anyway.
Simon Willnauer 7 年之前
父节点
当前提交
c12d232215

+ 4 - 1
server/src/main/java/org/elasticsearch/index/IndexService.java

@@ -64,6 +64,7 @@ import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardNotFoundException;
 import org.elasticsearch.index.shard.ShardNotFoundException;
 import org.elasticsearch.index.shard.ShardPath;
 import org.elasticsearch.index.shard.ShardPath;
 import org.elasticsearch.index.similarity.SimilarityService;
 import org.elasticsearch.index.similarity.SimilarityService;
+import org.elasticsearch.index.store.DirectoryService;
 import org.elasticsearch.index.store.IndexStore;
 import org.elasticsearch.index.store.IndexStore;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.Translog;
@@ -377,7 +378,9 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
                     warmer.warm(searcher, shard, IndexService.this.indexSettings);
                     warmer.warm(searcher, shard, IndexService.this.indexSettings);
                 }
                 }
             };
             };
-            store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock,
+            // TODO we can remove either IndexStore or DirectoryService. All we need is a simple Supplier<Directory>
+            DirectoryService directoryService = indexStore.newDirectoryService(path);
+            store = new Store(shardId, this.indexSettings, directoryService.newDirectory(), lock,
                     new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)));
                     new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)));
             indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier,
             indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier,
                 indexCache, mapperService, similarityService, engineFactory,
                 indexCache, mapperService, similarityService, engineFactory,

+ 5 - 8
server/src/main/java/org/elasticsearch/index/store/Store.java

@@ -64,7 +64,6 @@ import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
 import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
 import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
 import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
 import org.elasticsearch.common.util.concurrent.RefCounted;
 import org.elasticsearch.common.util.concurrent.RefCounted;
@@ -153,18 +152,16 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
         }
         }
     };
     };
 
 
-    public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock) throws IOException {
-        this(shardId, indexSettings, directoryService, shardLock, OnClose.EMPTY);
+    public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock) {
+        this(shardId, indexSettings, directory, shardLock, OnClose.EMPTY);
     }
     }
 
 
-    public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock,
-                 OnClose onClose) throws IOException {
+    public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock,
+                 OnClose onClose) {
         super(shardId, indexSettings);
         super(shardId, indexSettings);
-        final Settings settings = indexSettings.getSettings();
-        Directory dir = directoryService.newDirectory();
         final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING);
         final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING);
         logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
         logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
-        ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(dir, refreshInterval);
+        ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(directory, refreshInterval);
         this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId));
         this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId));
         this.shardLock = shardLock;
         this.shardLock = shardLock;
         this.onClose = onClose;
         this.onClose = onClose;

+ 1 - 8
server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java

@@ -50,7 +50,6 @@ import org.elasticsearch.index.mapper.ParseContext.Document;
 import org.elasticsearch.index.mapper.ParsedDocument;
 import org.elasticsearch.index.mapper.ParsedDocument;
 import org.elasticsearch.index.mapper.SeqNoFieldMapper;
 import org.elasticsearch.index.mapper.SeqNoFieldMapper;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.seqno.SequenceNumbers;
-import org.elasticsearch.index.store.DirectoryService;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.TranslogConfig;
 import org.elasticsearch.index.translog.TranslogConfig;
@@ -106,13 +105,7 @@ public class RefreshListenersTests extends ESTestCase {
         ShardId shardId = new ShardId(new Index("index", "_na_"), 1);
         ShardId shardId = new ShardId(new Index("index", "_na_"), 1);
         String allocationId = UUIDs.randomBase64UUID(random());
         String allocationId = UUIDs.randomBase64UUID(random());
         Directory directory = newDirectory();
         Directory directory = newDirectory();
-        DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
-            @Override
-            public Directory newDirectory() throws IOException {
-                return directory;
-            }
-        };
-        store = new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
+        store = new Store(shardId, indexSettings, directory, new DummyShardLock(shardId));
         IndexWriterConfig iwc = newIndexWriterConfig();
         IndexWriterConfig iwc = newIndexWriterConfig();
         TranslogConfig translogConfig = new TranslogConfig(shardId, createTempDir("translog"), indexSettings,
         TranslogConfig translogConfig = new TranslogConfig(shardId, createTempDir("translog"), indexSettings,
             BigArrays.NON_RECYCLING_INSTANCE);
             BigArrays.NON_RECYCLING_INSTANCE);

+ 20 - 75
server/src/test/java/org/elasticsearch/index/store/StoreTests.java

@@ -104,12 +104,10 @@ public class StoreTests extends ESTestCase {
     private static final Version MIN_SUPPORTED_LUCENE_VERSION = org.elasticsearch.Version.CURRENT
     private static final Version MIN_SUPPORTED_LUCENE_VERSION = org.elasticsearch.Version.CURRENT
         .minimumIndexCompatibilityVersion().luceneVersion;
         .minimumIndexCompatibilityVersion().luceneVersion;
 
 
-    public void testRefCount() throws IOException {
+    public void testRefCount() {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
         IndexSettings indexSettings = INDEX_SETTINGS;
         IndexSettings indexSettings = INDEX_SETTINGS;
-
-        Store store = new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, indexSettings, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
         int incs = randomIntBetween(1, 100);
         int incs = randomIntBetween(1, 100);
         for (int i = 0; i < incs; i++) {
         for (int i = 0; i < incs; i++) {
             if (randomBoolean()) {
             if (randomBoolean()) {
@@ -296,8 +294,7 @@ public class StoreTests extends ESTestCase {
 
 
     public void testNewChecksums() throws IOException {
     public void testNewChecksums() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
         // set default codec - all segments need checksums
         // set default codec - all segments need checksums
         IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec()));
         IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec()));
         int docs = 1 + random().nextInt(100);
         int docs = 1 + random().nextInt(100);
@@ -347,7 +344,7 @@ public class StoreTests extends ESTestCase {
         assertConsistent(store, metadata);
         assertConsistent(store, metadata);
 
 
         TestUtil.checkIndex(store.directory());
         TestUtil.checkIndex(store.directory());
-        assertDeleteContent(store, directoryService);
+        assertDeleteContent(store, store.directory());
         IOUtils.close(store);
         IOUtils.close(store);
     }
     }
 
 
@@ -455,32 +452,11 @@ public class StoreTests extends ESTestCase {
 
 
     }
     }
 
 
-    public void assertDeleteContent(Store store, DirectoryService service) throws IOException {
+    public void assertDeleteContent(Store store, Directory dir) throws IOException {
         deleteContent(store.directory());
         deleteContent(store.directory());
         assertThat(Arrays.toString(store.directory().listAll()), store.directory().listAll().length, equalTo(0));
         assertThat(Arrays.toString(store.directory().listAll()), store.directory().listAll().length, equalTo(0));
         assertThat(store.stats().sizeInBytes(), equalTo(0L));
         assertThat(store.stats().sizeInBytes(), equalTo(0L));
-        assertThat(service.newDirectory().listAll().length, equalTo(0));
-    }
-
-    private static final class LuceneManagedDirectoryService extends DirectoryService {
-        private final Directory dir;
-        private final Random random;
-
-        LuceneManagedDirectoryService(Random random) {
-            this(random, true);
-        }
-
-        LuceneManagedDirectoryService(Random random, boolean preventDoubleWrite) {
-            super(new ShardId(INDEX_SETTINGS.getIndex(), 1), INDEX_SETTINGS);
-            dir = StoreTests.newDirectory(random);
-            this.random = random;
-        }
-
-        @Override
-        public Directory newDirectory() throws IOException {
-            return dir;
-        }
-
+        assertThat(dir.listAll().length, equalTo(0));
     }
     }
 
 
     public static void assertConsistent(Store store, Store.MetadataSnapshot metadata) throws IOException {
     public static void assertConsistent(Store store, Store.MetadataSnapshot metadata) throws IOException {
@@ -511,8 +487,7 @@ public class StoreTests extends ESTestCase {
             iwc.setMergePolicy(NoMergePolicy.INSTANCE);
             iwc.setMergePolicy(NoMergePolicy.INSTANCE);
             iwc.setUseCompoundFile(random.nextBoolean());
             iwc.setUseCompoundFile(random.nextBoolean());
             final ShardId shardId = new ShardId("index", "_na_", 1);
             final ShardId shardId = new ShardId("index", "_na_", 1);
-            DirectoryService directoryService = new LuceneManagedDirectoryService(random);
-            Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+            Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
             IndexWriter writer = new IndexWriter(store.directory(), iwc);
             IndexWriter writer = new IndexWriter(store.directory(), iwc);
             final boolean lotsOfSegments = rarely(random);
             final boolean lotsOfSegments = rarely(random);
             for (Document d : docs) {
             for (Document d : docs) {
@@ -526,7 +501,7 @@ public class StoreTests extends ESTestCase {
             writer.commit();
             writer.commit();
             writer.close();
             writer.close();
             first = store.getMetadata(null);
             first = store.getMetadata(null);
-            assertDeleteContent(store, directoryService);
+            assertDeleteContent(store, store.directory());
             store.close();
             store.close();
         }
         }
         long time = new Date().getTime();
         long time = new Date().getTime();
@@ -541,8 +516,7 @@ public class StoreTests extends ESTestCase {
             iwc.setMergePolicy(NoMergePolicy.INSTANCE);
             iwc.setMergePolicy(NoMergePolicy.INSTANCE);
             iwc.setUseCompoundFile(random.nextBoolean());
             iwc.setUseCompoundFile(random.nextBoolean());
             final ShardId shardId = new ShardId("index", "_na_", 1);
             final ShardId shardId = new ShardId("index", "_na_", 1);
-            DirectoryService directoryService = new LuceneManagedDirectoryService(random);
-            store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+            store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
             IndexWriter writer = new IndexWriter(store.directory(), iwc);
             IndexWriter writer = new IndexWriter(store.directory(), iwc);
             final boolean lotsOfSegments = rarely(random);
             final boolean lotsOfSegments = rarely(random);
             for (Document d : docs) {
             for (Document d : docs) {
@@ -639,8 +613,7 @@ public class StoreTests extends ESTestCase {
 
 
     public void testCleanupFromSnapshot() throws IOException {
     public void testCleanupFromSnapshot() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
         // this time random codec....
         // this time random codec....
         IndexWriterConfig indexWriterConfig = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec());
         IndexWriterConfig indexWriterConfig = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec());
         // we keep all commits and that allows us clean based on multiple snapshots
         // we keep all commits and that allows us clean based on multiple snapshots
@@ -727,11 +700,10 @@ public class StoreTests extends ESTestCase {
 
 
     public void testOnCloseCallback() throws IOException {
     public void testOnCloseCallback() throws IOException {
         final ShardId shardId = new ShardId(new Index(randomRealisticUnicodeOfCodepointLengthBetween(1, 10), "_na_"), randomIntBetween(0, 100));
         final ShardId shardId = new ShardId(new Index(randomRealisticUnicodeOfCodepointLengthBetween(1, 10), "_na_"), randomIntBetween(0, 100));
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
         final AtomicInteger count = new AtomicInteger(0);
         final AtomicInteger count = new AtomicInteger(0);
         final ShardLock lock = new DummyShardLock(shardId);
         final ShardLock lock = new DummyShardLock(shardId);
 
 
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, lock, theLock -> {
+        Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), lock, theLock -> {
             assertEquals(shardId, theLock.getShardId());
             assertEquals(shardId, theLock.getShardId());
             assertEquals(lock, theLock);
             assertEquals(lock, theLock);
             count.incrementAndGet();
             count.incrementAndGet();
@@ -748,11 +720,10 @@ public class StoreTests extends ESTestCase {
 
 
     public void testStoreStats() throws IOException {
     public void testStoreStats() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
         Settings settings = Settings.builder()
         Settings settings = Settings.builder()
                 .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
                 .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
                 .put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMinutes(0)).build();
                 .put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMinutes(0)).build();
-        Store store = new Store(shardId, IndexSettingsModule.newIndexSettings("index", settings), directoryService,
+        Store store = new Store(shardId, IndexSettingsModule.newIndexSettings("index", settings), StoreTests.newDirectory(random()),
             new DummyShardLock(shardId));
             new DummyShardLock(shardId));
         long initialStoreSize = 0;
         long initialStoreSize = 0;
         for (String extraFiles : store.directory().listAll()) {
         for (String extraFiles : store.directory().listAll()) {
@@ -843,8 +814,7 @@ public class StoreTests extends ESTestCase {
 
 
     public void testUserDataRead() throws IOException {
     public void testUserDataRead() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
         IndexWriterConfig config = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec());
         IndexWriterConfig config = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec());
         SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
         SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
         config.setIndexDeletionPolicy(deletionPolicy);
         config.setIndexDeletionPolicy(deletionPolicy);
@@ -867,7 +837,7 @@ public class StoreTests extends ESTestCase {
         assertThat(metadata.getCommitUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId));
         assertThat(metadata.getCommitUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId));
         assertThat(metadata.getCommitUserData().get(Translog.TRANSLOG_GENERATION_KEY), equalTo(translogId));
         assertThat(metadata.getCommitUserData().get(Translog.TRANSLOG_GENERATION_KEY), equalTo(translogId));
         TestUtil.checkIndex(store.directory());
         TestUtil.checkIndex(store.directory());
-        assertDeleteContent(store, directoryService);
+        assertDeleteContent(store, store.directory());
         IOUtils.close(store);
         IOUtils.close(store);
     }
     }
 
 
@@ -893,8 +863,7 @@ public class StoreTests extends ESTestCase {
     public void testMarkCorruptedOnTruncatedSegmentsFile() throws IOException {
     public void testMarkCorruptedOnTruncatedSegmentsFile() throws IOException {
         IndexWriterConfig iwc = newIndexWriterConfig();
         IndexWriterConfig iwc = newIndexWriterConfig();
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId));
         IndexWriter writer = new IndexWriter(store.directory(), iwc);
         IndexWriter writer = new IndexWriter(store.directory(), iwc);
 
 
         int numDocs = 1 + random().nextInt(10);
         int numDocs = 1 + random().nextInt(10);
@@ -945,15 +914,7 @@ public class StoreTests extends ESTestCase {
         writer.commit();
         writer.commit();
         writer.close();
         writer.close();
         assertTrue(Store.canOpenIndex(logger, tempDir, shardId, (id, l) -> new DummyShardLock(id)));
         assertTrue(Store.canOpenIndex(logger, tempDir, shardId, (id, l) -> new DummyShardLock(id)));
-
-        DirectoryService directoryService = new DirectoryService(shardId, INDEX_SETTINGS) {
-
-            @Override
-            public Directory newDirectory() throws IOException {
-                return dir;
-            }
-        };
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, dir, new DummyShardLock(shardId));
         store.markStoreCorrupted(new CorruptIndexException("foo", "bar"));
         store.markStoreCorrupted(new CorruptIndexException("foo", "bar"));
         assertFalse(Store.canOpenIndex(logger, tempDir, shardId, (id, l) -> new DummyShardLock(id)));
         assertFalse(Store.canOpenIndex(logger, tempDir, shardId, (id, l) -> new DummyShardLock(id)));
         store.close();
         store.close();
@@ -962,14 +923,7 @@ public class StoreTests extends ESTestCase {
     public void testDeserializeCorruptionException() throws IOException {
     public void testDeserializeCorruptionException() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final Directory dir = new RAMDirectory(); // I use ram dir to prevent that virusscanner being a PITA
         final Directory dir = new RAMDirectory(); // I use ram dir to prevent that virusscanner being a PITA
-        DirectoryService directoryService = new DirectoryService(shardId, INDEX_SETTINGS) {
-
-            @Override
-            public Directory newDirectory() throws IOException {
-                return dir;
-            }
-        };
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, dir, new DummyShardLock(shardId));
         CorruptIndexException ex = new CorruptIndexException("foo", "bar");
         CorruptIndexException ex = new CorruptIndexException("foo", "bar");
         store.markStoreCorrupted(ex);
         store.markStoreCorrupted(ex);
         try {
         try {
@@ -998,14 +952,7 @@ public class StoreTests extends ESTestCase {
     public void testCanReadOldCorruptionMarker() throws IOException {
     public void testCanReadOldCorruptionMarker() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final Directory dir = new RAMDirectory(); // I use ram dir to prevent that virusscanner being a PITA
         final Directory dir = new RAMDirectory(); // I use ram dir to prevent that virusscanner being a PITA
-        DirectoryService directoryService = new DirectoryService(shardId, INDEX_SETTINGS) {
-
-            @Override
-            public Directory newDirectory() throws IOException {
-                return dir;
-            }
-        };
-        Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        Store store = new Store(shardId, INDEX_SETTINGS, dir, new DummyShardLock(shardId));
 
 
         CorruptIndexException exception = new CorruptIndexException("foo", "bar");
         CorruptIndexException exception = new CorruptIndexException("foo", "bar");
         String uuid = Store.CORRUPTED + UUIDs.randomBase64UUID();
         String uuid = Store.CORRUPTED + UUIDs.randomBase64UUID();
@@ -1065,8 +1012,7 @@ public class StoreTests extends ESTestCase {
 
 
     public void testEnsureIndexHasHistoryUUID() throws IOException {
     public void testEnsureIndexHasHistoryUUID() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
-        try (Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId))) {
+        try (Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId))) {
 
 
             store.createEmpty();
             store.createEmpty();
 
 
@@ -1098,8 +1044,7 @@ public class StoreTests extends ESTestCase {
 
 
     public void testHistoryUUIDCanBeForced() throws IOException {
     public void testHistoryUUIDCanBeForced() throws IOException {
         final ShardId shardId = new ShardId("index", "_na_", 1);
         final ShardId shardId = new ShardId("index", "_na_", 1);
-        DirectoryService directoryService = new LuceneManagedDirectoryService(random());
-        try (Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId))) {
+        try (Store store = new Store(shardId, INDEX_SETTINGS, StoreTests.newDirectory(random()), new DummyShardLock(shardId))) {
 
 
             store.createEmpty();
             store.createEmpty();
 
 

+ 5 - 13
server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java

@@ -63,7 +63,6 @@ import org.elasticsearch.index.shard.IndexShard;
 import org.elasticsearch.index.shard.IndexShardRelocatedException;
 import org.elasticsearch.index.shard.IndexShardRelocatedException;
 import org.elasticsearch.index.shard.IndexShardState;
 import org.elasticsearch.index.shard.IndexShardState;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.index.store.DirectoryService;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.StoreFileMetaData;
 import org.elasticsearch.index.store.StoreFileMetaData;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.Translog;
@@ -461,18 +460,11 @@ public class RecoverySourceHandlerTests extends ESTestCase {
         return newStore(path, true);
         return newStore(path, true);
     }
     }
     private Store newStore(Path path, boolean checkIndex) throws IOException {
     private Store newStore(Path path, boolean checkIndex) throws IOException {
-        DirectoryService directoryService = new DirectoryService(shardId, INDEX_SETTINGS) {
-
-            @Override
-            public Directory newDirectory() throws IOException {
-                BaseDirectoryWrapper baseDirectoryWrapper = RecoverySourceHandlerTests.newFSDirectory(path);
-                if (checkIndex == false) {
-                    baseDirectoryWrapper.setCheckIndexOnClose(false); // don't run checkindex we might corrupt the index in these tests
-                }
-                return baseDirectoryWrapper;
-            }
-        };
-        return new Store(shardId,  INDEX_SETTINGS, directoryService, new DummyShardLock(shardId));
+        BaseDirectoryWrapper baseDirectoryWrapper = RecoverySourceHandlerTests.newFSDirectory(path);
+        if (checkIndex == false) {
+            baseDirectoryWrapper.setCheckIndexOnClose(false); // don't run checkindex we might corrupt the index in these tests
+        }
+        return new Store(shardId,  INDEX_SETTINGS, baseDirectoryWrapper, new DummyShardLock(shardId));
     }
     }
 
 
 
 

+ 1 - 8
test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java

@@ -77,7 +77,6 @@ import org.elasticsearch.index.seqno.LocalCheckpointTracker;
 import org.elasticsearch.index.seqno.ReplicationTracker;
 import org.elasticsearch.index.seqno.ReplicationTracker;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.index.store.DirectoryService;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.TranslogConfig;
 import org.elasticsearch.index.translog.TranslogConfig;
@@ -358,13 +357,7 @@ public abstract class EngineTestCase extends ESTestCase {
     }
     }
 
 
     protected Store createStore(final IndexSettings indexSettings, final Directory directory) throws IOException {
     protected Store createStore(final IndexSettings indexSettings, final Directory directory) throws IOException {
-        final DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
-            @Override
-            public Directory newDirectory() throws IOException {
-                return directory;
-            }
-        };
-        return new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
+        return new Store(shardId, indexSettings, directory, new DummyShardLock(shardId));
     }
     }
 
 
     protected Translog createTranslog(LongSupplier primaryTermSupplier) throws IOException {
     protected Translog createTranslog(LongSupplier primaryTermSupplier) throws IOException {

+ 1 - 8
test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java

@@ -59,7 +59,6 @@ import org.elasticsearch.index.seqno.ReplicationTracker;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.similarity.SimilarityService;
 import org.elasticsearch.index.similarity.SimilarityService;
 import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
 import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
-import org.elasticsearch.index.store.DirectoryService;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
@@ -161,13 +160,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
     }
     }
 
 
     protected Store createStore(ShardId shardId, IndexSettings indexSettings, Directory directory) throws IOException {
     protected Store createStore(ShardId shardId, IndexSettings indexSettings, Directory directory) throws IOException {
-        final DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
-            @Override
-            public Directory newDirectory() throws IOException {
-                return directory;
-            }
-        };
-        return new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
+        return new Store(shardId, indexSettings, directory, new DummyShardLock(shardId));
     }
     }
 
 
     /**
     /**

+ 2 - 9
x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java

@@ -37,7 +37,6 @@ import org.elasticsearch.index.mapper.ParsedDocument;
 import org.elasticsearch.index.mapper.SeqNoFieldMapper;
 import org.elasticsearch.index.mapper.SeqNoFieldMapper;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.index.store.DirectoryService;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.Translog;
 import org.elasticsearch.index.translog.TranslogConfig;
 import org.elasticsearch.index.translog.TranslogConfig;
@@ -261,14 +260,8 @@ public class FollowingEngineTests extends ESTestCase {
     }
     }
 
 
     private static Store createStore(
     private static Store createStore(
-            final ShardId shardId, final IndexSettings indexSettings, final Directory directory) throws IOException {
-        final DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
-            @Override
-            public Directory newDirectory() throws IOException {
-                return directory;
-            }
-        };
-        return new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId));
+            final ShardId shardId, final IndexSettings indexSettings, final Directory directory) {
+        return new Store(shardId, indexSettings, directory, new DummyShardLock(shardId));
     }
     }
 
 
     private FollowingEngine createEngine(Store store, EngineConfig config) throws IOException {
     private FollowingEngine createEngine(Store store, EngineConfig config) throws IOException {