You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2020/09/16 11:38:23 UTC

[lucene-solr] branch branch_8x updated (78b8a0a -> 9cd3af5)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a change to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 78b8a0a  LUCENE-9486: Use ByteBuffersDataOutput to collect data like on master.
     new c3bdc00  LUCENE-9510: Don't compress temporary stored fields and term vectors when index sorting is enabled. (#1874)
     new 9cd3af5  LUCENE-9525: Better handle small documents with Lucene87StoredFieldsFormat. (#1876)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 lucene/CHANGES.txt                                 |  3 ++
 .../codecs/lucene87/BugfixDeflater_JDK8252739.java | 13 +++--
 .../DeflateWithPresetDictCompressionMode.java      | 22 ++++----
 .../lucene87/LZ4WithPresetDictCompressionMode.java | 26 +++++-----
 .../lucene87/Lucene87StoredFieldsFormat.java       | 31 +++---------
 .../lucene/index/SortingStoredFieldsConsumer.java  | 59 ++++++++++++++++++----
 .../lucene/index/SortingTermVectorsConsumer.java   | 22 ++++----
 .../DeflateWithPresetCompressingCodec.java         |  2 +-
 .../compressing/LZ4WithPresetCompressingCodec.java |  2 +-
 9 files changed, 98 insertions(+), 82 deletions(-)


[lucene-solr] 02/02: LUCENE-9525: Better handle small documents with Lucene87StoredFieldsFormat. (#1876)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 9cd3af50f8093ddf9c70c90fa7cc8e1103ecabb7
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Wed Sep 16 13:09:00 2020 +0200

    LUCENE-9525: Better handle small documents with Lucene87StoredFieldsFormat. (#1876)
    
    Instead of configuring a dictionary size and a block size, the format
    now tries to have 10 sub blocks per bigger block, and adapts the size of
    the dictionary and of the sub blocks to this overall block size.
---
 .../codecs/lucene87/BugfixDeflater_JDK8252739.java | 13 +++++----
 .../DeflateWithPresetDictCompressionMode.java      | 22 +++++++--------
 .../lucene87/LZ4WithPresetDictCompressionMode.java | 26 +++++++++---------
 .../lucene87/Lucene87StoredFieldsFormat.java       | 31 +++++-----------------
 .../DeflateWithPresetCompressingCodec.java         |  2 +-
 .../compressing/LZ4WithPresetCompressingCodec.java |  2 +-
 6 files changed, 36 insertions(+), 60 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/BugfixDeflater_JDK8252739.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/BugfixDeflater_JDK8252739.java
index 8dc2152..f53e233 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/BugfixDeflater_JDK8252739.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/BugfixDeflater_JDK8252739.java
@@ -21,6 +21,7 @@ import java.util.zip.DataFormatException;
 import java.util.zip.Deflater;
 import java.util.zip.Inflater;
 
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.SuppressForbidden;
 
 /**
@@ -39,16 +40,14 @@ interface BugfixDeflater_JDK8252739 {
    * on a {@code Deflater}.
    * */
   @SuppressForbidden(reason = "Works around bug, so it must call forbidden method")
-  public static BugfixDeflater_JDK8252739 createBugfix(Deflater deflater, int dictLength) {
-    if (dictLength < 0) {
-      throw new IllegalArgumentException("dictLength must be >= 0");
-    }
+  public static BugfixDeflater_JDK8252739 createBugfix(Deflater deflater) {
     if (IS_BUGGY_JDK) {
-      final byte[] dictBytesScratch = new byte[dictLength];
+      final BytesRefBuilder dictBytesScratch = new BytesRefBuilder();
       return (dictBytes, off, len) -> {
         if (off > 0) {
-          System.arraycopy(dictBytes, off, dictBytesScratch, 0, len);
-          deflater.setDictionary(dictBytesScratch, 0, len);
+          dictBytesScratch.grow(len);
+          System.arraycopy(dictBytes, off, dictBytesScratch.bytes(), 0, len);
+          deflater.setDictionary(dictBytesScratch.bytes(), 0, len);
         } else {
           deflater.setDictionary(dictBytes, off, len);
         }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java
index 4269740..5b54f7d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java
@@ -39,20 +39,20 @@ import org.apache.lucene.util.BytesRef;
  */
 public final class DeflateWithPresetDictCompressionMode extends CompressionMode {
 
-  private final int dictLength, subBlockLength;
+  // Shoot for 10 sub blocks
+  private static final int NUM_SUB_BLOCKS = 10;
+  // And a dictionary whose size is about 6x smaller than sub blocks
+  private static final int DICT_SIZE_FACTOR = 6;
 
   /** Sole constructor. */
-  public DeflateWithPresetDictCompressionMode(int dictLength, int subBlockLength) {
-    this.dictLength = dictLength;
-    this.subBlockLength = subBlockLength;
-  }
+  public DeflateWithPresetDictCompressionMode() {}
 
   @Override
   public Compressor newCompressor() {
     // notes:
     // 3 is the highest level that doesn't have lazy match evaluation
     // 6 is the default, higher than that is just a waste of cpu
-    return new DeflateWithPresetDictCompressor(6, dictLength, subBlockLength);
+    return new DeflateWithPresetDictCompressor(6);
   }
 
   @Override
@@ -155,18 +155,15 @@ public final class DeflateWithPresetDictCompressionMode extends CompressionMode
 
   private static class DeflateWithPresetDictCompressor extends Compressor {
 
-    private final int dictLength, blockLength;
     final Deflater compressor;
     final BugfixDeflater_JDK8252739 deflaterBugfix;
     byte[] compressed;
     boolean closed;
 
-    DeflateWithPresetDictCompressor(int level, int dictLength, int blockLength) {
+    DeflateWithPresetDictCompressor(int level) {
       compressor = new Deflater(level, true);
-      deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor, dictLength);
+      deflaterBugfix = BugfixDeflater_JDK8252739.createBugfix(compressor);
       compressed = new byte[64];
-      this.dictLength = dictLength;
-      this.blockLength = blockLength;
     }
 
     private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
@@ -198,7 +195,8 @@ public final class DeflateWithPresetDictCompressionMode extends CompressionMode
 
     @Override
     public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
-      final int dictLength = Math.min(this.dictLength, len);
+      final int dictLength = len / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
+      final int blockLength = (len - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
       out.writeVInt(dictLength);
       out.writeVInt(blockLength);
       final int end = off + len;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java
index 0d10cfd..15b961d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java
@@ -36,17 +36,17 @@ import org.apache.lucene.util.compress.LZ4;
  */
 public final class LZ4WithPresetDictCompressionMode extends CompressionMode {
 
-  private final int dictLength, subBlockLength;
+  // Shoot for 10 sub blocks
+  private static final int NUM_SUB_BLOCKS = 10;
+  // And a dictionary whose size is about 16x smaller than sub blocks
+  private static final int DICT_SIZE_FACTOR = 16;
 
   /** Sole constructor. */
-  public LZ4WithPresetDictCompressionMode(int dictLength, int subBlockLength) {
-    this.dictLength = dictLength;
-    this.subBlockLength = subBlockLength;
-  }
+  public LZ4WithPresetDictCompressionMode() {}
 
   @Override
   public Compressor newCompressor() {
-    return new LZ4WithPresetDictCompressor(dictLength, subBlockLength);
+    return new LZ4WithPresetDictCompressor();
   }
 
   @Override
@@ -147,18 +147,14 @@ public final class LZ4WithPresetDictCompressionMode extends CompressionMode {
 
   private static class LZ4WithPresetDictCompressor extends Compressor {
 
-    final int dictLength;
-    final int blockLength;
     final ByteBuffersDataOutput compressed;
     final LZ4.FastCompressionHashTable hashTable;
-    final byte[] buffer;
+    byte[] buffer;
 
-    LZ4WithPresetDictCompressor(int dictLength, int blockLength) {
+    LZ4WithPresetDictCompressor() {
       compressed = ByteBuffersDataOutput.newResettableInstance();
       hashTable = new LZ4.FastCompressionHashTable();
-      this.dictLength = dictLength;
-      this.blockLength = blockLength;
-      buffer = new byte[dictLength + blockLength];
+      buffer = BytesRef.EMPTY_BYTES;
     }
 
     private void doCompress(byte[] bytes, int dictLen, int len, DataOutput out) throws IOException {
@@ -170,7 +166,9 @@ public final class LZ4WithPresetDictCompressionMode extends CompressionMode {
 
     @Override
     public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
-      final int dictLength = Math.min(this.dictLength, len);
+      final int dictLength = len / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
+      final int blockLength = (len - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
+      buffer = ArrayUtil.grow(buffer, dictLength + blockLength);
       out.writeVInt(dictLength);
       out.writeVInt(blockLength);
       final int end = off + len;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
index bf9a267..0e9c9f5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
@@ -151,35 +151,16 @@ public class Lucene87StoredFieldsFormat extends StoredFieldsFormat {
     }
   }
 
-  // 8kB seems to be a good trade-off between higher compression rates by not
-  // having to fully bootstrap a dictionary, and indexing rate by not spending
-  // too much CPU initializing data-structures to find strings in this preset
-  // dictionary.
-  private static final int BEST_COMPRESSION_DICT_LENGTH = 8 * 1024;
-  // 48kB seems like a nice trade-off because it's small enough to keep
-  // retrieval fast, yet sub blocks can find strings in a window of 26kB of
-  // data on average (the window grows from 8kB to 32kB in the first 24kB, and
-  // then DEFLATE can use 32kB for the last 24kB) which is close enough to the
-  // maximum window length of DEFLATE of 32kB.
-  private static final int BEST_COMPRESSION_SUB_BLOCK_LENGTH = 48 * 1024;
-  // We shoot for 10 sub blocks per block, which should hopefully amortize the
-  // space overhead of having the first 8kB compressed without any preset dict,
-  // and then remove 8kB in order to avoid creating a tiny 11th sub block if
-  // documents are small.
-  private static final int BEST_COMPRESSION_BLOCK_LENGTH = BEST_COMPRESSION_DICT_LENGTH + 10 * BEST_COMPRESSION_SUB_BLOCK_LENGTH - 8 * 1024;
+  // Shoot for 10 sub blocks of 48kB each.
+  private static final int BEST_COMPRESSION_BLOCK_LENGTH = 10 * 48 * 1024;
 
   /** Compression mode for {@link Mode#BEST_COMPRESSION} */
-  public static final CompressionMode BEST_COMPRESSION_MODE = new DeflateWithPresetDictCompressionMode(BEST_COMPRESSION_DICT_LENGTH, BEST_COMPRESSION_SUB_BLOCK_LENGTH);
+  public static final CompressionMode BEST_COMPRESSION_MODE = new DeflateWithPresetDictCompressionMode();
 
-  // We need to re-initialize the hash table for every sub block with the
-  // content of the dictionary, so we keep it small to not hurt indexing.
-  private static final int BEST_SPEED_DICT_LENGTH = 4 * 1024;
-  // 60kB so that dict_length + block_length == max window size
-  private static final int BEST_SPEED_SUB_BLOCK_LENGTH = 60 * 1024;
-  // shoot for 10 sub blocks in addition to the dictionary
-  private static final int BEST_SPEED_BLOCK_LENGTH = BEST_SPEED_DICT_LENGTH + 10 * BEST_SPEED_SUB_BLOCK_LENGTH - 8 * 1024;
+  // Shoot for 10 sub blocks of 60kB each.
+  private static final int BEST_SPEED_BLOCK_LENGTH = 10 * 60 * 1024;
 
   /** Compression mode for {@link Mode#BEST_SPEED} */
-  public static final CompressionMode BEST_SPEED_MODE = new LZ4WithPresetDictCompressionMode(BEST_SPEED_DICT_LENGTH, BEST_SPEED_SUB_BLOCK_LENGTH);
+  public static final CompressionMode BEST_SPEED_MODE = new LZ4WithPresetDictCompressionMode();
 
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
index b65d355..cf20279 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
@@ -25,7 +25,7 @@ public class DeflateWithPresetCompressingCodec extends CompressingCodec {
   public DeflateWithPresetCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
     super("DeflateWithPresetCompressingStoredFieldsData", 
           withSegmentSuffix ? "DeflateWithPresetCompressingStoredFields" : "",
-          new DeflateWithPresetDictCompressionMode(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
+          new DeflateWithPresetDictCompressionMode(), chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** No-arg constructor. */
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java
index ea413fc..690d26c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java
@@ -25,7 +25,7 @@ public class LZ4WithPresetCompressingCodec extends CompressingCodec {
   public LZ4WithPresetCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
     super("LZ4WithPresetCompressingStoredFieldsData", 
           withSegmentSuffix ? "DeflateWithPresetCompressingStoredFields" : "",
-          new LZ4WithPresetDictCompressionMode(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
+          new LZ4WithPresetDictCompressionMode(), chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** No-arg constructor. */


[lucene-solr] 01/02: LUCENE-9510: Don't compress temporary stored fields and term vectors when index sorting is enabled. (#1874)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit c3bdc006a292392ec5dffd57298426bf731c9887
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Wed Sep 16 13:05:22 2020 +0200

    LUCENE-9510: Don't compress temporary stored fields and term vectors when index sorting is enabled. (#1874)
    
    When index sorting is enabled, stored fields and term vectors can't be
    written on the fly like in the normal case, so they are written into
    temporary files that then get resorted. For these temporary files,
    disabling compression speeds up indexing significantly.
    
    On a synthetic test that indexes stored fields and a doc value field
    populated with random values that is used for index sorting, this
    resulted in a 3x indexing speedup.
---
 lucene/CHANGES.txt                                 |  3 ++
 .../lucene/index/SortingStoredFieldsConsumer.java  | 59 ++++++++++++++++++----
 .../lucene/index/SortingTermVectorsConsumer.java   | 22 ++++----
 3 files changed, 62 insertions(+), 22 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 2333661..03bcc00 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -69,6 +69,9 @@ Optimizations
 * LUCENE-9373: FunctionMatchQuery now accepts a "matchCost" optimization hint.
   (Maxim Glazkov, David Smiley)
 
+* LUCENE-9510: Indexing with an index sort is now faster by not compressing
+  temporary representations of the data. (Adrien Grand)
+
 Bug Fixes
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
index 4647a70..5878ee6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
@@ -20,20 +20,64 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.io.Reader;
 import java.nio.charset.StandardCharsets;
-import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Compressor;
+import org.apache.lucene.codecs.compressing.Decompressor;
 import org.apache.lucene.document.StoredField;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
+
+  static final CompressionMode NO_COMPRESSION = new CompressionMode() {
+    @Override
+    public Compressor newCompressor() {
+      return new Compressor() {
+        @Override
+        public void close() throws IOException {}
+
+        @Override
+        public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+          out.writeBytes(bytes, off, len);
+        }
+      };
+    }
+
+    @Override
+    public Decompressor newDecompressor() {
+      return new Decompressor() {
+        @Override
+        public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes)
+            throws IOException {
+          bytes.bytes = ArrayUtil.grow(bytes.bytes, length);
+          in.skipBytes(offset);
+          in.readBytes(bytes.bytes, 0, length);
+          bytes.offset = 0;
+          bytes.length = length;
+        }
+
+        @Override
+        public Decompressor clone() {
+          return this;
+        }
+      };
+    }
+  };
+  private static final StoredFieldsFormat TEMP_STORED_FIELDS_FORMAT = new CompressingStoredFieldsFormat(
+      "TempStoredFields", NO_COMPRESSION, 128*1024, 1, 10);
   TrackingTmpOutputDirectoryWrapper tmpDirectory;
 
   SortingStoredFieldsConsumer(Codec codec, Directory directory, SegmentInfo info) {
@@ -44,21 +88,14 @@ final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
   protected void initStoredFieldsWriter() throws IOException {
     if (writer == null) {
       this.tmpDirectory = new TrackingTmpOutputDirectoryWrapper(directory);
-      this.writer = codec.storedFieldsFormat().fieldsWriter(tmpDirectory, info, IOContext.DEFAULT);
+      this.writer = TEMP_STORED_FIELDS_FORMAT.fieldsWriter(tmpDirectory, info, IOContext.DEFAULT);
     }
   }
 
   @Override
   void flush(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
     super.flush(state, sortMap);
-    if (sortMap == null) {
-      // we're lucky the index is already sorted, just rename the temporary file and return
-      for (Map.Entry<String, String> entry : tmpDirectory.getTemporaryFiles().entrySet()) {
-        tmpDirectory.rename(entry.getValue(), entry.getKey());
-      }
-      return;
-    }
-    StoredFieldsReader reader = codec.storedFieldsFormat()
+    StoredFieldsReader reader = TEMP_STORED_FIELDS_FORMAT
         .fieldsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT);
     // Don't pull a merge instance, since merge instances optimize for
     // sequential access while we consume stored fields in random order here.
@@ -69,7 +106,7 @@ final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
       CopyVisitor visitor = new CopyVisitor(sortWriter);
       for (int docID = 0; docID < state.segmentInfo.maxDoc(); docID++) {
         sortWriter.startDocument();
-        reader.visitDocument(sortMap.newToOld(docID), visitor);
+        reader.visitDocument(sortMap == null ? docID : sortMap.newToOld(docID), visitor);
         sortWriter.finishDocument();
       }
       sortWriter.finish(state.fieldInfos, state.segmentInfo.maxDoc());
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
index 5162f39..0499081 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
@@ -23,8 +23,10 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.compressing.CompressingTermVectorsFormat;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
@@ -35,6 +37,9 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntBlockPool;
 
 final class SortingTermVectorsConsumer extends TermVectorsConsumer {
+
+  private static final TermVectorsFormat TEMP_TERM_VECTORS_FORMAT = new CompressingTermVectorsFormat(
+      "TempTermVectors", "", SortingStoredFieldsConsumer.NO_COMPRESSION, 8*1024, 10);
   TrackingTmpOutputDirectoryWrapper tmpDirectory;
 
   SortingTermVectorsConsumer(final IntBlockPool.Allocator intBlockAllocator, final ByteBlockPool.Allocator byteBlockAllocator, Directory directory, SegmentInfo info, Codec codec) {
@@ -45,22 +50,17 @@ final class SortingTermVectorsConsumer extends TermVectorsConsumer {
   void flush(Map<String, TermsHashPerField> fieldsToFlush, final SegmentWriteState state, Sorter.DocMap sortMap, NormsProducer norms) throws IOException {
     super.flush(fieldsToFlush, state, sortMap, norms);
     if (tmpDirectory != null) {
-      if (sortMap == null) {
-        // we're lucky the index is already sorted, just rename the temporary file and return
-        for (Map.Entry<String, String> entry : tmpDirectory.getTemporaryFiles().entrySet()) {
-          tmpDirectory.rename(entry.getValue(), entry.getKey());
-        }
-        return;
-      }
-      TermVectorsReader reader = codec.termVectorsFormat()
+      TermVectorsReader reader = TEMP_TERM_VECTORS_FORMAT
           .vectorsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT);
-      TermVectorsReader mergeReader = reader.getMergeInstance();
+      // Don't pull a merge instance, since merge instances optimize for
+      // sequential access while term vectors will likely be accessed in random
+      // order here.
       TermVectorsWriter writer = codec.termVectorsFormat()
           .vectorsWriter(state.directory, state.segmentInfo, IOContext.DEFAULT);
       try {
         reader.checkIntegrity();
         for (int docID = 0; docID < state.segmentInfo.maxDoc(); docID++) {
-          Fields vectors = mergeReader.get(sortMap.newToOld(docID));
+          Fields vectors = reader.get(sortMap == null ? docID : sortMap.newToOld(docID));
           writeTermVectors(writer, vectors, state.fieldInfos);
         }
         writer.finish(state.fieldInfos, state.segmentInfo.maxDoc());
@@ -77,7 +77,7 @@ final class SortingTermVectorsConsumer extends TermVectorsConsumer {
     if (writer == null) {
       IOContext context = new IOContext(new FlushInfo(lastDocID, bytesUsed.get()));
       tmpDirectory = new TrackingTmpOutputDirectoryWrapper(directory);
-      writer = codec.termVectorsFormat().vectorsWriter(tmpDirectory, info, context);
+      writer = TEMP_TERM_VECTORS_FORMAT.vectorsWriter(tmpDirectory, info, context);
       lastDocID = 0;
     }
   }