You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/11/07 12:07:49 UTC

svn commit: r1713103 - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/memory/ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/store/ core/src/test/org/apache/lucene/index/ test-framework/src/java/org/apa...

Author: mikemccand
Date: Sat Nov  7 11:07:48 2015
New Revision: 1713103

URL: http://svn.apache.org/viewvc?rev=1713103&view=rev
Log:
LUCENE-6886: Directory.createTempOutput always uses .tmp extension, and codecs are not allowed to

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sat Nov  7 11:07:48 2015
@@ -85,6 +85,10 @@ Changes in Runtime Behavior
 * LUCENE-6789: IndexSearcher's default Similarity is changed to BM25Similarity.
   Use ClassicSimilarity to get the old vector space DefaultSimilarity. (Robert Muir)
 
+* LUCENE-6886: Reserve the .tmp file name extension for temp files,
+  and codec components are no longer allowed to use this extension
+  (Robert Muir, Mike McCandless)
+
 ======================= Lucene 5.4.0 =======================
 
 New Features

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java Sat Nov  7 11:07:48 2015
@@ -120,7 +120,7 @@ import org.apache.lucene.util.fst.Util;
  */
 
 public class FSTTermsWriter extends FieldsConsumer {
-  static final String TERMS_EXTENSION = "tmp";
+  static final String TERMS_EXTENSION = "tfp";
   static final String TERMS_CODEC_NAME = "FSTTerms";
   public static final int TERMS_VERSION_START = 2;
   public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_START;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Sat Nov  7 11:07:48 2015
@@ -17,13 +17,6 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.CollectionUtil;
-import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.InfoStream;
-
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -34,11 +27,19 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.regex.Matcher;
 
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InfoStream;
+
 /*
  * This class keeps track of each SegmentInfos instance that
  * is still "live", either because it corresponds to a
@@ -284,6 +285,11 @@ final class IndexFileDeleter implements
         String segmentName = IndexFileNames.parseSegmentName(fileName);
         assert segmentName.startsWith("_"): "wtf? file=" + fileName;
 
+        if (fileName.toLowerCase(Locale.ROOT).endsWith(".tmp")) {
+          // A temp file: don't try to look at its gen
+          continue;
+        }
+
         maxSegmentName = Math.max(maxSegmentName, Integer.parseInt(segmentName.substring(1), Character.MAX_RADIX));
 
         Long curGen = maxPerSegmentGen.get(segmentName);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Sat Nov  7 11:07:48 2015
@@ -21,6 +21,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
@@ -268,6 +269,9 @@ public final class SegmentInfo {
       if (!m.matches()) {
         throw new IllegalArgumentException("invalid codec filename '" + file + "', must match: " + IndexFileNames.CODEC_FILE_PATTERN.pattern());
       }
+      if (file.toLowerCase(Locale.ROOT).endsWith(".tmp")) {
+        throw new IllegalArgumentException("invalid codec filename '" + file + "', cannot end with .tmp extension");
+      }
     }
   }
   

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java Sat Nov  7 11:07:48 2015
@@ -18,7 +18,6 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
-import java.util.Random;
 
 /**
  * Base implementation for a concrete {@link Directory} that uses a {@link LockFactory} for locking.
@@ -32,22 +31,6 @@ public abstract class BaseDirectory exte
    * this Directory instance). */
   protected final LockFactory lockFactory;
 
-  /** Subclasses can use this to generate temp file name candidates */
-  protected static final Random tempFileRandom;
-
-  static {
-    String prop = System.getProperty("tests.seed");
-    int seed;
-    if (prop != null) {
-      // So if there is a test failure that relied on temp file names,
-      //we remain reproducible based on the test seed:
-      seed = prop.hashCode();
-    } else {
-      seed = (int) System.currentTimeMillis();
-    }
-    tempFileRandom = new Random(seed);
-  }
-
   /** Sole constructor. */
   protected BaseDirectory(LockFactory lockFactory) {
     super();
@@ -64,13 +47,13 @@ public abstract class BaseDirectory exte
 
   @Override
   protected final void ensureOpen() throws AlreadyClosedException {
-    if (!isOpen)
+    if (!isOpen) {
       throw new AlreadyClosedException("this Directory is closed");
+    }
   }
 
   @Override
   public String toString() {
     return super.toString()  + " lockFactory=" + lockFactory;
   }
-  
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java Sat Nov  7 11:07:48 2015
@@ -73,7 +73,8 @@ public abstract class Directory implemen
   public abstract IndexOutput createOutput(String name, IOContext context) throws IOException;
 
   /** Creates a new, empty file for writing in the directory, with a
-   *  temporary file name derived from prefix and suffix.  Use
+   *  temporary file name including prefix and suffix, ending with the
+   *  reserved extension <code>.tmp</code>.  Use
    *  {@link IndexOutput#getName} to see what name was used.  */
   public abstract IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException;
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Sat Nov  7 11:07:48 2015
@@ -31,7 +31,9 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
 
@@ -118,6 +120,9 @@ public abstract class FSDirectory extend
 
   protected final Path directory; // The underlying filesystem directory
 
+  /** Used to generate temp file names in {@link #createTempOutput}. */
+  private final AtomicLong nextTempFileCounter = new AtomicLong();
+
   /** Create a new FSDirectory for the named location (ctor for subclasses).
    * The directory is created at the named location if it does not yet exist.
    * 
@@ -231,9 +236,10 @@ public abstract class FSDirectory extend
   public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
     ensureOpen();
     while (true) {
-      String name = prefix + tempFileRandom.nextInt(Integer.MAX_VALUE) + "." + suffix;
       try {
-        return new FSIndexOutput(name, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
+        String name = IndexFileNames.segmentFileName(prefix, suffix + "_" + Long.toString(nextTempFileCounter.getAndIncrement(), Character.MAX_RADIX), "tmp");
+        return new FSIndexOutput(name,
+                                 StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
       } catch (FileAlreadyExistsException faee) {
         // Retry with next random name
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Sat Nov  7 11:07:48 2015
@@ -28,6 +28,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 
@@ -51,6 +52,9 @@ public class RAMDirectory extends BaseDi
   protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<>();
   protected final AtomicLong sizeInBytes = new AtomicLong();
   
+  /** Used to generate temp file names in {@link #createTempOutput}. */
+  private final AtomicLong nextTempFileCounter = new AtomicLong();
+
   /** Constructs an empty {@link Directory}. */
   public RAMDirectory() {
     this(new SingleInstanceLockFactory());
@@ -186,7 +190,7 @@ public class RAMDirectory extends BaseDi
 
     // ... then try to find a unique name for it:
     while (true) {
-      String name = prefix + tempFileRandom.nextInt(Integer.MAX_VALUE) + "." + suffix;
+      String name = IndexFileNames.segmentFileName(prefix, suffix + "_" + Long.toString(nextTempFileCounter.getAndIncrement(), Character.MAX_RADIX), "tmp");
       if (fileMap.putIfAbsent(name, file) == null) {
         return new RAMOutputStream(name, file, true);
       }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Sat Nov  7 11:07:48 2015
@@ -18,9 +18,11 @@ package org.apache.lucene.index;
  */
 
 import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.StringReader;
+import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -2810,5 +2812,31 @@ public class TestIndexWriter extends Luc
     assertFalse(r2.getIndexCommit().getSegmentsFileName().equals(r.getIndexCommit().getSegmentsFileName()));
     IOUtils.close(r, r2, w, dir);
   }
+
+  public void testLeftoverTempFiles() throws Exception {
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriter w = new IndexWriter(dir, iwc);
+    w.close();
+    
+    IndexOutput out = dir.createTempOutput("_0", "bkd", IOContext.DEFAULT);
+    String tempName = out.getName();
+    out.close();
+    iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    w = new IndexWriter(dir, iwc);
+
+    // Make sure IW deleted the unref'd file:
+    try {
+      dir.openInput(tempName, IOContext.DEFAULT);
+      fail("did not hit exception");
+    } catch (FileNotFoundException | NoSuchFileException e) {
+      // expected
+    }
+    w.close();
+    dir.close();
+  }
 }
 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1713103&r1=1713102&r2=1713103&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Sat Nov  7 11:07:48 2015
@@ -30,6 +30,7 @@ import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
@@ -620,6 +621,10 @@ public class MockDirectoryWrapper extend
     
     IndexOutput delegateOutput = in.createTempOutput(prefix, suffix, LuceneTestCase.newIOContext(randomState, context));
     String name = delegateOutput.getName();
+    if (name.toLowerCase(Locale.ROOT).endsWith(".tmp") == false) {
+      throw new IllegalStateException("wrapped directory failed to use .tmp extension: got: " + name);
+    }
+
     unSyncedFiles.add(name);
     createdFiles.add(name);
     final IndexOutput io = new MockIndexOutputWrapper(this, delegateOutput, name);