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/10/19 15:38:44 UTC

svn commit: r1709414 [2/5] - in /lucene/dev/branches/lucene6835: ./ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ l...

Modified: lucene/dev/branches/lucene6835/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java Mon Oct 19 13:38:42 2015
@@ -17,24 +17,20 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.Closeable;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardCopyOption;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Locale;
 
+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.TrackingDirectoryWrapper;
+
 /**
  * On-disk sorting of byte arrays. Each byte array (entry) is a composed of the following
  * fields:
@@ -43,14 +39,12 @@ import java.util.Locale;
  *   <li>exactly the above count of bytes for the sequence to be sorted.
  * </ul>
  * 
- * @see #sort(Path, Path)
+ * @see #sort(String)
  * @lucene.experimental
  * @lucene.internal
  */
 public final class OfflineSorter {
 
-  private static Path DEFAULT_TEMP_DIR;
-
   /** Convenience constant for megabytes */
   public final static long MB = 1024 * 1024;
   /** Convenience constant for gigabytes */
@@ -72,6 +66,10 @@ public final class OfflineSorter {
    */
   public final static int MAX_TEMPFILES = 128;
 
+  private final Directory dir;
+
+  private final String tempFileNamePrefix;
+
   /** 
    * A bit more descriptive unit for constructors.
    * 
@@ -142,7 +140,7 @@ public final class OfflineSorter {
     /** number of partition merges */
     public int mergeRounds;
     /** number of lines of data read */
-    public int lines;
+    public int lineCount;
     /** time spent merging sorted partitions (in milliseconds) */
     public long mergeTime;
     /** time spent sorting data (in milliseconds) */
@@ -162,17 +160,16 @@ public final class OfflineSorter {
       return String.format(Locale.ROOT,
           "time=%.2f sec. total (%.2f reading, %.2f sorting, %.2f merging), lines=%d, temp files=%d, merges=%d, soft ram limit=%.2f MB",
           totalTime / 1000.0d, readTime / 1000.0d, sortTime / 1000.0d, mergeTime / 1000.0d,
-          lines, tempMergeFiles, mergeRounds,
+          lineCount, tempMergeFiles, mergeRounds,
           (double) bufferSize / MB);
     }
   }
 
   private final BufferSize ramBufferSize;
-  private final Path tempDirectory;
   
   private final Counter bufferBytesUsed = Counter.newCounter();
   private final BytesRefArray buffer = new BytesRefArray(bufferBytesUsed);
-  private SortInfo sortInfo;
+  SortInfo sortInfo;
   private int maxTempFiles;
   private final Comparator<BytesRef> comparator;
   
@@ -182,27 +179,25 @@ public final class OfflineSorter {
   /**
    * Defaults constructor.
    * 
-   * @see #getDefaultTempDir()
    * @see BufferSize#automatic()
    */
-  public OfflineSorter() throws IOException {
-    this(DEFAULT_COMPARATOR, BufferSize.automatic(), getDefaultTempDir(), MAX_TEMPFILES);
+  public OfflineSorter(Directory dir, String tempFileNamePrefix) throws IOException {
+    this(dir, tempFileNamePrefix, DEFAULT_COMPARATOR, BufferSize.automatic(), MAX_TEMPFILES);
   }
   
   /**
    * Defaults constructor with a custom comparator.
    * 
-   * @see #getDefaultTempDir()
    * @see BufferSize#automatic()
    */
-  public OfflineSorter(Comparator<BytesRef> comparator) throws IOException {
-    this(comparator, BufferSize.automatic(), getDefaultTempDir(), MAX_TEMPFILES);
+  public OfflineSorter(Directory dir, String tempFileNamePrefix, Comparator<BytesRef> comparator) throws IOException {
+    this(dir, tempFileNamePrefix, comparator, BufferSize.automatic(), MAX_TEMPFILES);
   }
 
   /**
    * All-details constructor.
    */
-  public OfflineSorter(Comparator<BytesRef> comparator, BufferSize ramBufferSize, Path tempDirectory, int maxTempfiles) {
+  public OfflineSorter(Directory dir, String tempFileNamePrefix, Comparator<BytesRef> comparator, BufferSize ramBufferSize, int maxTempfiles) {
     if (ramBufferSize.bytes < ABSOLUTE_MIN_SORT_BUFFER_SIZE) {
       throw new IllegalArgumentException(MIN_BUFFER_SIZE_MSG + ": " + ramBufferSize.bytes);
     }
@@ -212,160 +207,129 @@ public final class OfflineSorter {
     }
 
     this.ramBufferSize = ramBufferSize;
-    this.tempDirectory = tempDirectory;
     this.maxTempFiles = maxTempfiles;
     this.comparator = comparator;
+    this.dir = dir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /** Returns the {@link Directory} we use to create temp files. */
+  public Directory getDirectory() {
+    return dir;
+  }
+
+  /** Returns the temp file name prefix passed to {@link Directory#createTempOutput} to generate temporary files. */
+  public String getTempFileNamePrefix() {
+    return tempFileNamePrefix;
   }
 
   /** 
-   * Sort input to output, explicit hint for the buffer size. The amount of allocated
-   * memory may deviate from the hint (may be smaller or larger).  
+   * Sort input to a new temp file, returning its name.
    */
-  public SortInfo sort(Path input, Path output) throws IOException {
+  public String sort(String inputFileName) throws IOException {
+    
     sortInfo = new SortInfo();
     sortInfo.totalTime = System.currentTimeMillis();
 
-    // NOTE: don't remove output here: its existence (often created by the caller
-    // up above using Files.createTempFile) prevents another concurrent caller
-    // of this API (from a different thread) from incorrectly re-using this file name
-
-    ArrayList<Path> merges = new ArrayList<>();
-    boolean success3 = false;
-    try {
-      ByteSequencesReader is = new ByteSequencesReader(input);
-      boolean success = false;
-      try {
-        int lines = 0;
-        while ((lines = readPartition(is)) > 0) {
-          merges.add(sortPartition(lines));
-          sortInfo.tempMergeFiles++;
-          sortInfo.lines += lines;
-
-          // Handle intermediate merges.
-          if (merges.size() == maxTempFiles) {
-            Path intermediate = Files.createTempFile(tempDirectory, "sort", "intermediate");
-            boolean success2 = false;
-            try {
-              mergePartitions(merges, intermediate);
-              success2 = true;
-            } finally {
-              if (success2) {
-                IOUtils.deleteFilesIfExist(merges);
-              } else {
-                IOUtils.deleteFilesIgnoringExceptions(merges);
-              }
-              merges.clear();
-              merges.add(intermediate);
-            }
-            sortInfo.tempMergeFiles++;
-          }
-        }
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(is);
-        } else {
-          IOUtils.closeWhileHandlingException(is);
+    List<String> segments = new ArrayList<>();
+
+    // So we can remove any partially written temp files on exception:
+    TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+
+    boolean success = false;
+    try (ByteSequencesReader is = new ByteSequencesReader(dir.openInput(inputFileName, IOContext.READONCE))) {
+
+      int lineCount;
+      while ((lineCount = readPartition(is)) > 0) {
+        segments.add(sortPartition(trackingDir));
+        sortInfo.tempMergeFiles++;
+        sortInfo.lineCount += lineCount;
+
+        // Handle intermediate merges.
+        if (segments.size() == maxTempFiles) {
+          mergePartitions(trackingDir, segments);
         }
       }
 
-      // One partition, try to rename or copy if unsuccessful.
-      if (merges.size() == 1) {     
-        Files.move(merges.get(0), output, StandardCopyOption.REPLACE_EXISTING);
-      } else { 
-        // otherwise merge the partitions with a priority queue.
-        mergePartitions(merges, output);
+      // Merge the partitions to the output file with a priority queue.
+      if (segments.size() > 1) {     
+        mergePartitions(trackingDir, segments);
       }
-      success3 = true;
-    } finally {
-      if (success3) {
-        IOUtils.deleteFilesIfExist(merges);
+
+      String result;
+      if (segments.isEmpty()) {
+        try (IndexOutput out = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT)) {
+          result = out.getName();
+        }
       } else {
-        IOUtils.deleteFilesIgnoringExceptions(merges);
-        IOUtils.deleteFilesIgnoringExceptions(output);
+        result = segments.get(0);
       }
-    }
 
-    sortInfo.totalTime = (System.currentTimeMillis() - sortInfo.totalTime); 
-    return sortInfo;
-  }
+      // We should be explicitly removing all intermediate files ourselves unless there is an exception:
+      assert trackingDir.getCreatedFiles().size() == 1 && trackingDir.getCreatedFiles().contains(result);
 
-  /** Used by test framework */
-  static void setDefaultTempDir(Path tempDir) {
-    DEFAULT_TEMP_DIR = tempDir;
-  }
+      sortInfo.totalTime = (System.currentTimeMillis() - sortInfo.totalTime); 
+      success = true;
 
-  /**
-   * Returns the default temporary directory. By default, java.io.tmpdir. If not accessible
-   * or not available, an IOException is thrown
-   */
-  public synchronized static Path getDefaultTempDir() throws IOException {
-    if (DEFAULT_TEMP_DIR == null) {
-      // Lazy init
-      String tempDirPath = System.getProperty("java.io.tmpdir");
-      if (tempDirPath == null)  {
-        throw new IOException("Java has no temporary folder property (java.io.tmpdir)?");
-      }
-      Path tempDirectory = Paths.get(tempDirPath);
-      if (Files.isWritable(tempDirectory) == false) {
-        throw new IOException("Java's temporary folder not present or writeable?: " 
-                              + tempDirectory.toAbsolutePath());
+      return result;
+
+    } finally {
+      if (success == false) {
+        IOUtils.deleteFilesIgnoringExceptions(trackingDir, trackingDir.getCreatedFiles());
       }
-      DEFAULT_TEMP_DIR = tempDirectory;
     }
-
-    return DEFAULT_TEMP_DIR;
   }
 
   /** Sort a single partition in-memory. */
-  protected Path sortPartition(int len) throws IOException {
+  protected String sortPartition(TrackingDirectoryWrapper trackingDir) throws IOException {
     BytesRefArray data = this.buffer;
-    Path tempFile = Files.createTempFile(tempDirectory, "sort", "partition");
 
-    long start = System.currentTimeMillis();
-    sortInfo.sortTime += (System.currentTimeMillis() - start);
-    
-    final ByteSequencesWriter out = new ByteSequencesWriter(tempFile);
-    BytesRef spare;
-    try {
+    try (IndexOutput tempFile = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT)) {
+      ByteSequencesWriter out = new ByteSequencesWriter(tempFile);
+      BytesRef spare;
+
+      long start = System.currentTimeMillis();
       BytesRefIterator iter = buffer.iterator(comparator);
-      while((spare = iter.next()) != null) {
+      sortInfo.sortTime += (System.currentTimeMillis() - start);
+
+      while ((spare = iter.next()) != null) {
         assert spare.length <= Short.MAX_VALUE;
         out.write(spare);
       }
       
-      out.close();
-
       // Clean up the buffer for the next partition.
       data.clear();
-      return tempFile;
-    } finally {
-      IOUtils.close(out);
+
+      return tempFile.getName();
     }
   }
 
-  /** Merge a list of sorted temporary files (partitions) into an output file */
-  void mergePartitions(List<Path> merges, Path outputFile) throws IOException {
+  /** Merge a list of sorted temporary files (partitions) into an output file.  Note that this closes the
+   *  incoming {@link IndexOutput}. */
+  void mergePartitions(Directory trackingDir, List<String> segments) throws IOException {
     long start = System.currentTimeMillis();
 
-    ByteSequencesWriter out = new ByteSequencesWriter(outputFile);
-
-    PriorityQueue<FileAndTop> queue = new PriorityQueue<FileAndTop>(merges.size()) {
+    PriorityQueue<FileAndTop> queue = new PriorityQueue<FileAndTop>(segments.size()) {
       @Override
       protected boolean lessThan(FileAndTop a, FileAndTop b) {
         return comparator.compare(a.current.get(), b.current.get()) < 0;
       }
     };
 
-    ByteSequencesReader [] streams = new ByteSequencesReader [merges.size()];
-    try {
+    ByteSequencesReader[] streams = new ByteSequencesReader[segments.size()];
+
+    String newSegmentName = null;
+
+    try (IndexOutput out = trackingDir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT)) {
+      newSegmentName = out.getName();
+      ByteSequencesWriter writer = new ByteSequencesWriter(out);
+
       // Open streams and read the top for each file
-      for (int i = 0; i < merges.size(); i++) {
-        streams[i] = new ByteSequencesReader(merges.get(i));
-        byte line[] = streams[i].read();
-        if (line != null) {
-          queue.insertWithOverflow(new FileAndTop(i, line));
-        }
+      for (int i = 0; i < segments.size(); i++) {
+        streams[i] = new ByteSequencesReader(dir.openInput(segments.get(i), IOContext.READONCE));
+        byte[] line = streams[i].read();
+        assert line != null;
+        queue.insertWithOverflow(new FileAndTop(i, line));
       }
   
       // Unix utility sort() uses ordered array of files to pick the next line from, updating
@@ -374,7 +338,7 @@ public final class OfflineSorter {
       // so it shouldn't make much of a difference (didn't check).
       FileAndTop top;
       while ((top = queue.top()) != null) {
-        out.write(top.current.bytes(), 0, top.current.length());
+        writer.write(top.current.bytes(), 0, top.current.length());
         if (!streams[top.fd].read(top.current)) {
           queue.pop();
         } else {
@@ -385,14 +349,15 @@ public final class OfflineSorter {
       sortInfo.mergeTime += System.currentTimeMillis() - start;
       sortInfo.mergeRounds++;
     } finally {
-      // The logic below is: if an exception occurs in closing out, it has a priority over exceptions
-      // happening in closing streams.
-      try {
-        IOUtils.close(streams);
-      } finally {
-        IOUtils.close(out);
-      }
+      IOUtils.close(streams);
     }
+
+    IOUtils.deleteFiles(trackingDir, segments);
+
+    segments.clear();
+    segments.add(newSegmentName);
+
+    sortInfo.tempMergeFiles++;
   }
 
   /** Read in a single partition of data */
@@ -428,18 +393,11 @@ public final class OfflineSorter {
    * Complementary to {@link ByteSequencesReader}.
    */
   public static class ByteSequencesWriter implements Closeable {
-    private final DataOutput os;
-
-    /** Constructs a ByteSequencesWriter to the provided Path */
-    public ByteSequencesWriter(Path path) throws IOException {
-      this(new DataOutputStream(
-          new BufferedOutputStream(
-              Files.newOutputStream(path))));
-    }
+    private final IndexOutput out;
 
     /** Constructs a ByteSequencesWriter to the provided DataOutput */
-    public ByteSequencesWriter(DataOutput os) {
-      this.os = os;
+    public ByteSequencesWriter(IndexOutput out) {
+      this.out = out;
     }
 
     /**
@@ -455,7 +413,7 @@ public final class OfflineSorter {
      * Writes a byte array.
      * @see #write(byte[], int, int)
      */
-    public void write(byte [] bytes) throws IOException {
+    public void write(byte[] bytes) throws IOException {
       write(bytes, 0, bytes.length);
     }
 
@@ -465,25 +423,23 @@ public final class OfflineSorter {
      * The length is written as a <code>short</code>, followed
      * by the bytes.
      */
-    public void write(byte [] bytes, int off, int len) throws IOException {
+    public void write(byte[] bytes, int off, int len) throws IOException {
       assert bytes != null;
       assert off >= 0 && off + len <= bytes.length;
       assert len >= 0;
       if (len > Short.MAX_VALUE) {
         throw new IllegalArgumentException("len must be <= " + Short.MAX_VALUE + "; got " + len);
       }
-      os.writeShort(len);
-      os.write(bytes, off, len);
+      out.writeShort((short) len);
+      out.writeBytes(bytes, off, len);
     }
     
     /**
-     * Closes the provided {@link DataOutput} if it is {@link Closeable}.
+     * Closes the provided {@link IndexOutput}.
      */
     @Override
     public void close() throws IOException {
-      if (os instanceof Closeable) {
-        ((Closeable) os).close();
-      }
+      out.close();
     }    
   }
 
@@ -492,18 +448,11 @@ public final class OfflineSorter {
    * Complementary to {@link ByteSequencesWriter}.
    */
   public static class ByteSequencesReader implements Closeable {
-    private final DataInput is;
+    private final IndexInput in;
 
-    /** Constructs a ByteSequencesReader from the provided Path */
-    public ByteSequencesReader(Path path) throws IOException {
-      this(new DataInputStream(
-          new BufferedInputStream(
-              Files.newInputStream(path))));
-    }
-
-    /** Constructs a ByteSequencesReader from the provided DataInput */
-    public ByteSequencesReader(DataInput is) {
-      this.is = is;
+    /** Constructs a ByteSequencesReader from the provided IndexInput */
+    public ByteSequencesReader(IndexInput in) {
+      this.in = in;
     }
 
     /**
@@ -517,14 +466,14 @@ public final class OfflineSorter {
     public boolean read(BytesRefBuilder ref) throws IOException {
       short length;
       try {
-        length = is.readShort();
+        length = in.readShort();
       } catch (EOFException e) {
         return false;
       }
 
       ref.grow(length);
       ref.setLength(length);
-      is.readFully(ref.bytes(), 0, length);
+      in.readBytes(ref.bytes(), 0, length);
       return true;
     }
 
@@ -540,25 +489,23 @@ public final class OfflineSorter {
     public byte[] read() throws IOException {
       short length;
       try {
-        length = is.readShort();
+        length = in.readShort();
       } catch (EOFException e) {
         return null;
       }
 
       assert length >= 0 : "Sanity: sequence length < 0: " + length;
-      byte [] result = new byte [length];
-      is.readFully(result);
+      byte[] result = new byte[length];
+      in.readBytes(result, 0, length);
       return result;
     }
 
     /**
-     * Closes the provided {@link DataInput} if it is {@link Closeable}.
+     * Closes the provided {@link IndexInput}.
      */
     @Override
     public void close() throws IOException {
-      if (is instanceof Closeable) {
-        ((Closeable) is).close();
-      }
+      in.close();
     }
   }
 

Modified: lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java (original)
+++ lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/index/TestCodecUtil.java Mon Oct 19 13:38:42 2015
@@ -283,7 +283,7 @@ public class TestCodecUtil extends Lucen
     final IndexOutput output = new RAMOutputStream(file, false);
     AtomicLong fakeChecksum = new AtomicLong();
     // wrap the index input where we control the checksum for mocking
-    IndexOutput fakeOutput = new IndexOutput("fake") {
+    IndexOutput fakeOutput = new IndexOutput("fake", "fake") {
       @Override
       public void close() throws IOException {
         output.close();

Modified: lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java (original)
+++ lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java Mon Oct 19 13:38:42 2015
@@ -30,6 +30,8 @@ import org.apache.lucene.document.Field.
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
@@ -179,4 +181,46 @@ public class TestIndexSearcher extends L
     dir.close();
   }
 
+  public void testGetQueryCache() throws IOException {
+    IndexSearcher searcher = new IndexSearcher(new MultiReader());
+    assertEquals(IndexSearcher.getDefaultQueryCache(), searcher.getQueryCache());
+    QueryCache dummyCache = new QueryCache() {
+      @Override
+      public Weight doCache(Weight weight, QueryCachingPolicy policy) {
+        return weight;
+      }
+    };
+    searcher.setQueryCache(dummyCache);
+    assertEquals(dummyCache, searcher.getQueryCache());
+
+    IndexSearcher.setDefaultQueryCache(dummyCache);
+    searcher = new IndexSearcher(new MultiReader());
+    assertEquals(dummyCache, searcher.getQueryCache());
+
+    searcher.setQueryCache(null);
+    assertNull(searcher.getQueryCache());
+
+    IndexSearcher.setDefaultQueryCache(null);
+    searcher = new IndexSearcher(new MultiReader());
+    assertNull(searcher.getQueryCache());
+  }
+
+  public void testGetQueryCachingPolicy() throws IOException {
+    IndexSearcher searcher = new IndexSearcher(new MultiReader());
+    assertEquals(IndexSearcher.getDefaultQueryCachingPolicy(), searcher.getQueryCachingPolicy());
+    QueryCachingPolicy dummyPolicy = new QueryCachingPolicy() {
+      @Override
+      public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+        return false;
+      }
+      @Override
+      public void onUse(Query query) {}
+    };
+    searcher.setQueryCachingPolicy(dummyPolicy);
+    assertEquals(dummyPolicy, searcher.getQueryCachingPolicy());
+
+    IndexSearcher.setDefaultQueryCachingPolicy(dummyPolicy);
+    searcher = new IndexSearcher(new MultiReader());
+    assertEquals(dummyPolicy, searcher.getQueryCachingPolicy());
+  }
 }

Modified: lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java Mon Oct 19 13:38:42 2015
@@ -17,20 +17,22 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+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.MockDirectoryWrapper;
 import org.apache.lucene.util.OfflineSorter.BufferSize;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter.SortInfo;
-import org.apache.lucene.util.OfflineSorter;
 
 /**
  * Tests for on-disk merge sorting.
@@ -52,35 +54,61 @@ public class TestOfflineSorter extends L
     super.tearDown();
   }
 
+  private static Directory newDirectoryNoVirusScanner() {
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
+
+  private static Directory newFSDirectoryNoVirusScanner() {
+    Directory dir = newFSDirectory(createTempDir());
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
+
   public void testEmpty() throws Exception {
-    checkSort(new OfflineSorter(), new byte [][] {});
+    try (Directory dir = newDirectoryNoVirusScanner()) {
+        checkSort(dir, new OfflineSorter(dir, "foo"), new byte [][] {});
+    }
   }
 
   public void testSingleLine() throws Exception {
-    checkSort(new OfflineSorter(), new byte [][] {
-        "Single line only.".getBytes(StandardCharsets.UTF_8)
-    });
+    try (Directory dir = newDirectoryNoVirusScanner()) {
+      checkSort(dir, new OfflineSorter(dir, "foo"), new byte [][] {
+          "Single line only.".getBytes(StandardCharsets.UTF_8)
+        });
+    }
   }
 
   public void testIntermediateMerges() throws Exception {
     // Sort 20 mb worth of data with 1mb buffer, binary merging.
-    SortInfo info = checkSort(new OfflineSorter(OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.getDefaultTempDir(), 2), 
-        generateRandom((int)OfflineSorter.MB * 20));
-    assertTrue(info.mergeRounds > 10);
+    try (Directory dir = newDirectoryNoVirusScanner()) {
+      SortInfo info = checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), 2), 
+          generateRandom((int)OfflineSorter.MB * 20));
+      assertTrue(info.mergeRounds > 10);
+    }
   }
 
   public void testSmallRandom() throws Exception {
     // Sort 20 mb worth of data with 1mb buffer.
-    SortInfo sortInfo = checkSort(new OfflineSorter(OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.getDefaultTempDir(), OfflineSorter.MAX_TEMPFILES), 
-        generateRandom((int)OfflineSorter.MB * 20));
-    assertEquals(1, sortInfo.mergeRounds);
+    try (Directory dir = newDirectoryNoVirusScanner()) {
+      SortInfo sortInfo = checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.MAX_TEMPFILES),
+                                    generateRandom((int)OfflineSorter.MB * 20));
+      assertEquals(1, sortInfo.mergeRounds);
+    }
   }
 
   @Nightly
   public void testLargerRandom() throws Exception {
     // Sort 100MB worth of data with 15mb buffer.
-    checkSort(new OfflineSorter(OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(16), OfflineSorter.getDefaultTempDir(), OfflineSorter.MAX_TEMPFILES), 
-        generateRandom((int)OfflineSorter.MB * 100));
+    try (Directory dir = newFSDirectoryNoVirusScanner()) {
+      checkSort(dir, new OfflineSorter(dir, "foo", OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(16), OfflineSorter.MAX_TEMPFILES), 
+                generateRandom((int)OfflineSorter.MB * 100));
+    }
   }
 
   private byte[][] generateRandom(int howMuchDataInBytes) {
@@ -101,8 +129,9 @@ public class TestOfflineSorter extends L
       final int max = Math.min(left.length, right.length);
       for (int i = 0, j = 0; i < max; i++, j++) {
         int diff = (left[i]  & 0xff) - (right[j] & 0xff); 
-        if (diff != 0) 
+        if (diff != 0) {
           return diff;
+        }
       }
       return left.length - right.length;
     }
@@ -111,54 +140,56 @@ public class TestOfflineSorter extends L
   /**
    * Check sorting data on an instance of {@link OfflineSorter}.
    */
-  private SortInfo checkSort(OfflineSorter sort, byte[][] data) throws IOException {
-    Path unsorted = writeAll("unsorted", data);
+  private SortInfo checkSort(Directory dir, OfflineSorter sorter, byte[][] data) throws IOException {
+
+    IndexOutput unsorted = dir.createTempOutput("unsorted", "tmp", IOContext.DEFAULT);
+    writeAll(unsorted, data);
 
+    IndexOutput golden = dir.createTempOutput("golden", "tmp", IOContext.DEFAULT);
     Arrays.sort(data, unsignedByteOrderComparator);
-    Path golden = writeAll("golden", data);
+    writeAll(golden, data);
 
-    Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
-    SortInfo sortInfo;
-    try {
-      sortInfo = sort.sort(unsorted, sorted);
-      //System.out.println("Input size [MB]: " + unsorted.length() / (1024 * 1024));
-      //System.out.println(sortInfo);
-      assertFilesIdentical(golden, sorted);
-    } finally {
-      IOUtils.rm(unsorted, golden, sorted);
-    }
+    String sorted = sorter.sort(unsorted.getName());
+    //System.out.println("Input size [MB]: " + unsorted.length() / (1024 * 1024));
+    //System.out.println(sortInfo);
+    assertFilesIdentical(dir, golden.getName(), sorted);
 
-    return sortInfo;
+    return sorter.sortInfo;
   }
 
   /**
    * Make sure two files are byte-byte identical.
    */
-  private void assertFilesIdentical(Path golden, Path sorted) throws IOException {
-    assertEquals(Files.size(golden), Files.size(sorted));
-
-    byte [] buf1 = new byte [64 * 1024];
-    byte [] buf2 = new byte [64 * 1024];
-    int len;
-    DataInputStream is1 = new DataInputStream(Files.newInputStream(golden));
-    DataInputStream is2 = new DataInputStream(Files.newInputStream(sorted));
-    while ((len = is1.read(buf1)) > 0) {
-      is2.readFully(buf2, 0, len);
-      for (int i = 0; i < len; i++) {
-        assertEquals(buf1[i], buf2[i]);
+  private void assertFilesIdentical(Directory dir, String golden, String sorted) throws IOException {
+    long numBytes = dir.fileLength(golden);
+    assertEquals(numBytes, dir.fileLength(sorted));
+
+    byte[] buf1 = new byte[64 * 1024];
+    byte[] buf2 = new byte[64 * 1024];
+    try (
+         IndexInput in1 = dir.openInput(golden, IOContext.READONCE);
+         IndexInput in2 = dir.openInput(sorted, IOContext.READONCE)
+         ) {
+      long left = numBytes;
+      while (left > 0) {
+        int chunk = (int) Math.min(buf1.length, left);
+        left -= chunk;
+        in1.readBytes(buf1, 0, chunk);
+        in2.readBytes(buf2, 0, chunk);
+        for (int i = 0; i < chunk; i++) {
+          assertEquals(buf1[i], buf2[i]);
+        }
       }
     }
-    IOUtils.close(is1, is2);
   }
 
-  private Path writeAll(String name, byte[][] data) throws IOException {
-    Path file = Files.createTempFile(tempDir, name, "");
-    ByteSequencesWriter w = new OfflineSorter.ByteSequencesWriter(file);
-    for (byte [] datum : data) {
-      w.write(datum);
+  /** NOTE: closes the provided {@link IndexOutput} */
+  private void writeAll(IndexOutput out, byte[][] data) throws IOException {
+    try (ByteSequencesWriter w = new OfflineSorter.ByteSequencesWriter(out)) {
+      for (byte [] datum : data) {
+        w.write(datum);
+      }
     }
-    w.close();
-    return file;
   }
   
   public void testRamBuffer() {
@@ -192,25 +223,27 @@ public class TestOfflineSorter extends L
     Thread[] threads = new Thread[TestUtil.nextInt(random(), 4, 10)];
     final AtomicBoolean failed = new AtomicBoolean();
     final int iters = atLeast(1000);
-    for(int i=0;i<threads.length;i++) {
-      threads[i] = new Thread() {
-          @Override
-          public void run() {
-            try {
-              for(int iter=0;iter<iters && failed.get() == false;iter++) {
-                checkSort(new OfflineSorter(), generateRandom(1024));
+    try (Directory dir = newDirectoryNoVirusScanner()) {
+      for(int i=0;i<threads.length;i++) {
+        final int threadID = i;
+        threads[i] = new Thread() {
+            @Override
+            public void run() {
+              try {
+                for(int iter=0;iter<iters && failed.get() == false;iter++) {
+                  checkSort(dir, new OfflineSorter(dir, "foo_" + threadID + "_" + iter), generateRandom(1024));
+                }
+              } catch (Throwable th) {
+                failed.set(true);
+                throw new RuntimeException(th);
               }
-            } catch (Throwable th) {
-              failed.set(true);
-              throw new RuntimeException(th);
             }
-          }
-        };
-      threads[i].start();
-    }
-
-    for(Thread thread : threads) {
-      thread.join();
+          };
+        threads[i].start();
+      }
+      for(Thread thread : threads) {
+        thread.join();
+      }
     }
 
     assertFalse(failed.get());

Modified: lucene/dev/branches/lucene6835/lucene/expressions/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/expressions/build.xml?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/expressions/build.xml (original)
+++ lucene/dev/branches/lucene6835/lucene/expressions/build.xml Mon Oct 19 13:38:42 2015
@@ -48,7 +48,7 @@
   <target name="regenerate" depends="run-antlr"/>
 
   <target name="resolve-antlr" xmlns:ivy="antlib:org.apache.ivy.ant">
-    <ivy:cachepath organisation="org.antlr" module="antlr4" revision="4.5"
+    <ivy:cachepath organisation="org.antlr" module="antlr4" revision="4.5.1-1"
                   inline="true" conf="default" type="jar" pathid="antlr.classpath"/>
   </target>
 

Modified: lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptBaseVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptBaseVisitor.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptBaseVisitor.java (original)
+++ lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptBaseVisitor.java Mon Oct 19 13:38:42 2015
@@ -1,6 +1,5 @@
 // ANTLR GENERATED CODE: DO NOT EDIT
 package org.apache.lucene.expressions.js;
-import org.antlr.v4.runtime.misc.NotNull;
 import org.antlr.v4.runtime.tree.AbstractParseTreeVisitor;
 
 /**

Modified: lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptLexer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptLexer.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptLexer.java (original)
+++ lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptLexer.java Mon Oct 19 13:38:42 2015
@@ -11,7 +11,7 @@ import org.antlr.v4.runtime.misc.*;
 
 @SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"})
 class JavascriptLexer extends Lexer {
-  static { RuntimeMetaData.checkVersion("4.5", RuntimeMetaData.VERSION); }
+  static { RuntimeMetaData.checkVersion("4.5.1", RuntimeMetaData.VERSION); }
 
   protected static final DFA[] _decisionToDFA;
   protected static final PredictionContextCache _sharedContextCache =

Modified: lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptParser.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptParser.java (original)
+++ lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptParser.java Mon Oct 19 13:38:42 2015
@@ -11,7 +11,7 @@ import java.util.ArrayList;
 
 @SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"})
 class JavascriptParser extends Parser {
-  static { RuntimeMetaData.checkVersion("4.5", RuntimeMetaData.VERSION); }
+  static { RuntimeMetaData.checkVersion("4.5.1", RuntimeMetaData.VERSION); }
 
   protected static final DFA[] _decisionToDFA;
   protected static final PredictionContextCache _sharedContextCache =

Modified: lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptVisitor.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptVisitor.java (original)
+++ lucene/dev/branches/lucene6835/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptVisitor.java Mon Oct 19 13:38:42 2015
@@ -1,6 +1,5 @@
 // ANTLR GENERATED CODE: DO NOT EDIT
 package org.apache.lucene.expressions.js;
-import org.antlr.v4.runtime.misc.NotNull;
 import org.antlr.v4.runtime.tree.ParseTreeVisitor;
 
 /**

Modified: lucene/dev/branches/lucene6835/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java (original)
+++ lucene/dev/branches/lucene6835/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java Mon Oct 19 13:38:42 2015
@@ -143,7 +143,7 @@ public class SlowRAMDirectory extends RA
     private final Random rand;
     
     public SlowIndexOutput(IndexOutput io) {
-      super("SlowIndexOutput(" + io + ")");
+      super("SlowIndexOutput(" + io + ")", io.getName());
       this.io = io;
       this.rand = forkRandom();
     }

Modified: lucene/dev/branches/lucene6835/lucene/ivy-versions.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/ivy-versions.properties?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/ivy-versions.properties (original)
+++ lucene/dev/branches/lucene6835/lucene/ivy-versions.properties Mon Oct 19 13:38:42 2015
@@ -20,7 +20,7 @@ com.codahale.metrics.version = 3.0.1
 /com.cybozu.labs/langdetect = 1.1-20120112
 /com.drewnoakes/metadata-extractor = 2.6.2
 
-/com.facebook.presto/presto-parser = 0.108
+/com.facebook.presto/presto-parser = 0.122
 
 com.fasterxml.jackson.core.version = 2.5.4
 /com.fasterxml.jackson.core/jackson-annotations = ${com.fasterxml.jackson.core.version}
@@ -89,7 +89,7 @@ com.sun.jersey.version = 1.9
 /net.sourceforge.argparse4j/argparse4j = 0.4.3
 /net.sourceforge.jmatio/jmatio = 1.0
 /net.sourceforge.nekohtml/nekohtml = 1.9.17
-/org.antlr/antlr4-runtime = 4.5
+/org.antlr/antlr4-runtime = 4.5.1-1
 
 /org.apache.ant/ant = 1.8.2
 /org.apache.avro/avro = 1.7.5

Modified: lucene/dev/branches/lucene6835/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java (original)
+++ lucene/dev/branches/lucene6835/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java Mon Oct 19 13:38:42 2015
@@ -18,10 +18,10 @@ package org.apache.lucene.store;
  */
 
 import java.io.EOFException;
-import java.io.IOException;
-import java.io.FileInputStream;
 import java.io.FileDescriptor;
+import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.Path;
@@ -149,7 +149,7 @@ public class NativeUnixDirectory extends
       return delegate.createOutput(name, context);
     } else {
       ensureCanWrite(name);
-      return new NativeUnixIndexOutput(getDirectory().resolve(name), mergeBufferSize);
+      return new NativeUnixIndexOutput(getDirectory().resolve(name), name, mergeBufferSize);
     }
   }
 
@@ -167,8 +167,8 @@ public class NativeUnixDirectory extends
     private long fileLength;
     private boolean isOpen;
 
-    public NativeUnixIndexOutput(Path path, int bufferSize) throws IOException {
-      super("NativeUnixIndexOutput(path=\"" + path.toString() + "\")");
+    public NativeUnixIndexOutput(Path path, String name, int bufferSize) throws IOException {
+      super("NativeUnixIndexOutput(path=\"" + path.toString() + "\")", name);
       //this.path = path;
       final FileDescriptor fd = NativePosixUtil.open_direct(path.toString(), false);
       fos = new FileOutputStream(fd);

Modified: lucene/dev/branches/lucene6835/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java (original)
+++ lucene/dev/branches/lucene6835/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java Mon Oct 19 13:38:42 2015
@@ -51,15 +51,27 @@ public class PayloadScoreQuery extends S
 
   private final SpanQuery wrappedQuery;
   private final PayloadFunction function;
+  private final boolean includeSpanScore;
 
   /**
    * Creates a new PayloadScoreQuery
    * @param wrappedQuery the query to wrap
    * @param function a PayloadFunction to use to modify the scores
+   * @param includeSpanScore include both span score and payload score in the scoring algorithm
    */
-  public PayloadScoreQuery(SpanQuery wrappedQuery, PayloadFunction function) {
+  public PayloadScoreQuery(SpanQuery wrappedQuery, PayloadFunction function, boolean includeSpanScore) {
     this.wrappedQuery = wrappedQuery;
     this.function = function;
+    this.includeSpanScore = includeSpanScore;
+  }
+
+  /**
+   * Creates a new PayloadScoreQuery that includes the underlying span scores
+   * @param wrappedQuery the query to wrap
+   * @param function a PayloadFunction to use to modify the scores
+   */
+  public PayloadScoreQuery(SpanQuery wrappedQuery, PayloadFunction function) {
+    this(wrappedQuery, function, true);
   }
 
   @Override
@@ -149,12 +161,16 @@ public class PayloadScoreQuery extends S
       if (scorer == null || scorer.advance(doc) != doc)
         return Explanation.noMatch("No match");
 
-      SpanWeight innerWeight = ((PayloadSpanWeight)scorer.getWeight()).innerWeight;
-      Explanation innerExpl = innerWeight.explain(context, doc);
       scorer.freq();  // force freq calculation
       Explanation payloadExpl = scorer.getPayloadExplanation();
 
-      return Explanation.match(scorer.scoreCurrentDoc(), "PayloadSpanQuery, product of:", innerExpl, payloadExpl);
+      if (includeSpanScore) {
+        SpanWeight innerWeight = ((PayloadSpanWeight) scorer.getWeight()).innerWeight;
+        Explanation innerExpl = innerWeight.explain(context, doc);
+        return Explanation.match(scorer.scoreCurrentDoc(), "PayloadSpanQuery, product of:", innerExpl, payloadExpl);
+      }
+
+      return scorer.getPayloadExplanation();
     }
   }
 
@@ -203,7 +219,9 @@ public class PayloadScoreQuery extends S
 
     @Override
     protected float scoreCurrentDoc() throws IOException {
-      return getSpanScore() * getPayloadScore();
+      if (includeSpanScore)
+        return getSpanScore() * getPayloadScore();
+      return getPayloadScore();
     }
 
     @Override

Modified: lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadExplanations.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadExplanations.java (original)
+++ lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadExplanations.java Mon Oct 19 13:38:42 2015
@@ -51,7 +51,7 @@ public class TestPayloadExplanations ext
 
   /** macro for payloadscorequery */
   private SpanQuery pt(String s, PayloadFunction fn) {
-    return new PayloadScoreQuery(new SpanTermQuery(new Term(FIELD,s)), fn);
+    return new PayloadScoreQuery(new SpanTermQuery(new Term(FIELD,s)), fn, random().nextBoolean());
   }
   
   /* simple PayloadTermQueries */
@@ -82,8 +82,6 @@ public class TestPayloadExplanations ext
     }
   }
 
-  // TODO: test the payloadnear query too!
-
   /*
     protected static final String[] docFields = {
     "w1 w2 w3 w4 w5",
@@ -95,7 +93,7 @@ public class TestPayloadExplanations ext
 
   public void testAllFunctions(SpanQuery query, int[] expected) throws Exception {
     for (PayloadFunction fn : functions) {
-      qtest(new PayloadScoreQuery(query, fn), expected);
+      qtest(new PayloadScoreQuery(query, fn, random().nextBoolean()), expected);
     }
   }
 

Modified: lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadScoreQuery.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadScoreQuery.java (original)
+++ lucene/dev/branches/lucene6835/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadScoreQuery.java Mon Oct 19 13:38:42 2015
@@ -35,6 +35,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryUtils;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
@@ -54,10 +55,14 @@ import org.junit.Test;
 public class TestPayloadScoreQuery extends LuceneTestCase {
 
   private static void checkQuery(SpanQuery query, PayloadFunction function, int[] expectedDocs, float[] expectedScores) throws IOException {
+    checkQuery(query, function, true, expectedDocs, expectedScores);
+  }
+
+  private static void checkQuery(SpanQuery query, PayloadFunction function, boolean includeSpanScore, int[] expectedDocs, float[] expectedScores) throws IOException {
 
     assertTrue("Expected docs and scores arrays must be the same length!", expectedDocs.length == expectedScores.length);
 
-    PayloadScoreQuery psq = new PayloadScoreQuery(query, function);
+    PayloadScoreQuery psq = new PayloadScoreQuery(query, function, includeSpanScore);
     TopDocs hits = searcher.search(psq, expectedDocs.length);
 
     for (int i = 0; i < hits.scoreDocs.length; i++) {
@@ -70,6 +75,8 @@ public class TestPayloadScoreQuery exten
 
     if (hits.scoreDocs.length > expectedDocs.length)
       fail("Unexpected hit in document " + hits.scoreDocs[expectedDocs.length]);
+
+    QueryUtils.check(random(), psq, searcher);
   }
 
   @Test
@@ -132,9 +139,19 @@ public class TestPayloadScoreQuery exten
         }, 0, true)
     }, 1, true);
 
-    checkQuery(q, new MaxPayloadFunction(), new int[]{ 122, 222 }, new float[]{ 4.0f, 4.0f });
-    checkQuery(q, new MinPayloadFunction(), new int[]{ 222, 122 }, new float[]{ 4.0f, 2.0f });
-    checkQuery(q, new AveragePayloadFunction(), new int[] { 222, 122 }, new float[]{ 4.0f, 3.666666f });
+    // check includeSpanScore makes a difference here
+    searcher.setSimilarity(new MultiplyingSimilarity());
+    try {
+      checkQuery(q, new MaxPayloadFunction(), new int[]{ 122, 222 }, new float[]{ 41.802513122558594f, 34.13160705566406f });
+      checkQuery(q, new MinPayloadFunction(), new int[]{ 222, 122 }, new float[]{ 34.13160705566406f, 20.901256561279297f });
+      checkQuery(q, new AveragePayloadFunction(), new int[] { 122, 222 }, new float[]{ 38.3189697265625f, 34.13160705566406f });
+      checkQuery(q, new MaxPayloadFunction(), false, new int[]{122, 222}, new float[]{4.0f, 4.0f});
+      checkQuery(q, new MinPayloadFunction(), false, new int[]{222, 122}, new float[]{4.0f, 2.0f});
+      checkQuery(q, new AveragePayloadFunction(), false, new int[]{222, 122}, new float[]{4.0f, 3.666666f});
+    }
+    finally {
+      searcher.setSimilarity(similarity);
+    }
 
   }
 
@@ -234,22 +251,26 @@ public class TestPayloadScoreQuery exten
     directory = null;
   }
 
-  static class BoostingSimilarity extends ClassicSimilarity {
+  static class MultiplyingSimilarity extends ClassicSimilarity {
 
     @Override
-    public float queryNorm(float sumOfSquaredWeights) {
-      return 1.0f;
+    public float scorePayload(int docId, int start, int end, BytesRef payload) {
+      //we know it is size 4 here, so ignore the offset/length
+      return payload.bytes[payload.offset];
     }
 
+  }
+
+  static class BoostingSimilarity extends MultiplyingSimilarity {
+
     @Override
-    public float coord(int overlap, int maxOverlap) {
+    public float queryNorm(float sumOfSquaredWeights) {
       return 1.0f;
     }
 
     @Override
-    public float scorePayload(int docId, int start, int end, BytesRef payload) {
-      //we know it is size 4 here, so ignore the offset/length
-      return payload.bytes[payload.offset];
+    public float coord(int overlap, int maxOverlap) {
+      return 1.0f;
     }
 
     //!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesConsumer.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesConsumer.java Mon Oct 19 13:38:42 2015
@@ -28,6 +28,7 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -39,9 +40,13 @@ class BKDTreeDocValuesConsumer extends D
   final IndexOutput out;
   final Map<Integer,Long> fieldIndexFPs = new HashMap<>();
   final SegmentWriteState state;
+  final Directory tempDir;
+  final String tempFileNamePrefix;
 
-  public BKDTreeDocValuesConsumer(DocValuesConsumer delegate, SegmentWriteState state, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
+  public BKDTreeDocValuesConsumer(Directory tempDir, String tempFileNamePrefix, DocValuesConsumer delegate, SegmentWriteState state, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
     BKDTreeWriter.verifyParams(maxPointsInLeafNode, maxPointsSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.delegate = delegate;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
     this.maxPointsSortInHeap = maxPointsSortInHeap;
@@ -91,7 +96,7 @@ class BKDTreeDocValuesConsumer extends D
   @Override
   public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
     delegate.addSortedNumericField(field, docToValueCount, values);
-    BKDTreeWriter writer = new BKDTreeWriter(maxPointsInLeafNode, maxPointsSortInHeap);
+    BKDTreeWriter writer = new BKDTreeWriter(tempDir, tempFileNamePrefix, maxPointsInLeafNode, maxPointsSortInHeap);
     Iterator<Number> valueIt = values.iterator();
     Iterator<Number> valueCountIt = docToValueCount.iterator();
     for (int docID=0;docID<state.segmentInfo.maxDoc();docID++) {

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeDocValuesFormat.java Mon Oct 19 13:38:42 2015
@@ -99,7 +99,7 @@ public class BKDTreeDocValuesFormat exte
 
   @Override
   public DocValuesConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
-    return new BKDTreeDocValuesConsumer(delegate.fieldsConsumer(state), state, maxPointsInLeafNode, maxPointsSortInHeap);
+    return new BKDTreeDocValuesConsumer(state.directory, state.segmentInfo.name, delegate.fieldsConsumer(state), state, maxPointsInLeafNode, maxPointsSortInHeap);
   }
 
   @Override

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java Mon Oct 19 13:38:42 2015
@@ -18,14 +18,13 @@ package org.apache.lucene.bkdtree;
  */
 
 import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -33,8 +32,8 @@ import org.apache.lucene.util.BytesRefBu
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.LongBitSet;
-import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO
@@ -88,22 +87,27 @@ class BKDTreeWriter {
   private final byte[] scratchBytes = new byte[BYTES_PER_DOC];
   private final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);
 
-  private OfflineSorter.ByteSequencesWriter writer;
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
+  private OfflineSorter.ByteSequencesWriter offlineWriter;
   private GrowingHeapLatLonWriter heapWriter;
 
-  private Path tempInput;
+  private IndexOutput tempInput;
   private final int maxPointsInLeafNode;
   private final int maxPointsSortInHeap;
 
   private long pointCount;
 
-  public BKDTreeWriter() throws IOException {
-    this(DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_POINTS_SORT_IN_HEAP);
+  public BKDTreeWriter(Directory tempDir, String tempFileNamePrefix) throws IOException {
+    this(tempDir, tempFileNamePrefix, DEFAULT_MAX_POINTS_IN_LEAF_NODE, DEFAULT_MAX_POINTS_SORT_IN_HEAP);
   }
 
   // TODO: instead of maxPointsSortInHeap, change to maxMBHeap ... the mapping is non-obvious:
-  public BKDTreeWriter(int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
+  public BKDTreeWriter(Directory tempDir, String tempFileNamePrefix, int maxPointsInLeafNode, int maxPointsSortInHeap) throws IOException {
     verifyParams(maxPointsInLeafNode, maxPointsSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
     this.maxPointsSortInHeap = maxPointsSortInHeap;
 
@@ -143,8 +147,8 @@ class BKDTreeWriter {
   private void switchToOffline() throws IOException {
 
     // For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
-    tempInput = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "in", "");
-    writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+    tempInput = tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT);
+    offlineWriter = new OfflineSorter.ByteSequencesWriter(tempInput);
     for(int i=0;i<pointCount;i++) {
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeInt(heapWriter.latEncs[i]);
@@ -152,7 +156,7 @@ class BKDTreeWriter {
       scratchBytesOutput.writeVInt(heapWriter.docIDs[i]);
       scratchBytesOutput.writeVLong(i);
       // TODO: can/should OfflineSorter optimize the fixed-width case?
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     }
 
     heapWriter = null;
@@ -165,7 +169,7 @@ class BKDTreeWriter {
     assert lonEnc < Integer.MAX_VALUE;
 
     if (pointCount >= maxPointsSortInHeap) {
-      if (writer == null) {
+      if (offlineWriter == null) {
         switchToOffline();
       }
       scratchBytesOutput.reset(scratchBytes);
@@ -173,7 +177,7 @@ class BKDTreeWriter {
       scratchBytesOutput.writeInt(lonEnc);
       scratchBytesOutput.writeVInt(docID);
       scratchBytesOutput.writeVLong(pointCount);
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     } else {
       // Not too many points added yet, continue using heap:
       heapWriter.append(latEnc, lonEnc, pointCount, docID);
@@ -184,7 +188,7 @@ class BKDTreeWriter {
 
   /** Changes incoming {@link ByteSequencesWriter} file to to fixed-width-per-entry file, because we need to be able to slice
    *  as we recurse in {@link #build}. */
-  private LatLonWriter convertToFixedWidth(Path in) throws IOException {
+  private LatLonWriter convertToFixedWidth(String in) throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     scratch.grow(BYTES_PER_DOC);
     BytesRef bytes = scratch.get();
@@ -194,7 +198,7 @@ class BKDTreeWriter {
     LatLonWriter sortedWriter = null;
     boolean success = false;
     try {
-      reader = new OfflineSorter.ByteSequencesReader(in);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(in, IOContext.READONCE));
       sortedWriter = getWriter(pointCount);
       for (long i=0;i<pointCount;i++) {
         boolean result = reader.read(scratch);
@@ -230,10 +234,10 @@ class BKDTreeWriter {
 
   private LatLonWriter sort(boolean lon) throws IOException {
     if (heapWriter != null) {
+      // All buffered points are still in heap
 
       assert pointCount < Integer.MAX_VALUE;
 
-      // All buffered points are still in heap
       new InPlaceMergeSorter() {
         @Override
         protected void swap(int i, int j) {
@@ -329,20 +333,19 @@ class BKDTreeWriter {
         }
       };
 
-      Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
-      boolean success = false;
       
+      boolean success = false;
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp);
+      String sortedFileName = sorter.sort(tempInput.getName());
       try {
-        OfflineSorter latSorter = new OfflineSorter(cmp);
-        latSorter.sort(tempInput, sorted);
-        LatLonWriter writer = convertToFixedWidth(sorted);
+        LatLonWriter writer = convertToFixedWidth(sortedFileName);
         success = true;
         return writer;
       } finally {
         if (success) {
-          IOUtils.rm(sorted);
+          tempDir.deleteFile(sortedFileName);
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(sorted);
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, sortedFileName);
         }
       }
     }
@@ -352,8 +355,9 @@ class BKDTreeWriter {
   public long finish(IndexOutput out) throws IOException {
     //System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapWriter);
 
-    if (writer != null) {
-      writer.close();
+    if (offlineWriter != null) {
+      // This also closes the temp file output:
+      offlineWriter.close();
     }
 
     LongBitSet bitSet = new LongBitSet(pointCount);
@@ -410,7 +414,9 @@ class BKDTreeWriter {
       if (success) {
         latSortedWriter.destroy();
         lonSortedWriter.destroy();
-        IOUtils.rm(tempInput);
+        if (tempInput != null) {
+          tempDir.deleteFile(tempInput.getName());
+        }
       } else {
         try {
           latSortedWriter.destroy();
@@ -422,7 +428,9 @@ class BKDTreeWriter {
         } catch (Throwable t) {
           // Suppress to keep throwing original exc
         }
-        IOUtils.deleteFilesIgnoringExceptions(tempInput);
+        if (tempInput != null) {
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName());
+        }
       }
     }
 
@@ -813,7 +821,7 @@ class BKDTreeWriter {
     if (count < maxPointsSortInHeap) {
       return new HeapLatLonWriter((int) count);
     } else {
-      return new OfflineLatLonWriter(count);
+      return new OfflineLatLonWriter(tempDir, tempFileNamePrefix, count);
     }
   }
 

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonReader.java Mon Oct 19 13:38:42 2015
@@ -17,34 +17,23 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 
-import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 
 final class OfflineLatLonReader implements LatLonReader {
-  final InputStreamDataInput in;
+  final IndexInput in;
   long countLeft;
   private int latEnc;
   private int lonEnc;
   private long ord;
   private int docID;
 
-  OfflineLatLonReader(Path tempFile, long start, long count) throws IOException {
-    InputStream fis = Files.newInputStream(tempFile);
-    long seekFP = start * BKDTreeWriter.BYTES_PER_DOC;
-    long skipped = 0;
-    while (skipped < seekFP) {
-      long inc = fis.skip(seekFP - skipped);
-      skipped += inc;
-      if (inc == 0) {
-        throw new RuntimeException("skip returned 0");
-      }
-    }
-    in = new InputStreamDataInput(new BufferedInputStream(fis));
+  OfflineLatLonReader(Directory tempDir, String tempFileName, long start, long count) throws IOException {
+    in = tempDir.openInput(tempFileName, IOContext.READONCE);
+    in.seek(start * BKDTreeWriter.BYTES_PER_DOC);
     this.countLeft = count;
   }
 

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/bkdtree/OfflineLatLonWriter.java Mon Oct 19 13:38:42 2015
@@ -17,29 +17,26 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.OfflineSorter;
-
-import java.io.BufferedOutputStream;
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
+
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
 
 final class OfflineLatLonWriter implements LatLonWriter {
 
-  final Path tempFile;
+  final Directory tempDir;
   final byte[] scratchBytes = new byte[BKDTreeWriter.BYTES_PER_DOC];
   final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);      
-  final OutputStreamDataOutput out;
+  final IndexOutput out;
   final long count;
   private long countWritten;
   private boolean closed;
 
-  public OfflineLatLonWriter(long count) throws IOException {
-    tempFile = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "size" + count + ".", "");
-    out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+  public OfflineLatLonWriter(Directory tempDir, String tempFileNamePrefix, long count) throws IOException {
+    this.tempDir = tempDir;
+    out = tempDir.createTempOutput(tempFileNamePrefix, "bkd", IOContext.DEFAULT);
     this.count = count;
   }
     
@@ -55,7 +52,7 @@ final class OfflineLatLonWriter implemen
   @Override
   public LatLonReader getReader(long start) throws IOException {
     assert closed;
-    return new OfflineLatLonReader(tempFile, start, count-start);
+    return new OfflineLatLonReader(tempDir, out.getName(), start, count-start);
   }
 
   @Override
@@ -69,12 +66,12 @@ final class OfflineLatLonWriter implemen
 
   @Override
   public void destroy() throws IOException {
-    IOUtils.rm(tempFile);
+    tempDir.deleteFile(out.getName());
   }
 
   @Override
   public String toString() {
-    return "OfflineLatLonWriter(count=" + count + " tempFile=" + tempFile + ")";
+    return "OfflineLatLonWriter(count=" + count + " tempFileName=" + out.getName() + ")";
   }
 }
 

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceReader.java Mon Oct 19 13:38:42 2015
@@ -17,33 +17,22 @@ package org.apache.lucene.rangetree;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 
-import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 
 final class OfflineSliceReader implements SliceReader {
-  final InputStreamDataInput in;
-  long countLeft;
+  final IndexInput in;
+  private long countLeft;
   private long value;
   private long ord;
   private int docID;
 
-  OfflineSliceReader(Path tempFile, long start, long count) throws IOException {
-    InputStream fis = Files.newInputStream(tempFile);
-    long seekFP = start * RangeTreeWriter.BYTES_PER_DOC;
-    long skipped = 0;
-    while (skipped < seekFP) {
-      long inc = fis.skip(seekFP - skipped);
-      skipped += inc;
-      if (inc == 0) {
-        throw new RuntimeException("skip returned 0");
-      }
-    }
-    in = new InputStreamDataInput(new BufferedInputStream(fis));
+  OfflineSliceReader(Directory tempDir, String tempFileName, long start, long count) throws IOException {
+    in = tempDir.openInput(tempFileName, IOContext.READONCE);
+    in.seek(start * RangeTreeWriter.BYTES_PER_DOC);
     this.countLeft = count;
   }
 

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/OfflineSliceWriter.java Mon Oct 19 13:38:42 2015
@@ -17,50 +17,47 @@ package org.apache.lucene.rangetree;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.OfflineSorter;
-
-import java.io.BufferedOutputStream;
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
+
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
 
 final class OfflineSliceWriter implements SliceWriter {
 
-  final Path tempFile;
+  final Directory tempDir;
   final byte[] scratchBytes = new byte[RangeTreeWriter.BYTES_PER_DOC];
   final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);      
-  final OutputStreamDataOutput out;
+  final IndexOutput tempFile;
   final long count;
   private boolean closed;
   private long countWritten;
 
-  public OfflineSliceWriter(long count) throws IOException {
-    tempFile = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "size" + count + ".", "");
-    out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+  public OfflineSliceWriter(Directory tempDir, String tempFileNamePrefix, long count) throws IOException {
+    this.tempDir = tempDir;
+    tempFile = tempDir.createTempOutput(tempFileNamePrefix, "rangetree", IOContext.DEFAULT);
     this.count = count;
   }
     
   @Override
   public void append(long value, long ord, int docID) throws IOException {
-    out.writeLong(value);
-    out.writeLong(ord);
-    out.writeInt(docID);
+    tempFile.writeLong(value);
+    tempFile.writeLong(ord);
+    tempFile.writeInt(docID);
     countWritten++;
   }
 
   @Override
   public SliceReader getReader(long start) throws IOException {
     assert closed;
-    return new OfflineSliceReader(tempFile, start, count-start);
+    return new OfflineSliceReader(tempDir, tempFile.getName(), start, count-start);
   }
 
   @Override
   public void close() throws IOException {
     closed = true;
-    out.close();
+    tempFile.close();
     if (count != countWritten) {
       throw new IllegalStateException("wrote " + countWritten + " values, but expected " + count);
     }
@@ -68,12 +65,12 @@ final class OfflineSliceWriter implement
 
   @Override
   public void destroy() throws IOException {
-    IOUtils.rm(tempFile);
+    tempDir.deleteFile(tempFile.getName());
   }
 
   @Override
   public String toString() {
-    return "OfflineSliceWriter(count=" + count + " tempFile=" + tempFile + ")";
+    return "OfflineSliceWriter(count=" + count + " tempFileName=" + tempFile.getName() + ")";
   }
 }
 

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeDocValuesConsumer.java Mon Oct 19 13:38:42 2015
@@ -91,7 +91,7 @@ class RangeTreeDocValuesConsumer extends
   @Override
   public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
     delegate.addSortedNumericField(field, docToValueCount, values);
-    RangeTreeWriter writer = new RangeTreeWriter(maxPointsInLeafNode, maxPointsSortInHeap);
+    RangeTreeWriter writer = new RangeTreeWriter(state.directory, state.segmentInfo.name, maxPointsInLeafNode, maxPointsSortInHeap);
     Iterator<Number> valueIt = values.iterator();
     Iterator<Number> valueCountIt = docToValueCount.iterator();
     //System.out.println("\nSNF: field=" + field.name);
@@ -127,7 +127,7 @@ class RangeTreeDocValuesConsumer extends
   @Override
   public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
     delegate.addSortedSetField(field, values, docToOrdCount, ords);
-    RangeTreeWriter writer = new RangeTreeWriter(maxPointsInLeafNode, maxPointsSortInHeap);
+    RangeTreeWriter writer = new RangeTreeWriter(state.directory, state.segmentInfo.name, maxPointsInLeafNode, maxPointsSortInHeap);
     Iterator<Number> docToOrdCountIt = docToOrdCount.iterator();
     Iterator<Number> ordsIt = ords.iterator();
     //System.out.println("\nSSF: field=" + field.name);

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/java/org/apache/lucene/rangetree/RangeTreeWriter.java Mon Oct 19 13:38:42 2015
@@ -18,22 +18,21 @@ package org.apache.lucene.rangetree;
  */
 
 import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
-import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO
@@ -77,10 +76,13 @@ class RangeTreeWriter {
   private final byte[] scratchBytes = new byte[BYTES_PER_DOC];
   private final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);
 
-  private OfflineSorter.ByteSequencesWriter writer;
+  private final Directory tempDir;
+  private final String tempFileNamePrefix;
+
+  private OfflineSorter.ByteSequencesWriter offlineWriter;
   private GrowingHeapSliceWriter heapWriter;
 
-  private Path tempInput;
+  private IndexOutput tempInput;
   private final int maxValuesInLeafNode;
   private final int maxValuesSortInHeap;
 
@@ -88,13 +90,15 @@ class RangeTreeWriter {
   private long globalMinValue = Long.MAX_VALUE;
   private long globalMaxValue = Long.MIN_VALUE;
 
-  public RangeTreeWriter() throws IOException {
-    this(DEFAULT_MAX_VALUES_IN_LEAF_NODE, DEFAULT_MAX_VALUES_SORT_IN_HEAP);
+  public RangeTreeWriter(Directory tempDir, String tempFileNamePrefix) throws IOException {
+    this(tempDir, tempFileNamePrefix, DEFAULT_MAX_VALUES_IN_LEAF_NODE, DEFAULT_MAX_VALUES_SORT_IN_HEAP);
   }
 
   // TODO: instead of maxValuesSortInHeap, change to maxMBHeap ... the mapping is non-obvious:
-  public RangeTreeWriter(int maxValuesInLeafNode, int maxValuesSortInHeap) throws IOException {
+  public RangeTreeWriter(Directory tempDir, String tempFileNamePrefix, int maxValuesInLeafNode, int maxValuesSortInHeap) throws IOException {
     verifyParams(maxValuesInLeafNode, maxValuesSortInHeap);
+    this.tempDir = tempDir;
+    this.tempFileNamePrefix = tempFileNamePrefix;
     this.maxValuesInLeafNode = maxValuesInLeafNode;
     this.maxValuesSortInHeap = maxValuesSortInHeap;
 
@@ -121,15 +125,15 @@ class RangeTreeWriter {
   private void switchToOffline() throws IOException {
 
     // For each .add we just append to this input file, then in .finish we sort this input and resursively build the tree:
-    tempInput = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "in", "");
-    writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+    tempInput = tempDir.createTempOutput(tempFileNamePrefix, "rangetree", IOContext.DEFAULT);
+    offlineWriter = new OfflineSorter.ByteSequencesWriter(tempInput);
     for(int i=0;i<valueCount;i++) {
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeLong(heapWriter.values[i]);
       scratchBytesOutput.writeVInt(heapWriter.docIDs[i]);
       scratchBytesOutput.writeVLong(i);
       // TODO: can/should OfflineSorter optimize the fixed-width case?
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     }
 
     heapWriter = null;
@@ -137,14 +141,14 @@ class RangeTreeWriter {
 
   void add(long value, int docID) throws IOException {
     if (valueCount >= maxValuesSortInHeap) {
-      if (writer == null) {
+      if (offlineWriter == null) {
         switchToOffline();
       }
       scratchBytesOutput.reset(scratchBytes);
       scratchBytesOutput.writeLong(value);
       scratchBytesOutput.writeVInt(docID);
       scratchBytesOutput.writeVLong(valueCount);
-      writer.write(scratchBytes, 0, scratchBytes.length);
+      offlineWriter.write(scratchBytes, 0, scratchBytes.length);
     } else {
       // Not too many points added yet, continue using heap:
       heapWriter.append(value, valueCount, docID);
@@ -157,7 +161,7 @@ class RangeTreeWriter {
 
   /** Changes incoming {@link ByteSequencesWriter} file to to fixed-width-per-entry file, because we need to be able to slice
    *  as we recurse in {@link #build}. */
-  private SliceWriter convertToFixedWidth(Path in) throws IOException {
+  private SliceWriter convertToFixedWidth(String in) throws IOException {
     BytesRefBuilder scratch = new BytesRefBuilder();
     scratch.grow(BYTES_PER_DOC);
     BytesRef bytes = scratch.get();
@@ -167,7 +171,7 @@ class RangeTreeWriter {
     SliceWriter sortedWriter = null;
     boolean success = false;
     try {
-      reader = new OfflineSorter.ByteSequencesReader(in);
+      reader = new OfflineSorter.ByteSequencesReader(tempDir.openInput(in, IOContext.READONCE));
       sortedWriter = getWriter(valueCount);
       for (long i=0;i<valueCount;i++) {
         boolean result = reader.read(scratch);
@@ -280,19 +284,18 @@ class RangeTreeWriter {
         }
       };
 
-      Path sorted = Files.createTempFile(OfflineSorter.getDefaultTempDir(), "sorted", "");
       boolean success = false;
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp);
+      String sortedFileName = sorter.sort(tempInput.getName());
       try {
-        OfflineSorter sorter = new OfflineSorter(cmp);
-        sorter.sort(tempInput, sorted);
-        SliceWriter writer = convertToFixedWidth(sorted);
+        SliceWriter writer = convertToFixedWidth(sortedFileName);
         success = true;
         return writer;
       } finally {
         if (success) {
-          IOUtils.rm(sorted);
+          tempDir.deleteFile(sortedFileName);
         } else {
-          IOUtils.deleteFilesIgnoringExceptions(sorted);
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, sortedFileName);
         }
       }
     }
@@ -301,8 +304,8 @@ class RangeTreeWriter {
   /** Writes the 1d BKD tree to the provided {@link IndexOutput} and returns the file offset where index was written. */
   public long finish(IndexOutput out) throws IOException {
 
-    if (writer != null) {
-      writer.close();
+    if (offlineWriter != null) {
+      offlineWriter.close();
     }
 
     if (valueCount == 0) {
@@ -357,14 +360,18 @@ class RangeTreeWriter {
     } finally {
       if (success) {
         sortedWriter.destroy();
-        IOUtils.rm(tempInput);
+        if (tempInput != null) {
+          tempDir.deleteFile(tempInput.getName());
+        }
       } else {
         try {
           sortedWriter.destroy();
         } catch (Throwable t) {
           // Suppress to keep throwing original exc
         }
-        IOUtils.deleteFilesIgnoringExceptions(tempInput);
+        if (tempInput != null) {
+          IOUtils.deleteFilesIgnoringExceptions(tempDir, tempInput.getName());
+        }
       }
     }
 
@@ -567,7 +574,7 @@ class RangeTreeWriter {
     if (count < maxValuesSortInHeap) {
       return new HeapSliceWriter((int) count);
     } else {
-      return new OfflineSliceWriter(count);
+      return new OfflineSliceWriter(tempDir, tempFileNamePrefix, count);
     }
   }
 }

Modified: lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java?rev=1709414&r1=1709413&r2=1709414&view=diff
==============================================================================
--- lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (original)
+++ lucene/dev/branches/lucene6835/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java Mon Oct 19 13:38:42 2015
@@ -17,6 +17,15 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.lucene53.Lucene53Codec;
@@ -37,24 +46,16 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase.Nightly;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.Nightly;
 import org.apache.lucene.util.TestUtil;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 public class TestBKDTree extends LuceneTestCase {
 
   private static boolean smallBBox;
@@ -163,7 +164,7 @@ public class TestBKDTree extends LuceneT
     // Every doc has 2 points:
     double[] lats = new double[2*numPoints];
     double[] lons = new double[2*numPoints];
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     // We rely on docID order:
     iwc.setMergePolicy(newLogMergePolicy());
@@ -378,7 +379,7 @@ public class TestBKDTree extends LuceneT
     if (lats.length > 100000) {
       dir = newFSDirectory(createTempDir("TestBKDTree"));
     } else {
-      dir = newDirectory();
+      dir = getDirectory();
     }
     Set<Integer> deleted = new HashSet<>();
     // RandomIndexWriter is too slow here:
@@ -608,7 +609,7 @@ public class TestBKDTree extends LuceneT
   }
 
   public void testAccountableHasDelegate() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = getDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     Codec codec = TestUtil.alwaysDocValuesFormat(getDocValuesFormat());
     iwc.setCodec(codec);
@@ -632,4 +633,12 @@ public class TestBKDTree extends LuceneT
     int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
     return new BKDTreeDocValuesFormat(maxPointsInLeaf, maxPointsSortInHeap);
   }
+
+  private static Directory getDirectory() {     
+    Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+    return dir;
+  }
 }