You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/09/26 11:50:09 UTC

[38/45] lucene-solr:jira/solr-12709: LUCENE-8505: IndexWriter#addIndices will now fail if the target index is sorted but the candidate is not.

LUCENE-8505: IndexWriter#addIndices will now fail if the target index is sorted but the candidate is not.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2bad3c49
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2bad3c49
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2bad3c49

Branch: refs/heads/jira/solr-12709
Commit: 2bad3c498517120c8aaaf805fd49a5cb459417c0
Parents: e437b2f
Author: Jim Ferenczi <ji...@apache.org>
Authored: Tue Sep 25 09:14:07 2018 +0200
Committer: Jim Ferenczi <ji...@apache.org>
Committed: Tue Sep 25 09:14:07 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 ++
 .../org/apache/lucene/index/IndexWriter.java    | 27 +++++++----
 .../org/apache/lucene/index/MergeState.java     | 44 ++----------------
 .../apache/lucene/index/TestIndexSorting.java   | 49 ++++++++++++++++++--
 4 files changed, 70 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2bad3c49/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d305759..60afb58 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -117,6 +117,9 @@ Changes in Runtime Behavior
   total hit counts accurately up to 1,000 in order to enable top-hits
   optimizations such as block-max WAND (LUCENE-8135). (Adrien Grand)
 
+* LUCENE-8505: IndexWriter#addIndices will now fail if the target index is sorted but
+  the candidate is not. (Jim Ferenczi)
+
 New Features
 
 * LUCENE-8340: LongPoint#newDistanceQuery may be used to boost scores based on

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2bad3c49/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 5affd85..db6ef9f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -53,6 +53,7 @@ import org.apache.lucene.search.DocValuesFieldExistsQuery;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
@@ -934,21 +935,30 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
   }
 
   /** Confirms that the incoming index sort (if any) matches the existing index sort (if any).  */
-  private void validateIndexSort() throws CorruptIndexException {
+  private void validateIndexSort() {
     Sort indexSort = config.getIndexSort();
     if (indexSort != null) {
       for(SegmentCommitInfo info : segmentInfos) {
         Sort segmentIndexSort = info.info.getIndexSort();
-        if (segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
+        if (segmentIndexSort == null || isCongruentSort(indexSort, segmentIndexSort) == false) {
           throw new IllegalArgumentException("cannot change previous indexSort=" + segmentIndexSort + " (from segment=" + info + ") to new indexSort=" + indexSort);
-        } else if (segmentIndexSort == null) {
-          // Flushed segments are not sorted if they were built with a version prior to 6.5.0
-          throw new CorruptIndexException("segment not sorted with indexSort=" + segmentIndexSort, info.info.toString());
         }
       }
     }
   }
 
+  /**
+   * Returns true if <code>indexSort</code> is a prefix of <code>otherSort</code>.
+   **/
+  static boolean isCongruentSort(Sort indexSort, Sort otherSort) {
+    final SortField[] fields1 = indexSort.getSort();
+    final SortField[] fields2 = otherSort.getSort();
+    if (fields1.length > fields2.length) {
+      return false;
+    }
+    return Arrays.asList(fields1).equals(Arrays.asList(fields2).subList(0, fields1.length));
+  }
+
   // reads latest field infos for the commit
   // this is used on IW init and addIndexes(Dir) to create/update the global field map.
   // TODO: fix tests abusing this method!
@@ -2824,8 +2834,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
             Sort segmentIndexSort = info.info.getIndexSort();
 
-            if (indexSort != null && segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
-              // TODO: we could make this smarter, e.g. if the incoming indexSort is congruent with our sort ("starts with") then it's OK
+            if (indexSort != null && (segmentIndexSort == null || isCongruentSort(indexSort, segmentIndexSort) == false)) {
               throw new IllegalArgumentException("cannot change index sort from " + segmentIndexSort + " to " + indexSort);
             }
 
@@ -2908,8 +2917,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     }
 
     Sort leafIndexSort = segmentMeta.getSort();
-    if (config.getIndexSort() != null && leafIndexSort != null
-        && config.getIndexSort().equals(leafIndexSort) == false) {
+    if (config.getIndexSort() != null &&
+          (leafIndexSort == null || isCongruentSort(config.getIndexSort(), leafIndexSort) == false)) {
       throw new IllegalArgumentException("cannot change index sort from " + leafIndexSort + " to " + config.getIndexSort());
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2bad3c49/lucene/core/src/java/org/apache/lucene/index/MergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
index 9ad69f6..0e47a5a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
@@ -34,6 +34,8 @@ import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
+import static org.apache.lucene.index.IndexWriter.isCongruentSort;
+
 /** Holds common state used during segment merging.
  *
  * @lucene.experimental */
@@ -223,50 +225,14 @@ public class MergeState {
       return originalReaders;
     }
 
-    /** If an incoming reader is not sorted, because it was flushed by IW older than {@link Version.LUCENE_7_0_0}
-     * or because we add unsorted segments from another index {@link IndexWriter#addIndexes(CodecReader...)} ,
-     * we sort it here:
-     */
-    final Sorter sorter = new Sorter(indexSort);
     List<CodecReader> readers = new ArrayList<>(originalReaders.size());
 
     for (CodecReader leaf : originalReaders) {
       Sort segmentSort = leaf.getMetaData().getSort();
-
-      if (segmentSort == null) {
-        // This segment was written by flush, so documents are not yet sorted, so we sort them now:
-        long t0 = System.nanoTime();
-        Sorter.DocMap sortDocMap = sorter.sort(leaf);
-        long t1 = System.nanoTime();
-        double msec = (t1-t0)/1000000.0;
-        
-        if (sortDocMap != null) {
-          if (infoStream.isEnabled("SM")) {
-            infoStream.message("SM", String.format(Locale.ROOT, "segment %s is not sorted; wrapping for sort %s now (%.2f msec to sort)", leaf, indexSort, msec));
-          }
-          needsIndexSort = true;
-          leaf = SlowCodecReaderWrapper.wrap(SortingLeafReader.wrap(new MergeReaderWrapper(leaf), sortDocMap));
-          leafDocMaps[readers.size()] = new DocMap() {
-              @Override
-              public int get(int docID) {
-                return sortDocMap.oldToNew(docID);
-              }
-            };
-        } else {
-          if (infoStream.isEnabled("SM")) {
-            infoStream.message("SM", String.format(Locale.ROOT, "segment %s is not sorted, but is already accidentally in sort %s order (%.2f msec to sort)", leaf, indexSort, msec));
-          }
-        }
-
-      } else {
-        if (segmentSort.equals(indexSort) == false) {
-          throw new IllegalArgumentException("index sort mismatch: merged segment has sort=" + indexSort + " but to-be-merged segment has sort=" + segmentSort);
-        }
-        if (infoStream.isEnabled("SM")) {
-          infoStream.message("SM", "segment " + leaf + " already sorted");
-        }
+      if (segmentSort == null || isCongruentSort(indexSort, segmentSort) == false) {
+        throw new IllegalArgumentException("index sort mismatch: merged segment has sort=" + indexSort +
+            " but to-be-merged segment has sort=" + (segmentSort == null ? "null" : segmentSort));
       }
-
       readers.add(leaf);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2bad3c49/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 3e62aad..3857a97 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -1820,19 +1821,52 @@ public class TestIndexSorting extends LuceneTestCase {
     dir.close();
   }
 
-  public void testAddIndexes(boolean withDeletes, boolean useReaders) throws Exception {
+  public void testBadAddIndexes() throws Exception {
     Directory dir = newDirectory();
     Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
     IndexWriterConfig iwc1 = newIndexWriterConfig();
-    if (random().nextBoolean()) {
-      iwc1.setIndexSort(indexSort);
+    iwc1.setIndexSort(indexSort);
+    IndexWriter w = new IndexWriter(dir, iwc1);
+    w.addDocument(new Document());
+    List<Sort> indexSorts = Arrays.asList(null, new Sort(new SortField("bar", SortField.Type.LONG)));
+    for (Sort sort : indexSorts) {
+      Directory dir2 = newDirectory();
+      IndexWriterConfig iwc2 = newIndexWriterConfig();
+      if (sort != null) {
+        iwc2.setIndexSort(sort);
+      }
+      IndexWriter w2 = new IndexWriter(dir2, iwc2);
+      w2.addDocument(new Document());
+      final IndexReader reader = w2.getReader();
+      w2.close();
+      IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> w.addIndexes(dir2));
+      assertThat(expected.getMessage(), containsString("cannot change index sort"));
+      CodecReader[] codecReaders = new CodecReader[reader.leaves().size()];
+      for (int i = 0; i < codecReaders.length; ++i) {
+        codecReaders[i] = (CodecReader) reader.leaves().get(i).reader();
+      }
+      expected = expectThrows(IllegalArgumentException.class, () -> w.addIndexes(codecReaders));
+      assertThat(expected.getMessage(), containsString("cannot change index sort"));
+
+      reader.close();
+      dir2.close();
     }
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    w.close();
+    dir.close();
+  }
+
+  public void testAddIndexes(boolean withDeletes, boolean useReaders) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc1 = newIndexWriterConfig();
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG), new SortField("bar", SortField.Type.LONG));
+    iwc1.setIndexSort(indexSort);
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc1);
     final int numDocs = atLeast(100);
     for (int i = 0; i < numDocs; ++i) {
       Document doc = new Document();
       doc.add(new StringField("id", Integer.toString(i), Store.NO));
       doc.add(new NumericDocValuesField("foo", random().nextInt(20)));
+      doc.add(new NumericDocValuesField("bar", random().nextInt(20)));
       w.addDocument(doc);
     }
     if (withDeletes) {
@@ -1848,7 +1882,12 @@ public class TestIndexSorting extends LuceneTestCase {
 
     Directory dir2 = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setIndexSort(indexSort);
+    if (indexSort != null && random().nextBoolean()) {
+      // test congruent index sort
+      iwc.setIndexSort(new Sort(new SortField("foo", SortField.Type.LONG)));
+    } else {
+      iwc.setIndexSort(indexSort);
+    }
     IndexWriter w2 = new IndexWriter(dir2, iwc);
 
     if (useReaders) {