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/04/17 10:05:25 UTC

[1/9] lucene-solr:jira/solr-11833: LUCENE-8253: Account for soft-deletes before they are flushed to disk

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-11833 96c2edb51 -> 2e4ac772e


LUCENE-8253: Account for soft-deletes before they are flushed to disk

Inside the IndexWriter buffers are only written to disk if it's needed
or "worth it" which doesn't guarantee soft deletes to be accounted
in time. This is not necessarily a problem since they are eventually
collected and segments that have soft-deletes will me merged eventually
but for tests and on par behavior compared to hard deletes this behavior
is tricky.
This change cuts over to accounting in-place just like hard-deletes. This
results in accurate delete numbers for soft deletes at any give point in time
once the reader is loaded or a pending soft delete occurs.

This change also fixes an issue where all updates to a DV field are allowed
event if the field is unknown. Now this only works if the field is equal
to the soft deletes field. This behavior was never released.


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

Branch: refs/heads/jira/solr-11833
Commit: c70cceaee56cecf35875cd2b5c8d5700f2b3cedb
Parents: 1e759bc
Author: Simon Willnauer <si...@apache.org>
Authored: Mon Apr 16 16:16:43 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Mon Apr 16 16:17:06 2018 +0200

----------------------------------------------------------------------
 .../org/apache/lucene/index/IndexWriter.java    | 28 ++++----
 .../org/apache/lucene/index/PendingDeletes.java | 21 ++++--
 .../apache/lucene/index/PendingSoftDeletes.java | 75 ++++++++++++++++----
 .../apache/lucene/index/ReadersAndUpdates.java  | 10 +--
 .../index/SoftDeletesRetentionMergePolicy.java  | 22 +++---
 .../lucene/index/TestPendingSoftDeletes.java    | 19 +++--
 .../TestSoftDeletesDirectoryReaderWrapper.java  | 32 +++++++++
 .../apache/lucene/search/TestLRUQueryCache.java | 46 ++++++++++++
 .../org/apache/lucene/util/LuceneTestCase.java  |  6 +-
 9 files changed, 206 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/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 e973b91..e8f3e13 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1594,7 +1594,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     if (softDeletes == null || softDeletes.length == 0) {
       throw new IllegalArgumentException("at least one soft delete must be present");
     }
-    return updateDocuments(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes, false)), docs);
+    return updateDocuments(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes)), docs);
   }
 
   /** Expert: attempts to delete by document ID, as long as
@@ -1831,7 +1831,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     if (softDeletes == null || softDeletes.length == 0) {
       throw new IllegalArgumentException("at least one soft delete must be present");
     }
-    return updateDocument(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes, false)), doc);
+    return updateDocument(DocumentsWriterDeleteQueue.newNode(buildDocValuesUpdate(term, softDeletes)), doc);
   }
 
 
@@ -1940,7 +1940,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    */
   public long updateDocValues(Term term, Field... updates) throws IOException {
     ensureOpen();
-    DocValuesUpdate[] dvUpdates = buildDocValuesUpdate(term, updates, true);
+    DocValuesUpdate[] dvUpdates = buildDocValuesUpdate(term, updates);
     try {
       long seqNo = docWriter.updateDocValues(dvUpdates);
       if (seqNo < 0) {
@@ -1954,7 +1954,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     }
   }
 
-  private DocValuesUpdate[] buildDocValuesUpdate(Term term, Field[] updates, boolean enforceFieldExistence) {
+  private DocValuesUpdate[] buildDocValuesUpdate(Term term, Field[] updates) {
     DocValuesUpdate[] dvUpdates = new DocValuesUpdate[updates.length];
     for (int i = 0; i < updates.length; i++) {
       final Field f = updates[i];
@@ -1965,7 +1965,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       if (dvType == DocValuesType.NONE) {
         throw new IllegalArgumentException("can only update NUMERIC or BINARY fields! field=" + f.name());
       }
-      if (enforceFieldExistence && !globalFieldNumberMap.contains(f.name(), dvType)) {
+      if (!globalFieldNumberMap.contains(f.name(), dvType) && f.name().equals(config.softDeletesField) == false) {
         throw new IllegalArgumentException("can only update existing docvalues fields! field=" + f.name() + ", type=" + dvType);
       }
       if (config.getIndexSortFields().contains(f.name())) {
@@ -5230,15 +5230,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    */
   public final int numDeletesToMerge(SegmentCommitInfo info) throws IOException {
     MergePolicy mergePolicy = config.getMergePolicy();
-    final ReadersAndUpdates rld = readerPool.get(info, false);
-    int numDeletesToMerge;
-    if (rld != null) {
-      numDeletesToMerge = rld.numDeletesToMerge(mergePolicy);
-    } else {
-      numDeletesToMerge = mergePolicy.numDeletesToMerge(info,  0, null);
+    final ReadersAndUpdates rld = readerPool.get(info, true);
+    try {
+      int numDeletesToMerge = rld.numDeletesToMerge(mergePolicy);
+      assert numDeletesToMerge <= info.info.maxDoc() :
+          "numDeletesToMerge: " + numDeletesToMerge + " > maxDoc: " + info.info.maxDoc();
+      return numDeletesToMerge;
+    } finally {
+      readerPool.release(rld);
     }
-    assert numDeletesToMerge <= info.info.maxDoc() :
-    "numDeletesToMerge: " + numDeletesToMerge + " > maxDoc: " + info.info.maxDoc();
-    return numDeletesToMerge;
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
index 52d06e8..c0aed38 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -18,7 +18,6 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
-import java.util.List;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.LiveDocsFormat;
@@ -26,6 +25,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOSupplier;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MutableBits;
 
@@ -135,7 +135,7 @@ class PendingDeletes {
   /**
    * Called once a new reader is opened for this segment ie. when deletes or updates are applied.
    */
-  void onNewReader(SegmentReader reader, SegmentCommitInfo info) throws IOException {
+  void onNewReader(CodecReader reader, SegmentCommitInfo info) throws IOException {
     if (liveDocsInitialized == false) {
       if (reader.hasDeletions()) {
         // we only initialize this once either in the ctor or here
@@ -235,10 +235,21 @@ class PendingDeletes {
   }
 
   /**
-   * Called before the given DocValuesFieldUpdates are applied
+   * Called before the given DocValuesFieldUpdates are written to disk
    * @param info the field to apply
-   * @param fieldUpdates the field updates
    */
-  void onDocValuesUpdate(FieldInfo info, List<DocValuesFieldUpdates> fieldUpdates) throws IOException {
+  void onDocValuesUpdate(FieldInfo info) {
+  }
+
+  /**
+   * Called for every field update for the given field
+   * @param field the field that's updated
+   * @param iterator the values to apply
+   */
+  void onDocValuesUpdate(String field, DocValuesFieldUpdates.Iterator iterator) throws IOException {
+  }
+
+  int numDeletesToMerge(MergePolicy policy, IOSupplier<CodecReader> readerIOSupplier) throws IOException {
+    return policy.numDeletesToMerge(info, numPendingDeletes(), readerIOSupplier);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
index 4c3db48..3dca782 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -17,11 +17,16 @@
 
 package org.apache.lucene.index;
 
+import java.io.Closeable;
 import java.io.IOException;
-import java.util.List;
+
+import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DocValuesFieldExistsQuery;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.IOSupplier;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MutableBits;
 
 final class PendingSoftDeletes extends PendingDeletes {
@@ -64,14 +69,12 @@ final class PendingSoftDeletes extends PendingDeletes {
   }
 
   @Override
-  void onNewReader(SegmentReader reader, SegmentCommitInfo info) throws IOException {
+  void onNewReader(CodecReader reader, SegmentCommitInfo info) throws IOException {
     super.onNewReader(reader, info);
     hardDeletes.onNewReader(reader, info);
     if (dvGeneration < info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
       final DocIdSetIterator iterator = DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(field, reader);
-      if (iterator == null) { // nothing is deleted we don't have a soft deletes field in this segment
-        this.pendingDeleteCount = 0;
-      } else {
+      if (iterator != null) { // nothing is deleted we don't have a soft deletes field in this segment
         assert info.info.maxDoc() > 0 : "maxDoc is 0";
         pendingDeleteCount += applySoftDeletes(iterator, getMutableBits());
       }
@@ -117,15 +120,8 @@ final class PendingSoftDeletes extends PendingDeletes {
   }
 
   @Override
-  void onDocValuesUpdate(FieldInfo info, List<DocValuesFieldUpdates> updatesToApply) throws IOException {
-    if (field.equals(info.name)) {
-      assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
-      assert dvGeneration != -2 : "docValues generation is still uninitialized";
-      DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
-      for(int i=0; i<subs.length; i++) {
-        subs[i] = updatesToApply.get(i).iterator();
-      }
-      DocValuesFieldUpdates.Iterator iterator = DocValuesFieldUpdates.mergedIterator(subs);
+  void onDocValuesUpdate(String field, DocValuesFieldUpdates.Iterator iterator) throws IOException {
+    if (this.field.equals(field)) {
       pendingDeleteCount += applySoftDeletes(new DocIdSetIterator() {
         int docID = -1;
         @Override
@@ -148,6 +144,13 @@ final class PendingSoftDeletes extends PendingDeletes {
           throw new UnsupportedOperationException();
         }
       }, getMutableBits());
+    }
+  }
+  @Override
+  void onDocValuesUpdate(FieldInfo info) {
+    if (field.equals(info.name)) {
+      assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
+      assert dvGeneration != -2 : "docValues generation is still uninitialized";
       dvGeneration = info.getDocValuesGen();
     }
   }
@@ -162,4 +165,48 @@ final class PendingSoftDeletes extends PendingDeletes {
     sb.append(" hardDeletes=").append(hardDeletes);
     return sb.toString();
   }
+
+  @Override
+  int numDeletesToMerge(MergePolicy policy, IOSupplier<CodecReader> readerIOSupplier) throws IOException {
+    if (dvGeneration == -2) {
+      FieldInfos fieldInfos = readFieldInfos();
+      FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+      // we try to only open a reader if it's really necessary ie. indices that are mainly append only might have
+      // big segments that don't even have any docs in the soft deletes field. In such a case it's simply
+      // enough to look at the FieldInfo for the field and check if the field has DocValues
+      if (fieldInfo != null && fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+        // in order to get accurate numbers we need to have a least one reader see here.
+        onNewReader(readerIOSupplier.get(), info);
+      } else {
+        // we are safe here since we don't have any doc values for the soft-delete field on disk
+        // no need to open a new reader
+        dvGeneration = fieldInfo == null ? -1 : fieldInfo.getDocValuesGen();
+      }
+    }
+    return super.numDeletesToMerge(policy, readerIOSupplier);
+  }
+
+  private FieldInfos readFieldInfos() throws IOException {
+    SegmentInfo segInfo = info.info;
+    Directory dir = segInfo.dir;
+    if (info.hasFieldUpdates() == false) {
+      // updates always outside of CFS
+      Closeable toClose;
+      if (segInfo.getUseCompoundFile()) {
+        toClose = dir = segInfo.getCodec().compoundFormat().getCompoundReader(segInfo.dir, segInfo, IOContext.READONCE);
+      } else {
+        toClose = null;
+        dir = segInfo.dir;
+      }
+      try {
+        return segInfo.getCodec().fieldInfosFormat().read(dir, segInfo, "", IOContext.READONCE);
+      } finally {
+        IOUtils.close(toClose);
+      }
+    } else {
+      FieldInfosFormat fisFormat = segInfo.getCodec().fieldInfosFormat();
+      final String segmentSuffix = Long.toString(info.getFieldInfosGen(), Character.MAX_RADIX);
+      return fisFormat.read(dir, segInfo, segmentSuffix, IOContext.READONCE);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
index b236bea..2543721 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -142,7 +142,7 @@ final class ReadersAndUpdates {
 
   /** Adds a new resolved (meaning it maps docIDs to new values) doc values packet.  We buffer these in RAM and write to disk when too much
    *  RAM is used or when a merge needs to kick off, or a commit/refresh. */
-  public synchronized void addDVUpdate(DocValuesFieldUpdates update) {
+  public synchronized void addDVUpdate(DocValuesFieldUpdates update) throws IOException {
     if (update.getFinished() == false) {
       throw new IllegalArgumentException("call finish first");
     }
@@ -166,6 +166,7 @@ final class ReadersAndUpdates {
       }
       fieldUpdates.add(update);
     }
+    pendingDeletes.onDocValuesUpdate(update.field, update.iterator());
   }
 
   public synchronized long getNumDVUpdates() {
@@ -271,10 +272,9 @@ final class ReadersAndUpdates {
       }
       return reader;
     };
-    return policy.numDeletesToMerge(info, pendingDeletes.numPendingDeletes(), readerSupplier);
+    return pendingDeletes.numDeletesToMerge(policy, readerSupplier);
   }
 
-
   public synchronized Bits getLiveDocs() {
     return pendingDeletes.getLiveDocs();
   }
@@ -344,7 +344,7 @@ final class ReadersAndUpdates {
       final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
       final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
       try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
-        pendingDeletes.onDocValuesUpdate(fieldInfo, updatesToApply);
+        pendingDeletes.onDocValuesUpdate(fieldInfo);
         // write the numeric updates to a new gen'd docvalues file
         fieldsConsumer.addNumericField(fieldInfo, new EmptyDocValuesProducer() {
             @Override
@@ -480,7 +480,7 @@ final class ReadersAndUpdates {
       final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
       try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
         // write the binary updates to a new gen'd docvalues file
-        pendingDeletes.onDocValuesUpdate(fieldInfo, updatesToApply);
+        pendingDeletes.onDocValuesUpdate(fieldInfo);
         fieldsConsumer.addBinaryField(fieldInfo, new EmptyDocValuesProducer() {
             @Override
             public BinaryDocValues getBinary(FieldInfo fieldInfoIn) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
index 51578f9..8538775 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java
@@ -72,7 +72,8 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
 
   @Override
   public boolean keepFullyDeletedSegment(CodecReader reader) throws IOException {
-    Scorer scorer = getScorer(field, retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc()));
+    /* we only need a single hit to keep it no need for soft deletes to be checked*/
+    Scorer scorer = getScorer(retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc()));
     if (scorer != null) {
       DocIdSetIterator iterator = scorer.iterator();
       boolean atLeastOneHit = iterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
@@ -98,7 +99,10 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
         return liveDocs.length();
       }
     }, reader.maxDoc() - reader.numDocs());
-    Scorer scorer = getScorer(softDeleteField, retentionQuery, wrappedReader);
+    BooleanQuery.Builder builder = new BooleanQuery.Builder();
+    builder.add(new DocValuesFieldExistsQuery(softDeleteField), BooleanClause.Occur.FILTER);
+    builder.add(retentionQuery, BooleanClause.Occur.FILTER);
+    Scorer scorer = getScorer(builder.build(), wrappedReader);
     if (scorer != null) {
       FixedBitSet cloneLiveDocs = cloneLiveDocs(liveDocs);
       DocIdSetIterator iterator = scorer.iterator();
@@ -133,13 +137,10 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
     }
   }
 
-  private static Scorer getScorer(String softDeleteField, Query retentionQuery, CodecReader reader) throws IOException {
-    BooleanQuery.Builder builder = new BooleanQuery.Builder();
-    builder.add(new DocValuesFieldExistsQuery(softDeleteField), BooleanClause.Occur.FILTER);
-    builder.add(retentionQuery, BooleanClause.Occur.FILTER);
+  private static Scorer getScorer(Query query, CodecReader reader) throws IOException {
     IndexSearcher s = new IndexSearcher(reader);
     s.setQueryCache(null);
-    Weight weight = s.createWeight(builder.build(), ScoreMode.COMPLETE_NO_SCORES, 1.0f);
+    Weight weight = s.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1.0f);
     return weight.scorer(reader.getContext());
   }
 
@@ -172,11 +173,14 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
 
   @Override
   public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
-    int numDeletesToMerge = super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
+    final int numDeletesToMerge = super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
     if (numDeletesToMerge != 0) {
       final CodecReader reader = readerSupplier.get();
       if (reader.getLiveDocs() != null) {
-        Scorer scorer = getScorer(field, retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc()));
+        BooleanQuery.Builder builder = new BooleanQuery.Builder();
+        builder.add(new DocValuesFieldExistsQuery(field), BooleanClause.Occur.FILTER);
+        builder.add(retentionQuerySupplier.get(), BooleanClause.Occur.FILTER);
+        Scorer scorer = getScorer(builder.build(), wrapLiveDocs(reader, null, reader.maxDoc()));
         if (scorer != null) {
           DocIdSetIterator iterator = scorer.iterator();
           Bits liveDocs = reader.getLiveDocs();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
index 255ff9e..eac4388 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
@@ -45,7 +45,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
 
   public void testDeleteSoft() throws IOException {
     Directory dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field here
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig().setSoftDeletesField("_soft_deletes"));
     Document doc = new Document();
     doc.add(new StringField("id", "1", Field.Store.YES));
     writer.softUpdateDocument(new Term("id", "1"), doc,
@@ -114,7 +114,10 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0);
     List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10));
-    deletes.onDocValuesUpdate(fieldInfo, updates);
+    for (DocValuesFieldUpdates update : updates) {
+      deletes.onDocValuesUpdate(update.field, update.iterator());
+    }
+    deletes.onDocValuesUpdate(fieldInfo);
     assertEquals(4, deletes.numPendingDeletes());
     assertTrue(deletes.getLiveDocs().get(0));
     assertFalse(deletes.getLiveDocs().get(1));
@@ -130,7 +133,10 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS);
     updates = Arrays.asList(singleUpdate(docsDeleted, 10));
     fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0);
-    deletes.onDocValuesUpdate(fieldInfo, updates);
+    for (DocValuesFieldUpdates update : updates) {
+      deletes.onDocValuesUpdate(update.field, update.iterator());
+    }
+    deletes.onDocValuesUpdate(fieldInfo);
     assertEquals(5, deletes.numPendingDeletes());
     assertTrue(deletes.getLiveDocs().get(0));
     assertFalse(deletes.getLiveDocs().get(1));
@@ -146,7 +152,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
 
   public void testUpdateAppliedOnlyOnce() throws IOException {
     Directory dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field hier
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig().setSoftDeletesField("_soft_deletes"));
     Document doc = new Document();
     doc.add(new StringField("id", "1", Field.Store.YES));
     writer.softUpdateDocument(new Term("id", "1"), doc,
@@ -169,7 +175,10 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
     List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3));
-    deletes.onDocValuesUpdate(fieldInfo, updates);
+    for (DocValuesFieldUpdates update : updates) {
+      deletes.onDocValuesUpdate(update.field, update.iterator());
+    }
+    deletes.onDocValuesUpdate(fieldInfo);
     assertEquals(1, deletes.numPendingDeletes());
     assertTrue(deletes.getLiveDocs().get(0));
     assertFalse(deletes.getLiveDocs().get(1));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
index dea7bc9..30a11b6 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesDirectoryReaderWrapper.java
@@ -27,6 +27,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
@@ -196,4 +197,35 @@ public class TestSoftDeletesDirectoryReaderWrapper extends LuceneTestCase {
     assertEquals(1, leafCalled.get());
     IOUtils.close(reader, writer, dir);
   }
+
+  public void testForceMergeDeletes() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField("soft_delete");
+    config.setMergePolicy(newMergePolicy(random(), false)); // no mock MP it might not select segments for force merge
+    if (random().nextBoolean()) {
+      config.setMergePolicy(new SoftDeletesRetentionMergePolicy("soft_delete",
+          () -> new MatchNoDocsQuery(), config.getMergePolicy()));
+    }
+    IndexWriter writer = new IndexWriter(dir, config);
+    // The first segment includes d1 and d2
+    for (int i = 0; i < 2; i++) {
+      Document d = new Document();
+      d.add(new StringField("id", Integer.toString(i), Field.Store.YES));
+      writer.addDocument(d);
+    }
+    writer.flush();
+    // The second segment includes only the tombstone
+    Document tombstone = new Document();
+    tombstone.add(new NumericDocValuesField("soft_delete", 1));
+    writer.softUpdateDocument(new Term("id", "1"), tombstone, new NumericDocValuesField("soft_delete", 1));
+    // Internally, forceMergeDeletes will call flush to flush pending updates
+    // Thus, we will have two segments - both having soft-deleted documents.
+    // We expect any MP to merge these segments into one segment
+    // when calling forceMergeDeletes.
+    writer.forceMergeDeletes(true);
+    assertEquals(1, writer.maxDoc());
+    assertEquals(1, writer.segmentInfos.asList().size());
+    writer.close();
+    dir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index f4240e1..d0eaa43 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -1541,7 +1541,53 @@ public class TestLRUQueryCache extends LuceneTestCase {
     reader.close();
     w.close();
     dir.close();
+  }
+
+
+  public void testQueryCacheSoftUpdate() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig().setSoftDeletesField("soft_delete");
+    IndexWriter w = new IndexWriter(dir, iwc);
+    LRUQueryCache queryCache = new LRUQueryCache(10, 1000 * 1000, ctx -> true);
+    IndexSearcher.setDefaultQueryCache(queryCache);
+    IndexSearcher.setDefaultQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
+
+    SearcherManager sm = new SearcherManager(w, new SearcherFactory());
+
+    Document doc = new Document();
+    doc.add(new StringField("id", "1", org.apache.lucene.document.Field.Store.YES));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("id", "2", org.apache.lucene.document.Field.Store.YES));
+    w.addDocument(doc);
+
+    sm.maybeRefreshBlocking();
+
+    IndexSearcher searcher = sm.acquire();
+    Query query = new BooleanQuery.Builder().add(new TermQuery(new Term("id", "1")), BooleanClause.Occur.FILTER).build();
+    assertEquals(1, searcher.count(query));
+    assertEquals(1, queryCache.getCacheSize());
+    assertEquals(0, queryCache.getEvictionCount());
 
+    boolean softDelete = true;
+    if (softDelete) {
+      Document tombstone = new Document();
+      tombstone.add(new NumericDocValuesField("soft_delete", 1));
+      w.softUpdateDocument(new Term("id", "1"), tombstone, new NumericDocValuesField("soft_delete", 1));
+      w.softUpdateDocument(new Term("id", "2"), tombstone, new NumericDocValuesField("soft_delete", 1));
+    } else {
+      w.deleteDocuments(new Term("id", "1"));
+      w.deleteDocuments(new Term("id", "2"));
+    }
+    sm.maybeRefreshBlocking();
+    // All docs in the first segment are deleted - we should drop it with the default merge policy.
+    sm.release(searcher);
+    assertEquals(0, queryCache.getCacheSize());
+    assertEquals(1, queryCache.getEvictionCount());
+    sm.close();
+    w.close();
+    dir.close();
   }
 
   public void testBulkScorerLocking() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c70cceae/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 1087e20..d6c0e7e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1049,7 +1049,11 @@ public abstract class LuceneTestCase extends Assert {
   }
 
   public static MergePolicy newMergePolicy(Random r) {
-    if (rarely(r)) {
+    return newMergePolicy(r, true);
+  }
+
+  public static MergePolicy newMergePolicy(Random r, boolean includeMockMP) {
+    if (includeMockMP && rarely(r)) {
       return new MockRandomMergePolicy(r);
     } else if (r.nextBoolean()) {
       return newTieredMergePolicy(r);


[7/9] lucene-solr:jira/solr-11833: SOLR-11920: Adding a file exists check before skipping a file for replication

Posted by ab...@apache.org.
SOLR-11920: Adding a file exists check before skipping a file for replication


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

Branch: refs/heads/jira/solr-11833
Commit: 72ee0f64d4eaad9f81ff9730ebfbc466548586e4
Parents: a417a2c
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Tue Apr 17 10:48:36 2018 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Tue Apr 17 10:48:36 2018 +0530

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/handler/IndexFetcher.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/72ee0f64/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 3c6859b..9fde19c 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -1037,8 +1037,9 @@ public class IndexFetcher {
       boolean alwaysDownload = filesToAlwaysDownloadIfNoChecksums(filename, size, compareResult);
       LOG.debug("Downloading file={} size={} checksum={} alwaysDownload={}", filename, size, file.get(CHECKSUM), alwaysDownload);
       if (!compareResult.equal || downloadCompleteIndex || alwaysDownload) {
-        if (downloadCompleteIndex && doDifferentialCopy && compareResult.equal && compareResult.checkSummed) {
-          File localFile = new File(indexDirPath, filename);
+        File localFile = new File(indexDirPath, filename);
+        if (downloadCompleteIndex && doDifferentialCopy && compareResult.equal && compareResult.checkSummed
+            && localFile.exists()) {
           LOG.info("Don't need to download this file. Local file's path is: {}, checksum is: {}",
               localFile.getAbsolutePath(), file.get(CHECKSUM));
           // A hard link here should survive the eventual directory move, and should be more space efficient as


[8/9] lucene-solr:jira/solr-11833: LUCENE-8257: All edges have a backing plane, so edge membership doesn't bleed through to the opposite side of the world.

Posted by ab...@apache.org.
LUCENE-8257: All edges have a backing plane, so edge membership doesn't bleed through to the opposite side of the world.


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

Branch: refs/heads/jira/solr-11833
Commit: 4ee92c22a4b731d3ec2f93409f3fe57ae348cea1
Parents: 72ee0f6
Author: Karl Wright <Da...@gmail.com>
Authored: Tue Apr 17 02:45:44 2018 -0400
Committer: Karl Wright <Da...@gmail.com>
Committed: Tue Apr 17 02:45:44 2018 -0400

----------------------------------------------------------------------
 .../spatial3d/geom/GeoComplexPolygon.java       | 19 ++++++++++++++-----
 .../lucene/spatial3d/geom/GeoPolygonTest.java   | 20 ++++++++++++++++++++
 2 files changed, 34 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ee92c22/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoComplexPolygon.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoComplexPolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoComplexPolygon.java
index 73ed92e..e925d31 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoComplexPolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoComplexPolygon.java
@@ -590,6 +590,8 @@ class GeoComplexPolygon extends GeoBasePolygon {
     }
   }
 
+  private final static double[] halfProportions = new double[]{0.5};
+  
   /**
    * An instance of this class describes a single edge, and includes what is necessary to reliably determine intersection
    * in the context of the even/odd algorithm used.
@@ -600,6 +602,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
     public final GeoPoint[] notablePoints;
     public final SidedPlane startPlane;
     public final SidedPlane endPlane;
+    public final SidedPlane backingPlane;
     public final Plane plane;
     public final XYZBounds planeBounds;
     public Edge previous = null;
@@ -612,13 +615,19 @@ class GeoComplexPolygon extends GeoBasePolygon {
       this.plane = new Plane(startPoint, endPoint);
       this.startPlane =  new SidedPlane(endPoint, plane, startPoint);
       this.endPlane = new SidedPlane(startPoint, plane, endPoint);
+      final GeoPoint interpolationPoint = plane.interpolate(startPoint, endPoint, halfProportions)[0];
+      this.backingPlane = new SidedPlane(interpolationPoint, interpolationPoint, 0.0);
       this.planeBounds = new XYZBounds();
       this.planeBounds.addPoint(startPoint);
       this.planeBounds.addPoint(endPoint);
-      this.planeBounds.addPlane(pm, this.plane, this.startPlane, this.endPlane);
+      this.planeBounds.addPlane(pm, this.plane, this.startPlane, this.endPlane, this.backingPlane);
       //System.err.println("Recording edge "+this+" from "+startPoint+" to "+endPoint+"; bounds = "+planeBounds);
     }
-    
+
+    public boolean isWithin(final double thePointX, final double thePointY, final double thePointZ) {
+      return plane.evaluateIsZero(thePointX, thePointY, thePointZ) && startPlane.isWithin(thePointX, thePointY, thePointZ) && endPlane.isWithin(thePointX, thePointY, thePointZ) && backingPlane.isWithin(thePointX, thePointY, thePointZ);
+    }
+
     // Hashcode and equals are system default!!
   }
   
@@ -945,7 +954,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
     @Override
     public boolean matches(final Edge edge) {
       // Early exit if the point is on the edge.
-      if (edge.plane.evaluateIsZero(thePointX, thePointY, thePointZ) && edge.startPlane.isWithin(thePointX, thePointY, thePointZ) && edge.endPlane.isWithin(thePointX, thePointY, thePointZ)) {
+      if (edge.isWithin(thePointX, thePointY, thePointZ)) {
         return false;
       }
       
@@ -1041,7 +1050,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
     @Override
     public boolean matches(final Edge edge) {
       // Early exit if the point is on the edge.
-      if (edge.plane.evaluateIsZero(thePointX, thePointY, thePointZ) && edge.startPlane.isWithin(thePointX, thePointY, thePointZ) && edge.endPlane.isWithin(thePointX, thePointY, thePointZ)) {
+      if (edge.isWithin(thePointX, thePointY, thePointZ)) {
         return false;
       }
       
@@ -1289,7 +1298,7 @@ class GeoComplexPolygon extends GeoBasePolygon {
     @Override
     public boolean matches(final Edge edge) {
       // Early exit if the point is on the edge, in which case we accidentally discovered the answer.
-      if (edge.plane.evaluateIsZero(thePointX, thePointY, thePointZ) && edge.startPlane.isWithin(thePointX, thePointY, thePointZ) && edge.endPlane.isWithin(thePointX, thePointY, thePointZ)) {
+      if (edge.isWithin(thePointX, thePointY, thePointZ)) {
         return false;
       }
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ee92c22/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
index 524475a..d841cbd 100755
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
@@ -1586,5 +1586,25 @@ shape:
     final GeoPoint point = new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(1.98E-321), Geo3DUtil.fromDegrees(0.005183505059185348));
     assertTrue(polygon.isWithin(point) == largePolygon.isWithin(point));
   }
+
+  @Test
+  public void testLUCENE8257() {
+    //POLYGON((12.9610296281349 -8.35317290232106,15.448601008878832 -3.990004427754539,22.375905319231205 0.2308875600810982,-13.473550791109867 30.10483127471788,-17.854443360411242 33.07441476406424,-3.928621142543736E-11 4.688559453373203E-11,0.0 -5.546974900361278E-104,12.9610296281349 -8.35317290232106))
+    final List<GeoPoint> points = new ArrayList<>();
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(-8.35317290232106), Geo3DUtil.fromDegrees(12.9610296281349)));
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(-3.990004427754539), Geo3DUtil.fromDegrees(15.448601008878832)));
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(0.2308875600810982), Geo3DUtil.fromDegrees(22.375905319231205)));
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(30.10483127471788), Geo3DUtil.fromDegrees(-13.473550791109867)));
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(33.07441476406424), Geo3DUtil.fromDegrees(-17.854443360411242)));
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(4.688559453373203E-11), Geo3DUtil.fromDegrees(-3.928621142543736E-11)));
+    points.add(new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(-5.546974900361278E-104), Geo3DUtil.fromDegrees(0.0)));
+    final GeoPolygonFactory.PolygonDescription description = new GeoPolygonFactory.PolygonDescription(points);
+    final GeoPolygon polygon = GeoPolygonFactory.makeGeoPolygon(PlanetModel.WGS84, description);
+    final GeoPolygon largePolygon = GeoPolygonFactory.makeLargeGeoPolygon(PlanetModel.WGS84, Collections.singletonList(description));
+
+    //POINT(-179.99999999999997 -9.638811778842766E-12)
+    final GeoPoint point = new GeoPoint(PlanetModel.WGS84, Geo3DUtil.fromDegrees(-9.638811778842766E-12), Geo3DUtil.fromDegrees(-179.99999999999997));
+    assertTrue(polygon.isWithin(point) == largePolygon.isWithin(point));
+  }
   
 }


[5/9] lucene-solr:jira/solr-11833: SOLR-11840: Fix bin/solr help-text inconsistencies

Posted by ab...@apache.org.
SOLR-11840: Fix bin/solr help-text inconsistencies


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

Branch: refs/heads/jira/solr-11833
Commit: 7edfd9c41018f16e7b6af1b58e9983e427eff9de
Parents: 449ecb6
Author: Jason Gerlowski <ge...@apache.org>
Authored: Mon Apr 16 22:21:27 2018 -0400
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Mon Apr 16 22:21:27 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt  |   2 +
 solr/bin/solr     |   8 ++--
 solr/bin/solr.cmd | 118 +++++++++++++++++++++++++++++--------------------
 3 files changed, 77 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7edfd9c4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 08c246f..dd4114a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -159,6 +159,8 @@ Bug Fixes
 
 * SOLR-11724: Cdcr bootstrapping should ensure that non-leader replicas should sync with the leader
   (Amrit Sarkar, Varun Thacker)
+
+* SOLR-11840: Fix bin/solr help-text inconsistencies (Jason Gerlowski)
  
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7edfd9c4/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index fd41acc..3cda782 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -347,7 +347,7 @@ function print_usage() {
     echo "                  on which example is run. The default value is server/solr. If passed relative dir,"
     echo "                  validation with current dir will be done, before trying default server/<dir>"
     echo ""
-    echo "  -t <dir>      Sets the solr.data.home system property, where Solr will store data (index)."
+    echo "  -t <dir>      Sets the solr.data.home system property, where Solr will store index data in <instance_dir>/data subdirectories."
     echo "                  If not set, Solr uses solr.solr.home for config and data."
     echo ""
     echo "  -e <example>  Name of the example to run; available examples:"
@@ -370,7 +370,7 @@ function print_usage() {
     echo ""
     echo "  -v and -q     Verbose (-v) or quiet (-q) logging. Sets default log level to DEBUG or WARN instead of INFO"
     echo ""
-    echo "  -V or -verbose Verbose messages from this script"
+    echo "  -V/-verbose   Verbose messages from this script"
     echo ""
   elif [ "$CMD" == "stop" ]; then
     echo ""
@@ -382,6 +382,8 @@ function print_usage() {
     echo ""
     echo "  -all          Find and stop all running Solr servers on this host"
     echo ""
+    echo "  -V/-verbose   Verbose messages from this script"
+    echo ""
     echo "  NOTE: To see if any Solr servers are running, do: solr status"
     echo ""
   elif [ "$CMD" == "healthcheck" ]; then
@@ -572,7 +574,7 @@ function print_usage() {
     echo ""
     echo "             Only the node names are listed, not data"
     echo ""
-    echo "         mkroot makes a znode on Zookeeper with no data. Can be used to make a path of arbitrary"
+    echo "         mkroot makes a znode in Zookeeper with no data. Can be used to make a path of arbitrary"
     echo "             depth but primarily intended to create a 'chroot'."
     echo ""
     echo "             <path>: The Zookeeper path to create. Leading slash is assumed if not present."

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7edfd9c4/solr/bin/solr.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index 0aaeb10..e9f6c45 100644
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -269,6 +269,8 @@ IF "%SCRIPT_CMD%"=="create_core" goto create_core_usage
 IF "%SCRIPT_CMD%"=="create_collection" goto create_collection_usage
 IF "%SCRIPT_CMD%"=="delete" goto delete_usage
 IF  "%SCRIPT_CMD%"=="zk" goto zk_usage
+IF "%SCRIPT_CMD%"=="auth" goto auth_usage
+IF "%SCRIPT_CMD%"=="status" goto status_usage
 goto done
 
 :script_usage
@@ -302,6 +304,9 @@ goto done
 @echo   -h host       Specify the hostname for this Solr instance
 @echo.
 @echo   -p port       Specify the port to start the Solr HTTP listener on; default is 8983
+@echo "                  The specified port (SOLR_PORT) will also be used to determine the stop port"
+@echo "                  STOP_PORT=(\$SOLR_PORT-1000) and JMX RMI listen port RMI_PORT=(\$SOLR_PORT+10000). "
+@echo "                  For instance, if you set -p 8985, then the STOP_PORT=7985 and RMI_PORT=18985"
 @echo.
 @echo   -d dir        Specify the Solr server directory; defaults to server
 @echo.
@@ -317,9 +322,9 @@ goto done
 @echo                   specified directory should contain a solr.xml file, unless solr.xml exists in Zookeeper.
 @echo                   This parameter is ignored when running examples (-e), as the solr.solr.home depends
 @echo                   on which example is run. The default value is server/solr. If passed a relative dir
-@echo                   validation with the current dir will be done before trying the default server/<dir>
+@echo                   validation with the current dir will be done before trying the default server/^<dir^>
 @echo.
-@echo   -t dir        Sets the solr.data.home system property, used as root for ^<instance_dir^>/data directories.
+@echo   -t dir        Sets the solr.data.home system property, where Solr will store index data in ^<instance_dir^>/data subdirectories.
 @echo                   If not set, Solr uses solr.solr.home for both config and data.
 @echo.
 @echo   -e example    Name of the example to run; available examples:
@@ -342,13 +347,21 @@ goto done
 @echo.
 @echo   -v and -q     Verbose (-v) or quiet (-q) logging. Sets default log level to DEBUG or WARN instead of INFO
 @echo.
-@echo   -V            Verbose messages from this script
+@echo   -V/-verbose   Verbose messages from this script
+@echo.
+goto done
+
+:status_usage
+@echo.
+@echo Usage: solr status
+@echo.
+@echo   NOTE: This command will show the status of all running Solr servers
 @echo.
 goto done
 
 :stop_usage
 @echo.
-@echo Usage: solr stop [-k key] [-p port]
+@echo Usage: solr stop [-k key] [-p port] [-V]
 @echo.
 @echo  -k key      Stop key; default is solrrocks
 @echo.
@@ -356,11 +369,15 @@ goto done
 @echo.
 @echo  -all        Find and stop all running Solr servers on this host
 @echo.
+@echo  -V/-verbose Verbose messages from this script
+@echo.
+@echo  NOTE: To see if any Solr servers are running, do: solr status
+@echo.
 goto done
 
 :healthcheck_usage
 @echo.
-@echo Usage: solr healthcheck [-c collection] [-z zkHost]
+@echo Usage: solr healthcheck [-c collection] [-z zkHost] [-V] 
 @echo.
 @echo Can be run from remote (non-Solr^) hosts, as long as a proper ZooKeeper connection is provided
 @echo.
@@ -390,7 +407,7 @@ goto done
 
 :delete_usage
 echo.
-echo Usage: solr delete [-c name] [-deleteConfig boolean] [-p port] [-V]
+echo Usage: solr delete [-c name] [-deleteConfig true^|false] [-p port] [-V]
 echo.
 echo  Deletes a core or collection depending on whether Solr is running in standalone (core) or SolrCloud
 echo  mode (collection). If you're deleting a collection in SolrCloud mode, the default behavior is to also
@@ -399,26 +416,26 @@ echo  You can override this behavior by passing -deleteConfig false when running
 echo.
 echo  Can be run on remote (non-Solr^) hosts, as long as a valid SOLR_HOST is provided in solr.in.cmd
 echo.
-echo   -c name     Name of core to create
+echo   -c name     Name of core to delete
 echo.
 echo   -deleteConfig boolean Delete the configuration directory from Zookeeper; default is true
 echo.
-echo   -p port     Port of a local Solr instance where you want to create the new core
+echo   -p port     Port of a local Solr instance where you want to delete the core/collection
 echo                 If not specified, the script will search the local system for a running
 echo                 Solr instance and will use the port of the first server it finds.
 echo.
-echo   -V            Enable more verbose output.
+echo   -V            Enables more verbose output.
 echo.
 goto done
 
 :create_core_usage
 echo.
-echo Usage: solr create_core [-c name] [-d confdir] [-p port] [-V]
+echo Usage: solr create_core [-c ^<core^>] [-d confdir] [-p port] [-V]
 echo.
 echo When a configSet is used, this can be run from any host.  If pointing at a non-configSet directory, this
 echo must be run from the host that you wish to create the core on.
 echo.
-echo   -c name     Name of core to create
+echo   -c ^<core^>     Name of core to create
 echo.
 echo   -d confdir  Configuration directory to copy when creating the new core, built-in options are:
 echo.
@@ -441,13 +458,13 @@ goto done
 
 :create_collection_usage
 echo.
-echo Usage: solr create_collection [-c name] [-d confdir] [-n confname] [-shards #] [-replicationFactor #] [-p port] [-V]
+echo Usage: solr create_collection [-c collection] [-d confdir] [-n confname] [-shards #] [-replicationFactor #] [-p port] [-V]
 echo.
 echo Can be run from remote (non-Solr^) hosts, as long as a valid SOLR_HOST is provided in solr.in.cmd.
 echo.
-echo   -c name               Name of collection to create
+echo   -c ^<collection^>         Name of collection to create
 echo.
-echo   -d confdir            Configuration directory to copy when creating the new collection, built-in options are:
+echo   -d ^<confdir^>            Configuration directory to copy when creating the new collection, built-in options are:
 echo.
 echo       _default: Minimal configuration, which supports enabling/disabling field-guessing support
 echo       sample_techproducts_configs: Example configuration with many optional features enabled to
@@ -467,9 +484,9 @@ echo                             will be uploaded to Zookeeper using the collect
 echo                             to use an existing directory or override the name of the configuration in
 echo                              Zookeeper, then use the -c option.
 echo.
-echo   -shards #             Number of shards to split the collection into
+echo   -shards #             Number of shards to split the collection into; default is 1
 echo.
-echo   -replicationFactor #  Number of copies of each document in the collection
+echo   -replicationFactor #  Number of copies of each document in the collection, default is 1 (no replication)
 echo.
 echo   -p port               Port of a local Solr instance where you want to create the new collection
 echo                           If not specified, the script will search the local system for a running
@@ -510,18 +527,19 @@ echo.
 echo.             ^<src^>, ^<dest^> : [file:][/]path/to/local/file or zk:/path/to/zk/node
 echo                              NOTE: ^<src^> and ^<dest^> may both be Zookeeper resources prefixed by 'zk:'
 echo             When ^<src^> is a zk resource, ^<dest^> may be '.'
+echo             If ^<dest^> ends with '/', then ^<dest^> will be a local folder or parent znode and the last
 echo             element of the ^<src^> path will be appended unless ^<src^> also ends in a slash. 
 echo             ^<dest^> may be zk:, which may be useful when using the cp -r form to backup/restore 
-echo              the entire zk state.
-echo              You must enclose local paths that end in a wildcard in quotes or just
-echo              end the local path in a slash. That is,
-echo              'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181' is equivalent to
-echo              'bin/solr zk cp -r ^"/some/dir/*^" zk:/ -z localhost:2181'
-echo              but 'bin/solr zk cp -r /some/dir/* zk:/ -z localhost:2181' will throw an error
-echo .
-echo              here's an example of backup/restore for a ZK configuration:
-echo              to copy to local: 'bin/solr zk cp -r zk:/ /some/dir -z localhost:2181'
-echo              to restore to ZK: 'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181'
+echo             the entire zk state.
+echo             You must enclose local paths that end in a wildcard in quotes or just
+echo             end the local path in a slash. That is,
+echo             'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181' is equivalent to
+echo             'bin/solr zk cp -r ^"/some/dir/*^" zk:/ -z localhost:2181'
+echo             but 'bin/solr zk cp -r /some/dir/* zk:/ -z localhost:2181' will throw an error.
+echo.
+echo             Here's an example of backup/restore for a ZK configuration:
+echo             to copy to local: 'bin/solr zk cp -r zk:/ /some/dir -z localhost:2181'
+echo             to restore to ZK: 'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181'
 echo.
 echo             The 'file:' prefix is stripped, thus 'file:/wherever' specifies an absolute local path and
 echo             'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute.
@@ -580,37 +598,37 @@ IF "%ZK_FULL%"=="true" (
 goto done
 
 :auth_usage
-echo Usage: solr auth enable [-type basicAuth] -credentials user:pass [-blockUnknown ^<true|false^>] [-updateIncludeFileOnly ^<true|false^>] [-V]
-echo        solr auth enable [-type basicAuth] -prompt ^<true|false^> [-blockUnknown ^<true|false^>] [-updateIncludeFileOnly ^<true|false^>] [-V]
-echo        solr auth disable [-updateIncludeFileOnly ^<true|false^>] [-V]
-echo
+echo Usage: solr auth enable [-type basicAuth] -credentials user:pass [-blockUnknown ^<true^|false^>] [-updateIncludeFileOnly ^<true^|false^>] [-V]
+echo        solr auth enable [-type basicAuth] -prompt ^<true^|false^> [-blockUnknown ^<true^|false^>] [-updateIncludeFileOnly ^<true^|false^>] [-V]
+echo        solr auth disable [-updateIncludeFileOnly ^<true^|false^>] [-V]
+echo.
 echo  Updates or enables/disables authentication.  Must be run on the machine hosting Solr.
-echo
+echo.
 echo   -type ^<type^>                 The authentication mechanism to enable. Defaults to 'basicAuth'.
-echo
+echo.
 echo   -credentials ^<user:pass^>     The username and password of the initial user
 echo                                Note: only one of -prompt or -credentials must be provided
-echo
-echo   -prompt ^<true|false^>         Prompts the user to provide the credentials
+echo.
+echo   -prompt ^<true^|false^>         Prompts the user to provide the credentials
 echo                                Note: only one of -prompt or -credentials must be provided
-echo
-echo   -blockUnknown ^<true|false^>   When true, this blocks out access to unauthenticated users. When not provided,
+echo.
+echo   -blockUnknown ^<true^|false^>   When true, this blocks out access to unauthenticated users. When not provided,
 echo                                this defaults to false (i.e. unauthenticated users can access all endpoints, except the
-echo                                operations like collection-edit, security-edit, core-admin-edit etc.). Check the reference
+echo                                operations like collection-edit, security-edit, core-admin-edit etc.^). Check the reference
 echo                                guide for Basic Authentication for more details.
-echo
-echo   -updateIncludeFileOnly ^<true|false^>    Only update the solr.in.sh or solr.in.cmd file, and skip actual enabling/disabling"
-echo                                          authentication (i.e. don't update security.json)"
-echo
+echo.
+echo   -updateIncludeFileOnly ^<true^|false^>    Only update the solr.in.sh or solr.in.cmd file, and skip actual enabling/disabling"
+echo                                          authentication (i.e. don't update security.json^)"
+echo.
 echo   -z zkHost                    Zookeeper connection string
-echo
-echo   -d <dir>                     Specify the Solr server directory"
-echo 
-echo   -s <dir>                     Specify the Solr home directory. This is where any credentials or authentication"
-echo                                configuration files (e.g. basicAuth.conf) would be placed."
-echo
+echo.
+echo   -d ^<dir^>                     Specify the Solr server directory"
+echo.
+echo   -s ^<dir^>                     Specify the Solr home directory. This is where any credentials or authentication"
+echo                                configuration files (e.g. basicAuth.conf^) would be placed."
+echo.
 echo   -V                           Enable more verbose output
-echo 
+echo.
 goto done
 
 REM Really basic command-line arg parsing
@@ -1885,6 +1903,10 @@ IF "%FIRST_ARG%"=="start" (
   goto create_collection_usage
 ) ELSE IF "%FIRST_ARG%"=="zk" (
   goto zk_short_usage
+) ELSE IF "%FIRST_ARG%"=="auth" (
+  goto auth_usage
+) ELSE IF "%FIRST_ARG%"=="status" (
+  goto status_usage
 ) ELSE (
   goto script_usage
 )


[3/9] lucene-solr:jira/solr-11833: SOLR-12147: Don't use memory-intensive codecs to test uninvert limits

Posted by ab...@apache.org.
SOLR-12147: Don't use memory-intensive codecs to test uninvert limits

This separates out TestDocTermOrds.testTriggerUninvertLimit into its own
test class, and suppresses Memory codec on it


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

Branch: refs/heads/jira/solr-11833
Commit: 37e065007fae77f1e1c13a5c3e2938a7adf7cdf2
Parents: c70ccea
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Apr 16 14:53:35 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Apr 16 15:24:04 2018 +0100

----------------------------------------------------------------------
 .../solr/uninverting/TestDocTermOrds.java       | 57 -------------
 .../TestDocTermOrdsUninvertLimit.java           | 88 ++++++++++++++++++++
 2 files changed, 88 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/37e06500/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
index 21f44ea..0aa91ad 100644
--- a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Objects;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -137,62 +136,6 @@ public class TestDocTermOrds extends LuceneTestCase {
     dir.close();
   }
 
-  /* UnInvertedField had a reference block limitation of 2^24. This unit test triggered it.
-   *
-   * With the current code, the test verifies that the old limit no longer applies.
-   * New limit is 2^31, which is not very realistic to unit-test. */
-  @SuppressWarnings({"ConstantConditions", "PointlessBooleanExpression"})
-  @Nightly
-  public void testTriggerUnInvertLimit() throws IOException {
-    final boolean SHOULD_TRIGGER = false; // Set this to true to use the test with the old implementation
-
-    assumeFalse("Don't run this massive test with MemoryPostingsFormat, as it can OOM",
-        Objects.equals(Codec.getDefault().postingsFormat().getName(), "Memory"));
-
-    // Ensure enough terms inside of a single UnInvert-pass-structure to trigger the limit
-    final int REF_LIMIT = (int) Math.pow(2, 24); // Maximum number of references within a single pass-structure
-    final int DOCS = (1<<16)-1;                  // The number of documents within a single pass (simplified)
-    final int TERMS = REF_LIMIT/DOCS;            // Each document must have this many references aka terms hit limit
-
-    Directory dir = newDirectory();
-    final RandomIndexWriter w = new RandomIndexWriter(random(), dir,
-        newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
-    Document doc = new Document();
-    Field field = newTextField("field", "", Field.Store.NO);
-    doc.add(field);
-
-    StringBuilder sb = new StringBuilder(TERMS*(Integer.toString(TERMS).length()+1));
-    for (int i = 0 ; i < TERMS ; i++) {
-      sb.append(" ").append(Integer.toString(i));
-    }
-    field.setStringValue(sb.toString());
-
-    for (int i = 0 ; i < DOCS ; i++) {
-      w.addDocument(doc);
-    }
-    //System.out.println("\n Finished adding " + DOCS + " documents of " + TERMS + " unique terms");
-    final IndexReader r = w.getReader();
-    w.close();
-    
-    try {
-      final LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
-      TestUtil.checkReader(ar);
-      final DocTermOrds dto = new DocTermOrds(ar, ar.getLiveDocs(), "field"); // bigTerms turned off
-      if (SHOULD_TRIGGER) {
-        fail("DocTermOrds should have failed with a \"Too many values for UnInvertedField\" message");
-      }
-    } catch (IllegalStateException e) {
-      if (!SHOULD_TRIGGER) {
-        fail("DocTermsOrd should not have failed with this implementation, but got exception " +
-            e.getClass().getSimpleName() + " with message " + e.getMessage());
-      }
-      // This is (hopefully) "Too many values for UnInvertedField faceting on field field", so all is as expected
-    } finally {
-      r.close();
-      dir.close();
-    }
-  }
-
   public void testRandom() throws Exception {
     Directory dir = newDirectory();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/37e06500/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java
new file mode 100644
index 0000000..af66b42
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrdsUninvertLimit.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.uninverting;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.index.SlowCompositeReaderWrapper;
+
+@LuceneTestCase.SuppressCodecs({"Memory"})
+public class TestDocTermOrdsUninvertLimit extends LuceneTestCase {
+
+  /* UnInvertedField had a reference block limitation of 2^24. This unit test triggered it.
+   *
+   * With the current code, the test verifies that the old limit no longer applies.
+   * New limit is 2^31, which is not very realistic to unit-test. */
+  @SuppressWarnings({"ConstantConditions", "PointlessBooleanExpression"})
+  @Nightly
+  public void testTriggerUnInvertLimit() throws IOException {
+    final boolean SHOULD_TRIGGER = false; // Set this to true to use the test with the old implementation
+
+    // Ensure enough terms inside of a single UnInvert-pass-structure to trigger the limit
+    final int REF_LIMIT = (int) Math.pow(2, 24); // Maximum number of references within a single pass-structure
+    final int DOCS = (1<<16)-1;                  // The number of documents within a single pass (simplified)
+    final int TERMS = REF_LIMIT/DOCS;            // Each document must have this many references aka terms hit limit
+
+    Directory dir = newDirectory();
+    final RandomIndexWriter w = new RandomIndexWriter(random(), dir,
+        newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
+    Document doc = new Document();
+    Field field = newTextField("field", "", Field.Store.NO);
+    doc.add(field);
+
+    StringBuilder sb = new StringBuilder(TERMS*(Integer.toString(TERMS).length()+1));
+    for (int i = 0 ; i < TERMS ; i++) {
+      sb.append(" ").append(Integer.toString(i));
+    }
+    field.setStringValue(sb.toString());
+
+    for (int i = 0 ; i < DOCS ; i++) {
+      w.addDocument(doc);
+    }
+    //System.out.println("\n Finished adding " + DOCS + " documents of " + TERMS + " unique terms");
+    final IndexReader r = w.getReader();
+    w.close();
+
+    try {
+      final LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
+      TestUtil.checkReader(ar);
+      final DocTermOrds dto = new DocTermOrds(ar, ar.getLiveDocs(), "field"); // bigTerms turned off
+      if (SHOULD_TRIGGER) {
+        fail("DocTermOrds should have failed with a \"Too many values for UnInvertedField\" message");
+      }
+    } catch (IllegalStateException e) {
+      if (!SHOULD_TRIGGER) {
+        fail("DocTermsOrd should not have failed with this implementation, but got exception " +
+            e.getClass().getSimpleName() + " with message " + e.getMessage());
+      }
+      // This is (hopefully) "Too many values for UnInvertedField faceting on field field", so all is as expected
+    } finally {
+      r.close();
+      dir.close();
+    }
+  }
+}


[2/9] lucene-solr:jira/solr-11833: Unbadapple TestIndexSorting

Posted by ab...@apache.org.
Unbadapple TestIndexSorting


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

Branch: refs/heads/jira/solr-11833
Commit: 0b753c34f7b3752bd962ea6dd661585d66dd22b8
Parents: 37e0650
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Apr 16 14:58:01 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Apr 16 15:24:04 2018 +0100

----------------------------------------------------------------------
 lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0b753c34/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 e29b9d0..3679d20 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -2221,7 +2221,6 @@ public class TestIndexSorting extends LuceneTestCase {
   }
 
   // pits index time sorting against query time sorting
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
   public void testRandom3() throws Exception {
     int numDocs = atLeast(1000);
     List<RandomDoc> docs = new ArrayList<>();


[6/9] lucene-solr:jira/solr-11833: SOLR-10169: PeerSync will hit an NPE on no response errors when looking for fingerprint.

Posted by ab...@apache.org.
SOLR-10169: PeerSync will hit an NPE on no response errors when looking for fingerprint.


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

Branch: refs/heads/jira/solr-11833
Commit: a417a2cd6a612e7ed1a99b14230c3f08e455f9c1
Parents: 7edfd9c
Author: Erick Erickson <er...@apache.org>
Authored: Mon Apr 16 21:51:02 2018 -0700
Committer: Erick Erickson <er...@apache.org>
Committed: Mon Apr 16 21:51:02 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                        |  4 +++-
 solr/core/src/java/org/apache/solr/update/PeerSync.java | 10 +++++++---
 2 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a417a2cd/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index dd4114a..e010366 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -161,7 +161,9 @@ Bug Fixes
   (Amrit Sarkar, Varun Thacker)
 
 * SOLR-11840: Fix bin/solr help-text inconsistencies (Jason Gerlowski)
- 
+
+* SOLR-10169: PeerSync will hit an NPE on no response errors when looking for fingerprint. (Erick Erickson)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a417a2cd/solr/core/src/java/org/apache/solr/update/PeerSync.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java
index 10b3ea1..3511a6c 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -388,9 +388,13 @@ public class PeerSync implements SolrMetricProducer {
       ShardResponse srsp = shardHandler.takeCompletedOrError();
       if (srsp == null) break;
 
-      Object replicaFingerprint = srsp.getSolrResponse().getResponse().get("fingerprint");
+      Object replicaFingerprint = null;
+      if (srsp.getSolrResponse() != null && srsp.getSolrResponse().getResponse() != null) {
+        replicaFingerprint = srsp.getSolrResponse().getResponse().get("fingerprint");
+      }
+
       if (replicaFingerprint == null) {
-        log.warn("Replica did not return a fingerprint - possibly an older Solr version");
+        log.warn("Replica did not return a fingerprint - possibly an older Solr version or exception");
         continue;
       }
       
@@ -402,7 +406,7 @@ public class PeerSync implements SolrMetricProducer {
           return true;
         }
       } catch(IOException e) {
-        log.warn("Could not cofirm if we are already in sync. Continue with PeerSync");
+        log.warn("Could not confirm if we are already in sync. Continue with PeerSync");
       }
     }
     


[9/9] lucene-solr:jira/solr-11833: Merge branch 'master' into jira/solr-11833

Posted by ab...@apache.org.
Merge branch 'master' into jira/solr-11833


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

Branch: refs/heads/jira/solr-11833
Commit: 2e4ac772edec9274cc910e4c9bac2e382031c289
Parents: 96c2edb 4ee92c2
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Apr 17 12:05:17 2018 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Apr 17 12:05:17 2018 +0200

----------------------------------------------------------------------
 .../codecs/lucene50/Lucene50CompoundFormat.java |   4 -
 .../org/apache/lucene/index/IndexWriter.java    |  28 ++---
 .../org/apache/lucene/index/PendingDeletes.java |  21 +++-
 .../apache/lucene/index/PendingSoftDeletes.java |  75 +++++++++---
 .../apache/lucene/index/ReadersAndUpdates.java  |  10 +-
 .../index/SoftDeletesRetentionMergePolicy.java  |  22 ++--
 .../apache/lucene/index/TestIndexSorting.java   |   1 -
 .../lucene/index/TestPendingSoftDeletes.java    |  19 ++-
 .../TestSoftDeletesDirectoryReaderWrapper.java  |  32 +++++
 .../apache/lucene/search/TestLRUQueryCache.java |  46 ++++++++
 .../spatial3d/geom/GeoComplexPolygon.java       |  19 ++-
 .../lucene/spatial3d/geom/GeoPolygonTest.java   |  20 ++++
 .../org/apache/lucene/util/LuceneTestCase.java  |   6 +-
 solr/CHANGES.txt                                |   6 +-
 solr/bin/solr                                   |   8 +-
 solr/bin/solr.cmd                               | 118 +++++++++++--------
 .../org/apache/solr/handler/IndexFetcher.java   |   5 +-
 .../java/org/apache/solr/update/PeerSync.java   |  10 +-
 .../solr/uninverting/TestDocTermOrds.java       |  57 ---------
 .../TestDocTermOrdsUninvertLimit.java           |  88 ++++++++++++++
 20 files changed, 418 insertions(+), 177 deletions(-)
----------------------------------------------------------------------



[4/9] lucene-solr:jira/solr-11833: Remove dead code

Posted by ab...@apache.org.
Remove dead code


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

Branch: refs/heads/jira/solr-11833
Commit: 449ecb601cac8644700c053df145a92c989e0e15
Parents: 0b753c3
Author: Simon Willnauer <si...@apache.org>
Authored: Mon Apr 16 17:26:44 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Mon Apr 16 17:27:19 2018 +0200

----------------------------------------------------------------------
 .../apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java    | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/449ecb60/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
index da2b93fc..8fc314e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
@@ -28,7 +28,6 @@ import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.StringHelper;
 
 /**
  * Lucene 5.0 compound file format
@@ -75,9 +74,6 @@ public final class Lucene50CompoundFormat extends CompoundFormat {
   public void write(Directory dir, SegmentInfo si, IOContext context) throws IOException {
     String dataFile = IndexFileNames.segmentFileName(si.name, "", DATA_EXTENSION);
     String entriesFile = IndexFileNames.segmentFileName(si.name, "", ENTRIES_EXTENSION);
-    
-    byte[] expectedID = si.getId();
-    byte[] id = new byte[StringHelper.ID_LENGTH];
 
     try (IndexOutput data =    dir.createOutput(dataFile, context);
          IndexOutput entries = dir.createOutput(entriesFile, context)) {