فهرست منبع

Replace internal usages of SimpleFS with NIOFS (#74996)

SimpleFS is deprecated and removed in Lucene 9. This commit replaces its 
internal usages with NIOFS. Two other usages (store type and SMB) need
to be deprecated before switching to niofs.
Nhat Nguyen 4 سال پیش
والد
کامیت
60c2838906

+ 25 - 13
distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java

@@ -9,9 +9,9 @@
 package org.elasticsearch.common.settings;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.elasticsearch.common.Randomness;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
@@ -198,8 +198,10 @@ public class KeyStoreWrapperTests extends ESTestCase {
     public void testFailWhenCannotConsumeSecretStream() throws Exception {
         assumeFalse("Cannot open unprotected keystore on FIPS JVM", inFipsJvm());
         Path configDir = env.configFile();
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
-        try (IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)) {
+        try (
+            Directory directory = newFSDirectory(configDir);
+            IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)
+        ) {
             CodecUtil.writeHeader(indexOutput, "elasticsearch.keystore", 3);
             indexOutput.writeByte((byte) 0); // No password
             SecureRandom random = Randomness.createSecure();
@@ -227,8 +229,10 @@ public class KeyStoreWrapperTests extends ESTestCase {
     public void testFailWhenCannotConsumeEncryptedBytesStream() throws Exception {
         assumeFalse("Cannot open unprotected keystore on FIPS JVM", inFipsJvm());
         Path configDir = env.configFile();
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
-        try (IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)) {
+        try (
+            Directory directory = newFSDirectory(configDir);
+            IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)
+        ) {
             CodecUtil.writeHeader(indexOutput, "elasticsearch.keystore", 3);
             indexOutput.writeByte((byte) 0); // No password
             SecureRandom random = Randomness.createSecure();
@@ -257,8 +261,10 @@ public class KeyStoreWrapperTests extends ESTestCase {
     public void testFailWhenSecretStreamNotConsumed() throws Exception {
         assumeFalse("Cannot open unprotected keystore on FIPS JVM", inFipsJvm());
         Path configDir = env.configFile();
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
-        try (IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)) {
+        try (
+            Directory directory = newFSDirectory(configDir);
+            IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)
+        ) {
             CodecUtil.writeHeader(indexOutput, "elasticsearch.keystore", 3);
             indexOutput.writeByte((byte) 0); // No password
             SecureRandom random = Randomness.createSecure();
@@ -285,8 +291,10 @@ public class KeyStoreWrapperTests extends ESTestCase {
     public void testFailWhenEncryptedBytesStreamIsNotConsumed() throws Exception {
         assumeFalse("Cannot open unprotected keystore on FIPS JVM", inFipsJvm());
         Path configDir = env.configFile();
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
-        try (IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)) {
+        try (
+            Directory directory = newFSDirectory(configDir);
+            IndexOutput indexOutput = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)
+        ) {
             CodecUtil.writeHeader(indexOutput, "elasticsearch.keystore", 3);
             indexOutput.writeByte((byte) 0); // No password
             SecureRandom random = Randomness.createSecure();
@@ -372,8 +380,10 @@ public class KeyStoreWrapperTests extends ESTestCase {
     public void testBackcompatV1() throws Exception {
         assumeFalse("Can't run in a FIPS JVM as PBE is not available", inFipsJvm());
         Path configDir = env.configFile();
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
-        try (IndexOutput output = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)) {
+        try (
+            Directory directory = newFSDirectory(configDir);
+            IndexOutput output = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)
+        ) {
             CodecUtil.writeHeader(output, "elasticsearch.keystore", 1);
             output.writeByte((byte) 0); // hasPassword = false
             output.writeString("PKCS12");
@@ -403,10 +413,12 @@ public class KeyStoreWrapperTests extends ESTestCase {
     public void testBackcompatV2() throws Exception {
         assumeFalse("Can't run in a FIPS JVM as PBE is not available", inFipsJvm());
         Path configDir = env.configFile();
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
         byte[] fileBytes = new byte[20];
         random().nextBytes(fileBytes);
-        try (IndexOutput output = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)) {
+        try (
+            Directory directory = newFSDirectory(configDir);
+            IndexOutput output = directory.createOutput("elasticsearch.keystore", IOContext.DEFAULT)
+        ) {
 
             CodecUtil.writeHeader(output, "elasticsearch.keystore", 2);
             output.writeByte((byte) 0); // hasPassword = false

+ 2 - 3
server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/AllocationIdIT.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.cluster.routing;
 
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.elasticsearch.action.admin.cluster.allocation.ClusterAllocationExplanation;
 import org.elasticsearch.action.admin.indices.stats.ShardStats;
 import org.elasticsearch.action.index.IndexRequestBuilder;
@@ -126,7 +125,7 @@ public class AllocationIdIT extends ESIntegTestCase {
         });
 
         internalCluster().stopNode(node1);
-        try(Store store = new Store(shardId, indexSettings, new SimpleFSDirectory(indexPath), new DummyShardLock(shardId))) {
+        try(Store store = new Store(shardId, indexSettings, newFSDirectory(indexPath), new DummyShardLock(shardId))) {
             store.removeCorruptionMarker();
         }
         node1 = internalCluster().startNode(node1DataPathSettings);
@@ -204,7 +203,7 @@ public class AllocationIdIT extends ESIntegTestCase {
     }
 
     private void putFakeCorruptionMarker(IndexSettings indexSettings, ShardId shardId, Path indexPath) throws IOException {
-        try(Store store = new Store(shardId, indexSettings, new SimpleFSDirectory(indexPath), new DummyShardLock(shardId))) {
+        try(Store store = new Store(shardId, indexSettings, newFSDirectory(indexPath), new DummyShardLock(shardId))) {
             store.markStoreCorrupted(new IOException("fake ioexception"));
         }
     }

+ 4 - 3
server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java

@@ -13,10 +13,11 @@ import org.apache.lucene.index.IndexFormatTooNewException;
 import org.apache.lucene.index.IndexFormatTooOldException;
 import org.apache.lucene.store.BufferedChecksumIndexInput;
 import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.util.SetOnce;
 import org.elasticsearch.cli.ExitCodes;
 import org.elasticsearch.cli.UserException;
@@ -205,7 +206,7 @@ public class KeyStoreWrapper implements SecureSettings {
             return null;
         }
 
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
+        Directory directory = new NIOFSDirectory(configDir);
         try (IndexInput indexInput = directory.openInput(KEYSTORE_FILENAME, IOContext.READONCE)) {
             ChecksumIndexInput input = new BufferedChecksumIndexInput(indexInput);
             final int formatVersion;
@@ -478,7 +479,7 @@ public class KeyStoreWrapper implements SecureSettings {
     public synchronized void save(Path configDir, char[] password) throws Exception {
         ensureOpen();
 
-        SimpleFSDirectory directory = new SimpleFSDirectory(configDir);
+        Directory directory = new NIOFSDirectory(configDir);
         // write to tmp file first, then overwrite
         String tmpFile = KEYSTORE_FILENAME + ".tmp";
         try (IndexOutput output = directory.createOutput(tmpFile, IOContext.DEFAULT)) {

+ 2 - 2
server/src/main/java/org/elasticsearch/env/NodeEnvironment.java

@@ -18,8 +18,8 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.store.NativeFSLockFactory;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
@@ -540,7 +540,7 @@ public final class NodeEnvironment  implements Closeable {
                 // resolve the directory the shard actually lives in
                 Path p = shardPaths[i].resolve("index");
                 // open a directory (will be immediately closed) on the shard's location
-                dirs[i] = new SimpleFSDirectory(p, indexSettings.getValue(FsDirectoryFactory.INDEX_LOCK_FACTOR_SETTING));
+                dirs[i] = new NIOFSDirectory(p, indexSettings.getValue(FsDirectoryFactory.INDEX_LOCK_FACTOR_SETTING));
                 // create a lock for the "write.lock" file
                 try {
                     locks[i] = dirs[i].obtainLock(IndexWriter.WRITE_LOCK_NAME);

+ 2 - 2
server/src/main/java/org/elasticsearch/gateway/MetadataStateFormat.java

@@ -18,7 +18,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.common.lucene.store.IndexOutputOutputStream;
@@ -292,7 +292,7 @@ public abstract class MetadataStateFormat<T> {
     }
 
     protected Directory newDirectory(Path dir) throws IOException {
-        return new SimpleFSDirectory(dir);
+        return new NIOFSDirectory(dir);
     }
 
 

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

@@ -33,7 +33,7 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -432,7 +432,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
     public static MetadataSnapshot readMetadataSnapshot(Path indexLocation, ShardId shardId, NodeEnvironment.ShardLocker shardLocker,
                                                         Logger logger) throws IOException {
         try (ShardLock lock = shardLocker.lock(shardId, "read metadata snapshot", TimeUnit.SECONDS.toMillis(5));
-             Directory dir = new SimpleFSDirectory(indexLocation)) {
+             Directory dir = new NIOFSDirectory(indexLocation)) {
             failIfCorrupted(dir);
             return new MetadataSnapshot(null, dir, logger);
         } catch (IndexNotFoundException ex) {
@@ -451,9 +451,9 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
      * be opened, an exception is thrown
      */
     public static void tryOpenIndex(Path indexLocation, ShardId shardId, NodeEnvironment.ShardLocker shardLocker,
-                                        Logger logger) throws IOException, ShardLockObtainFailedException {
+                                    Logger logger) throws IOException, ShardLockObtainFailedException {
         try (ShardLock lock = shardLocker.lock(shardId, "open index", TimeUnit.SECONDS.toMillis(5));
-             Directory dir = new SimpleFSDirectory(indexLocation)) {
+             Directory dir = new NIOFSDirectory(indexLocation)) {
             failIfCorrupted(dir);
             SegmentInfos segInfo = Lucene.readSegmentInfos(dir);
             logger.trace("{} loaded segment info [{}]", shardId, segInfo);

+ 2 - 2
server/src/main/java/org/elasticsearch/index/translog/Checkpoint.java

@@ -17,8 +17,8 @@ import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.store.OutputStreamIndexOutput;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.elasticsearch.common.io.Channels;
 import org.elasticsearch.index.seqno.SequenceNumbers;
 
@@ -141,7 +141,7 @@ final class Checkpoint {
     }
 
     public static Checkpoint read(Path path) throws IOException {
-        try (Directory dir = new SimpleFSDirectory(path.getParent())) {
+        try (Directory dir = new NIOFSDirectory(path.getParent())) {
             try (IndexInput indexInput = dir.openInput(path.getFileName().toString(), IOContext.DEFAULT)) {
                 // We checksum the entire file before we even go and parse it. If it's corrupted we barf right here.
                 CodecUtil.checksumEntireFile(indexInput);

+ 2 - 2
server/src/main/java/org/elasticsearch/indices/analysis/HunspellService.java

@@ -12,7 +12,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.apache.lucene.analysis.hunspell.Dictionary;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.common.io.FileSystemUtils;
@@ -184,7 +184,7 @@ public class HunspellService {
 
             affixStream = Files.newInputStream(affixFiles[0]);
 
-            try (Directory tmp = new SimpleFSDirectory(env.tmpFile())) {
+            try (Directory tmp = new NIOFSDirectory(env.tmpFile())) {
                 return new Dictionary(tmp, "hunspell", affixStream, dicStreams, ignoreCase);
             }
 

+ 1 - 2
server/src/test/java/org/elasticsearch/gateway/MetadataStateFormatTests.java

@@ -14,7 +14,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.elasticsearch.cluster.ClusterModule;
 import org.elasticsearch.cluster.metadata.Metadata;
@@ -155,7 +154,7 @@ public class MetadataStateFormatTests extends ESTestCase {
     }
 
     public static void corruptFile(Path fileToCorrupt, Logger logger) throws IOException {
-        try (SimpleFSDirectory dir = new SimpleFSDirectory(fileToCorrupt.getParent())) {
+        try (Directory dir = newFSDirectory(fileToCorrupt.getParent())) {
             long checksumBeforeCorruption;
             try (IndexInput input = dir.openInput(fileToCorrupt.getFileName().toString(), IOContext.DEFAULT)) {
                 checksumBeforeCorruption = CodecUtil.retrieveChecksum(input);

+ 5 - 6
server/src/test/java/org/elasticsearch/gateway/PersistedClusterStateServiceTests.java

@@ -18,7 +18,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
@@ -278,7 +277,7 @@ public class PersistedClusterStateServiceTests extends ESTestCase {
             }
 
             final Path brokenPath = nodeEnvironment.nodeDataPath();
-            try (Directory directory = new SimpleFSDirectory(brokenPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME))) {
+            try (Directory directory = newFSDirectory(brokenPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME))) {
                 final IndexWriterConfig indexWriterConfig = new IndexWriterConfig();
                 indexWriterConfig.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
                 try (IndexWriter indexWriter = new IndexWriter(directory, indexWriterConfig)) {
@@ -310,8 +309,8 @@ public class PersistedClusterStateServiceTests extends ESTestCase {
                 writeState(writer2, 0L, newClusterState, oldClusterState);
             }
 
-            try (Directory directory = new SimpleFSDirectory(brokenPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME));
-                 Directory dupDirectory = new SimpleFSDirectory(dupPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME))) {
+            try (Directory directory = newFSDirectory(brokenPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME));
+                 Directory dupDirectory = newFSDirectory(dupPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME))) {
                 try (IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig())) {
                     indexWriter.addIndexes(dupDirectory);
                     indexWriter.commit();
@@ -355,8 +354,8 @@ public class PersistedClusterStateServiceTests extends ESTestCase {
                 writeState(writer2, 0L, newClusterState, oldClusterState);
             }
 
-            try (Directory directory = new SimpleFSDirectory(brokenPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME));
-                 Directory dupDirectory = new SimpleFSDirectory(dupPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME))) {
+            try (Directory directory = newFSDirectory(brokenPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME));
+                 Directory dupDirectory = newFSDirectory(dupPath.resolve(PersistedClusterStateService.METADATA_DIRECTORY_NAME))) {
                 try (IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig())) {
                     indexWriter.deleteDocuments(new Term("type", "global")); // do not duplicate global metadata
                     indexWriter.addIndexes(dupDirectory);

+ 1 - 2
server/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java

@@ -18,7 +18,6 @@ import org.apache.lucene.analysis.hunspell.Dictionary;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.common.io.Streams;
@@ -402,7 +401,7 @@ public class AnalysisModuleTests extends ESTestCase {
         InputStream aff = getClass().getResourceAsStream("/indices/analyze/conf_dir/hunspell/en_US/en_US.aff");
         InputStream dic = getClass().getResourceAsStream("/indices/analyze/conf_dir/hunspell/en_US/en_US.dic");
         Dictionary dictionary;
-        try (Directory tmp = new SimpleFSDirectory(environment.tmpFile())) {
+        try (Directory tmp = newFSDirectory(environment.tmpFile())) {
             dictionary = new Dictionary(tmp, "hunspell", aff, dic);
         }
         AnalysisModule module = new AnalysisModule(environment, singletonList(new AnalysisPlugin() {

+ 2 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotRepository.java

@@ -17,7 +17,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.FilterDirectory;
-import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.store.NIOFSDirectory;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.ClusterState;
@@ -146,7 +146,7 @@ public final class SourceOnlySnapshotRepository extends FilterRepository {
         final List<Closeable> toClose = new ArrayList<>(3);
         try {
             SourceOnlySnapshot.LinkedFilesDirectory overlayDir = new SourceOnlySnapshot.LinkedFilesDirectory(
-                new SimpleFSDirectory(snapPath));
+                new NIOFSDirectory(snapPath));
             toClose.add(overlayDir);
             Store tempStore = new Store(store.shardId(), store.indexSettings(), overlayDir, new ShardLock(store.shardId()) {
                 @Override