You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2017/01/16 00:10:19 UTC

[01/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9928 Unwrap Directory consistently whenever it's passed as an argument.

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-5944 bb5dfb619 -> 0402c1a06


SOLR-9928 Unwrap Directory consistently whenever it's passed as an argument.


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

Branch: refs/heads/jira/solr-5944
Commit: e5f39f62f76677a5f500af4f323c0c31afb26228
Parents: 1a95c5a
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Sat Jan 7 13:22:37 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Sat Jan 7 13:24:22 2017 +0100

----------------------------------------------------------------------
 .../solr/core/MetricsDirectoryFactory.java      | 60 ++++++++------------
 1 file changed, 25 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e5f39f62/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
index 8706c61..f441579 100644
--- a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
@@ -74,21 +74,28 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
     }
   }
 
-  @Override
-  public void doneWithDirectory(Directory dir) throws IOException {
-    // unwrap
+  /**
+   * Unwrap just one level if the argument is a {@link MetricsDirectory}
+   * @param dir directory
+   * @return delegate if the instance was a {@link MetricsDirectory}, otherwise unchanged.
+   */
+  private static Directory unwrap(Directory dir) {
     if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
+      return ((MetricsDirectory)dir).getDelegate();
+    } else {
+      return dir;
     }
+  }
+
+  @Override
+  public void doneWithDirectory(Directory dir) throws IOException {
+    dir = unwrap(dir);
     in.doneWithDirectory(dir);
   }
 
   @Override
   public void addCloseListener(Directory dir, CachingDirectoryFactory.CloseListener closeListener) {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.addCloseListener(dir, closeListener);
   }
 
@@ -115,19 +122,13 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void remove(Directory dir) throws IOException {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.remove(dir);
   }
 
   @Override
   public void remove(Directory dir, boolean afterCoreClose) throws IOException {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.remove(dir, afterCoreClose);
   }
 
@@ -152,8 +153,9 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
   }
 
   @Override
-  public long size(Directory directory) throws IOException {
-    return in.size(directory);
+  public long size(Directory dir) throws IOException {
+    dir = unwrap(dir);
+    return in.size(dir);
   }
 
   @Override
@@ -183,6 +185,8 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void move(Directory fromDir, Directory toDir, String fileName, IOContext ioContext) throws IOException {
+    fromDir = unwrap(fromDir);
+    toDir = unwrap(toDir);
     in.move(fromDir, toDir, fileName, ioContext);
   }
 
@@ -198,10 +202,7 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory) dir).getDelegate();
-    }
-
+    dir = unwrap(dir);
     in.renameWithOverwrite(dir, fileName, toName);
   }
 
@@ -221,16 +222,8 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
   }
 
   @Override
-  protected Directory getBaseDir(Directory dir) {
-    return in.getBaseDir(dir);
-  }
-
-  @Override
   public void incRef(Directory dir) {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.incRef(dir);
   }
 
@@ -248,10 +241,7 @@ public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCor
 
   @Override
   public void release(Directory dir) throws IOException {
-    // unwrap
-    if (dir instanceof MetricsDirectory) {
-      dir = ((MetricsDirectory)dir).getDelegate();
-    }
+    dir = unwrap(dir);
     in.release(dir);
   }
 


[21/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7624: Remove deprecated TermsQuery

Posted by ho...@apache.org.
LUCENE-7624: Remove deprecated TermsQuery


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

Branch: refs/heads/jira/solr-5944
Commit: 17cd0f00cc1a7bce647eedfe56c860a02aa22654
Parents: 22940f5
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Jan 9 15:33:07 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Jan 9 15:33:07 2017 +0000

----------------------------------------------------------------------
 .../org/apache/lucene/queries/TermsQuery.java   | 65 --------------------
 1 file changed, 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/17cd0f00/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
deleted file mode 100644
index 5effa83..0000000
--- a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.lucene.queries;
-
-import java.util.Arrays;
-import java.util.Collection;
-
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.TermInSetQuery;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * @deprecated Use {@link org.apache.lucene.search.TermInSetQuery}
- */
-@Deprecated
-public class TermsQuery extends TermInSetQuery {
-
-  /**
-   * Creates a new {@link TermsQuery} from the given collection. It
-   * can contain duplicate terms and multiple fields.
-   */
-  public TermsQuery(Collection<Term> terms) {
-    super(terms);
-  }
-
-  /**
-   * Creates a new {@link TermsQuery} from the given collection for
-   * a single field. It can contain duplicate terms.
-   */
-  public TermsQuery(String field, Collection<BytesRef> terms) {
-    super(field, terms);
-  }
-
-  /**
-   * Creates a new {@link TermsQuery} from the given {@link BytesRef} array for
-   * a single field.
-   */
-  public TermsQuery(String field, BytesRef...terms) {
-    this(field, Arrays.asList(terms));
-  }
-
-  /**
-   * Creates a new {@link TermsQuery} from the given array. The array can
-   * contain duplicate terms and multiple fields.
-   */
-  public TermsQuery(final Term... terms) {
-    this(Arrays.asList(terms));
-  }
-
-
-}


[10/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7588: the parallell search method was failing to pass on the user's requested sort when merge-sorting all hits

Posted by ho...@apache.org.
LUCENE-7588: the parallell search method was failing to pass on the user's requested sort when merge-sorting all hits


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

Branch: refs/heads/jira/solr-5944
Commit: 1aa9c4251289e71ab8e87b03797b20f4a8fda0a5
Parents: 570880d
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 8 06:24:03 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 8 06:24:03 2017 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/facet/DrillSideways.java  | 37 ++++++++++----------
 1 file changed, 19 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aa9c425/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
index 61530bc..b2391f5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
@@ -16,6 +16,17 @@
  */
 package org.apache.lucene.facet;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
@@ -34,20 +45,10 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.util.ThreadInterruptedException;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
 /**
  * Computes drill down and sideways counts for the provided
  * {@link DrillDownQuery}.  Drill sideways counts include
@@ -234,8 +235,8 @@ public class DrillSideways {
 
       if (executor != null) { // We have an executor, let use the multi-threaded version
 
-        final CollectorManager<TopFieldCollector, TopDocs> collectorManager =
-                new CollectorManager<TopFieldCollector, TopDocs>() {
+        final CollectorManager<TopFieldCollector, TopFieldDocs> collectorManager =
+                new CollectorManager<TopFieldCollector, TopFieldDocs>() {
 
                   @Override
                   public TopFieldCollector newCollector() throws IOException {
@@ -243,16 +244,16 @@ public class DrillSideways {
                   }
 
                   @Override
-                  public TopDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
-                    final TopDocs[] topDocs = new TopDocs[collectors.size()];
+                  public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
+                    final TopFieldDocs[] topFieldDocs = new TopFieldDocs[collectors.size()];
                     int pos = 0;
                     for (TopFieldCollector collector : collectors)
-                      topDocs[pos++] = collector.topDocs();
-                    return TopDocs.merge(topN, topDocs);
+                      topFieldDocs[pos++] = collector.topDocs();
+                    return TopDocs.merge(sort, topN, topFieldDocs);
                   }
 
                 };
-        ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
+        ConcurrentDrillSidewaysResult<TopFieldDocs> r = search(query, collectorManager);
         return new DrillSidewaysResult(r.facets, r.collectorResult);
 
       } else {


[48/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-5944: Merge branch 'master' into jira/solr-5944

Posted by ho...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
----------------------------------------------------------------------
diff --cc solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
index cd464a7,0000000..f30c827
mode 100644,000000..100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@@ -1,1099 -1,0 +1,1087 @@@
 +
 +/*
 + * 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.update;
 +
 +import static org.junit.internal.matchers.StringContains.containsString;
 +import static org.apache.solr.update.UpdateLogTest.buildAddUpdateCommand;
 +
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.LinkedHashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Random;
 +import java.util.Set;
 +
 +import org.apache.lucene.index.FieldInfo;
 +import org.apache.lucene.util.TestUtil;
 +import org.apache.solr.SolrTestCaseJ4;
 +import org.apache.solr.client.solrj.SolrClient;
 +import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
 +import org.apache.solr.common.SolrDocument;
 +import org.apache.solr.common.SolrDocumentList;
 +import org.apache.solr.common.SolrException;
 +import org.apache.solr.common.SolrInputDocument;
 +import org.apache.solr.common.SolrInputField;
 +import org.apache.solr.request.SolrQueryRequest;
 +import org.apache.solr.update.processor.DistributedUpdateProcessor;
- import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
 +import org.apache.solr.schema.IndexSchema;
 +import org.apache.solr.schema.SchemaField;
 +import org.apache.solr.search.SolrIndexSearcher;
 +import org.apache.solr.update.processor.AtomicUpdateDocumentMerger;
 +import org.apache.solr.util.RefCounted;
 +import org.junit.Before;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +
- import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
- 
 +
 +/**
 + * Tests the in-place updates (docValues updates) for a standalone Solr instance.
 + */
 +public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
 +  private static SolrClient client;
 +
 +  @BeforeClass
 +  public static void beforeClass() throws Exception {
 +    
 +    initCore("solrconfig-tlog.xml", "schema-inplace-updates.xml");
 +
 +    // sanity check that autocommits are disabled
 +    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoCommmitMaxTime);
 +    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxTime);
 +    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoCommmitMaxDocs);
 +    assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxDocs);
 +
 +    // validate that the schema was not changed to an unexpected state
 +    IndexSchema schema = h.getCore().getLatestSchema();
 +    for (String fieldName : Arrays.asList("_version_",
 +                                          "inplace_l_dvo",
 +                                          "inplace_updatable_float",
 +                                          "inplace_updatable_int", 
 +                                          "inplace_updatable_float_with_default",
 +                                          "inplace_updatable_int_with_default")) {
 +      // these fields must only be using docValues to support inplace updates
 +      SchemaField field = schema.getField(fieldName);
 +      assertTrue(field.toString(),
 +                 field.hasDocValues() && ! field.indexed() && ! field.stored());
 +    }
 +    for (String fieldName : Arrays.asList("title_s", "regular_l", "stored_i")) {
 +      // these fields must support atomic updates, but not inplace updates (ie: stored)
 +      SchemaField field = schema.getField(fieldName);
 +      assertTrue(field.toString(), field.stored());
 +    }    
 +
 +    // Don't close this client, it would shutdown the CoreContainer
 +    client = new EmbeddedSolrServer(h.getCoreContainer(), h.coreName);
 +  }
 +
-   @Override
-   public void clearIndex() {
- 
-   }
-   
 +  @Before
 +  public void deleteAllAndCommit() throws Exception {
-     // workaround for SOLR-9934
-     // we need to ensure that all low level IndexWriter metadata (about docvalue fields) is also deleted
-     deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString()));
-     // nocommit: if SOLR-9934 is committed before this branch is merged, replace above line with simple call to clearIndex(); 
-     
++    clearIndex();
 +    assertU(commit("softCommit", "false"));
 +  }
 +
 +  @Test
 +  public void testUpdatingDocValues() throws Exception {
 +    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "inplace_updatable_float", 41), null);
 +    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second", "inplace_updatable_float", 42), null);
 +    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third", "inplace_updatable_float", 43), null);
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "*:*"), "//*[@numFound='3']");
 +
 +    // the reason we're fetching these docids is to validate that the subsequent updates 
 +    // are done in place and don't cause the docids to change
 +    int docid1 = getDocId("1");
 +    int docid2 = getDocId("2");
 +    int docid3 = getDocId("3");
 +
 +    // Check docValues were "set"
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
 +    version2 = addAndAssertVersion(version2, "id", "2", "inplace_updatable_float", map("set", 300));
 +    version3 = addAndAssertVersion(version3, "id", "3", "inplace_updatable_float", map("set", 100));
 +    assertU(commit("softCommit", "false"));
 +
 +    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
 +        "//*[@numFound='3']",
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='200.0']",
 +        "//result/doc[2]/float[@name='inplace_updatable_float'][.='300.0']",
 +        "//result/doc[3]/float[@name='inplace_updatable_float'][.='100.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
 +        "//result/doc[2]/long[@name='_version_'][.='"+version2+"']",
 +        "//result/doc[3]/long[@name='_version_'][.='"+version3+"']",
 +        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']",
 +        "//result/doc[2]/int[@name='[docid]'][.='"+docid2+"']",
 +        "//result/doc[3]/int[@name='[docid]'][.='"+docid3+"']"
 +        );
 +
 +    // Check docValues are "inc"ed
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 1));
 +    version2 = addAndAssertVersion(version2, "id", "2", "inplace_updatable_float", map("inc", -2));
 +    version3 = addAndAssertVersion(version3, "id", "3", "inplace_updatable_float", map("inc", 3));
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
 +        "//*[@numFound='3']",
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='201.0']",
 +        "//result/doc[2]/float[@name='inplace_updatable_float'][.='298.0']",
 +        "//result/doc[3]/float[@name='inplace_updatable_float'][.='103.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
 +        "//result/doc[2]/long[@name='_version_'][.='"+version2+"']",
 +        "//result/doc[3]/long[@name='_version_'][.='"+version3+"']",
 +        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']",
 +        "//result/doc[2]/int[@name='[docid]'][.='"+docid2+"']",
 +        "//result/doc[3]/int[@name='[docid]'][.='"+docid3+"']"
 +        );
 +
 +    // Check back to back "inc"s are working (off the transaction log)
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 1));
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 2)); // new value should be 204
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "id:1", "fl", "*,[docid]"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='204.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
 +        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']");
 +
 +    // Now let the document be atomically updated (non-inplace), ensure the old docvalue is part of new doc
 +    version1 = addAndAssertVersion(version1, "id", "1", "title_s", map("set", "new first"));
 +    assertU(commit("softCommit", "false"));
 +    int newDocid1 = getDocId("1");
 +    assertTrue(newDocid1 != docid1);
 +    docid1 = newDocid1;
 +
 +    assertQ(req("q", "id:1"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='204.0']",
 +        "//result/doc[1]/str[@name='title_s'][.='new first']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']");
 +
 +    // Check if atomic update with "inc" to a docValue works
 +    version2 = addAndAssertVersion(version2, "id", "2", "title_s", map("set", "new second"), "inplace_updatable_float", map("inc", 2));
 +    assertU(commit("softCommit", "false"));
 +    int newDocid2 = getDocId("2");
 +    assertTrue(newDocid2 != docid2);
 +    docid2 = newDocid2;
 +
 +    assertQ(req("q", "id:2"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='300.0']",
 +        "//result/doc[1]/str[@name='title_s'][.='new second']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version2+"']");
 +
 +    // Check if docvalue "inc" update works for a newly created document, which is not yet committed
 +    // Case1: docvalue was supplied during add of new document
 +    long version4 = addAndGetVersion(sdoc("id", "4", "title_s", "fourth", "inplace_updatable_float", "400"), params());
 +    version4 = addAndAssertVersion(version4, "id", "4", "inplace_updatable_float", map("inc", 1));
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "id:4"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='401.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version4+"']");
 +
 +    // Check if docvalue "inc" update works for a newly created document, which is not yet committed
 +    // Case2: docvalue was not supplied during add of new document, should assume default
 +    long version5 = addAndGetVersion(sdoc("id", "5", "title_s", "fifth"), params());
 +    version5 = addAndAssertVersion(version5, "id", "5", "inplace_updatable_float", map("inc", 1));
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "id:5"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='1.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version5+"']");
 +
 +    // Check if docvalue "set" update works for a newly created document, which is not yet committed
 +    long version6 = addAndGetVersion(sdoc("id", "6", "title_s", "sixth"), params());
 +    version6 = addAndAssertVersion(version6, "id", "6", "inplace_updatable_float", map("set", 600));
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "id:6"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='600.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version6+"']");
 +
 +    // Check optimistic concurrency works
 +    long v20 = addAndGetVersion(sdoc("id", "20", "title_s","first", "inplace_updatable_float", 100), params());    
 +    SolrException exception = expectThrows(SolrException.class, () -> {
 +      addAndGetVersion(sdoc("id","20", "_version_", -1, "inplace_updatable_float", map("inc", 1)), null);
 +    });
 +    assertEquals(exception.toString(), SolrException.ErrorCode.CONFLICT.code, exception.code());
 +    assertThat(exception.getMessage(), containsString("expected=-1"));
 +    assertThat(exception.getMessage(), containsString("actual="+v20));
 +
 +
 +    long oldV20 = v20;
 +    v20 = addAndAssertVersion(v20, "id","20", "_version_", v20, "inplace_updatable_float", map("inc", 1));
 +    exception = expectThrows(SolrException.class, () -> {
 +      addAndGetVersion(sdoc("id","20", "_version_", oldV20, "inplace_updatable_float", map("inc", 1)), null);
 +    });
 +    assertEquals(exception.toString(), SolrException.ErrorCode.CONFLICT.code, exception.code());
 +    assertThat(exception.getMessage(), containsString("expected="+oldV20));
 +    assertThat(exception.getMessage(), containsString("actual="+v20));
 +
 +    v20 = addAndAssertVersion(v20, "id","20", "_version_", v20, "inplace_updatable_float", map("inc", 1));
 +    // RTG before a commit
 +    assertJQ(req("qt","/get", "id","20", "fl","id,inplace_updatable_float,_version_"),
 +        "=={'doc':{'id':'20', 'inplace_updatable_float':" + 102.0 + ",'_version_':" + v20 + "}}");
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "id:20"), 
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='102.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+v20+"']");
 +
 +    // Check if updated DVs can be used for search
 +    assertQ(req("q", "inplace_updatable_float:102"), 
 +        "//result/doc[1]/str[@name='id'][.='20']",
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='102.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+v20+"']");
 +
 +    // Check if updated DVs can be used for sorting
 +    assertQ(req("q", "*:*", "sort", "inplace_updatable_float asc"), 
 +        "//result/doc[4]/str[@name='id'][.='1']",
 +        "//result/doc[4]/float[@name='inplace_updatable_float'][.='204.0']",
 +
 +        "//result/doc[5]/str[@name='id'][.='2']",
 +        "//result/doc[5]/float[@name='inplace_updatable_float'][.='300.0']",
 +
 +        "//result/doc[3]/str[@name='id'][.='3']",
 +        "//result/doc[3]/float[@name='inplace_updatable_float'][.='103.0']",
 +
 +        "//result/doc[6]/str[@name='id'][.='4']",
 +        "//result/doc[6]/float[@name='inplace_updatable_float'][.='401.0']",
 +
 +        "//result/doc[1]/str[@name='id'][.='5']",
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='1.0']",
 +
 +        "//result/doc[7]/str[@name='id'][.='6']",
 +        "//result/doc[7]/float[@name='inplace_updatable_float'][.='600.0']",
 +
 +        "//result/doc[2]/str[@name='id'][.='20']",
 +        "//result/doc[2]/float[@name='inplace_updatable_float'][.='102.0']");
 +  }
 +
 +  @Test
 +  public void testUpdateTwoDifferentFields() throws Exception {
 +    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "inplace_updatable_float", 42), null);
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "*:*"), "//*[@numFound='1']");
 +
 +    int docid1 = getDocId("1");
 +
 +    // Check docValues were "set"
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_int", map("set", 10));
 +    assertU(commit("softCommit", "false"));
 +
 +    assertU(commit("softCommit", "false"));
 +
 +    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
 +        "//*[@numFound='1']",
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='200.0']",
 +        "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
 +        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']"
 +        );
 +
 +    // two different update commands, updating each of the fields separately
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_int", map("inc", 1));
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", 1));
 +    // same update command, updating both the fields together
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_int", map("inc", 1),
 +        "inplace_updatable_float", map("inc", 1));
 +
 +    if (random().nextBoolean()) {
 +      assertU(commit("softCommit", "false"));
 +      assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
 +          "//*[@numFound='1']",
 +          "//result/doc[1]/float[@name='inplace_updatable_float'][.='202.0']",
 +          "//result/doc[1]/int[@name='inplace_updatable_int'][.='12']",
 +          "//result/doc[1]/long[@name='_version_'][.='"+version1+"']",
 +          "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']"
 +          );
 +    } 
 +
 +    // RTG
 +    assertJQ(req("qt","/get", "id","1", "fl","id,inplace_updatable_float,inplace_updatable_int"),
 +        "=={'doc':{'id':'1', 'inplace_updatable_float':" + 202.0 + ",'inplace_updatable_int':" + 12 + "}}");
 +
 +  }
 +
 +  @Test
 +  public void testDVUpdatesWithDBQofUpdatedValue() throws Exception {
 +    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "inplace_updatable_float", "0"), null);
 +    assertU(commit());
 +
 +    // in-place update
 +    addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 100), "_version_", version1);
 +
 +    // DBQ where q=inplace_updatable_float:100
 +    assertU(delQ("inplace_updatable_float:100"));
 +
 +    assertU(commit());
 +
 +    assertQ(req("q", "*:*"), "//*[@numFound='0']");
 +  }
 +
 +  @Test
 +  public void testDVUpdatesWithDelete() throws Exception {
 +    long version1 = 0;
 +
 +    for (boolean postAddCommit : Arrays.asList(true, false)) {
 +      for (boolean delById : Arrays.asList(true, false)) {
 +        for (boolean postDelCommit : Arrays.asList(true, false)) {
 +          addAndGetVersion(sdoc("id", "1", "title_s", "first"), params());
 +          if (postAddCommit) assertU(commit());
 +          assertU(delById ? delI("1") : delQ("id:1"));
 +          if (postDelCommit) assertU(commit());
 +          version1 = addAndGetVersion(sdoc("id", "1", "inplace_updatable_float", map("set", 200)), params());
 +          // assert current doc#1 doesn't have old value of "title_s"
 +          assertU(commit());
 +          assertQ(req("q", "title_s:first", "sort", "id asc", "fl", "*,[docid]"),
 +              "//*[@numFound='0']");
 +        }
 +      }
 +    }
 +
 +    // Update to recently deleted (or non-existent) document with a "set" on updatable 
 +    // field should succeed, since it is executed internally as a full update
 +    // because AUDM.doInPlaceUpdateMerge() returns false
 +    assertU(random().nextBoolean()? delI("1"): delQ("id:1"));
 +    if (random().nextBoolean()) assertU(commit());
 +    addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
 +    assertU(commit());
 +    assertQ(req("q", "id:1", "sort", "id asc", "fl", "*"),
 +        "//*[@numFound='1']",
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='200.0']");
 +
 +    // Another "set" on the same field should be an in-place update 
 +    int docid1 = getDocId("1");
 +    addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 300));
 +    assertU(commit());
 +    assertQ(req("q", "id:1", "fl", "*,[docid]"),
 +        "//result/doc[1]/float[@name='inplace_updatable_float'][.='300.0']",
 +        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']");
 +  }
 +
 +  public static long addAndAssertVersion(long expectedCurrentVersion, Object... fields) throws Exception {
 +    assert 0 < expectedCurrentVersion;
 +    long currentVersion = addAndGetVersion(sdoc(fields), null);
 +    assertTrue(currentVersion > expectedCurrentVersion);
 +    return currentVersion;
 +  }
 +
 +  /**
 +   * Helper method to search for the specified (uniqueKey field) id using <code>fl=[docid]</code> 
 +   * and return the internal lucene docid.
 +   */
 +  private int getDocId(String id) throws NumberFormatException, Exception {
 +    SolrDocumentList results = client.query(params("q","id:" + id, "fl", "[docid]")).getResults();
 +    assertEquals(1, results.getNumFound());
 +    assertEquals(1, results.size());
 +    Object docid = results.get(0).getFieldValue("[docid]");
 +    assertTrue(docid instanceof Integer);
 +    return ((Integer)docid);
 +  }
 +
 +  @Test
 +  public void testUpdateOfNonExistentDVsShouldNotFail() throws Exception {
 +    // schema sanity check: assert that the nonexistent_field_i_dvo doesn't exist already
 +    FieldInfo fi;
 +    RefCounted<SolrIndexSearcher> holder = h.getCore().getSearcher();
 +    try {
 +      fi = holder.get().getSlowAtomicReader().getFieldInfos().fieldInfo("nonexistent_field_i_dvo");
 +    } finally {
 +      holder.decref();
 +    }
 +    assertNull(fi);
 +
 +    // Partial update
 +    addAndGetVersion(sdoc("id", "0", "nonexistent_field_i_dvo", map("set", "42")), null);
 +
 +    addAndGetVersion(sdoc("id", "1"), null);
 +    addAndGetVersion(sdoc("id", "1", "nonexistent_field_i_dvo", map("inc", "1")), null);
 +    addAndGetVersion(sdoc("id", "1", "nonexistent_field_i_dvo", map("inc", "1")), null);
 +
 +    assertU(commit());
 +
 +    assertQ(req("q", "*:*"), "//*[@numFound='2']");    
 +    assertQ(req("q", "nonexistent_field_i_dvo:42"), "//*[@numFound='1']");    
 +    assertQ(req("q", "nonexistent_field_i_dvo:2"), "//*[@numFound='1']");    
 +  }
 +
 +  @Test
 +  public void testOnlyPartialUpdatesBetweenCommits() throws Exception {
 +    // Full updates
 +    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first", "val1_i_dvo", "1", "val2_l_dvo", "1"), params());
 +    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second", "val1_i_dvo", "2", "val2_l_dvo", "2"), params());
 +    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third", "val1_i_dvo", "3", "val2_l_dvo", "3"), params());
 +    assertU(commit("softCommit", "false"));
 +
 +    assertQ(req("q", "*:*", "fl", "*,[docid]"), "//*[@numFound='3']");
 +
 +    int docid1 = getDocId("1");
 +    int docid2 = getDocId("2");
 +    int docid3 = getDocId("3");
 +
 +    int numPartialUpdates = 1 + random().nextInt(5000);
 +    for (int i=0; i<numPartialUpdates; i++) {
 +      version1 = addAndAssertVersion(version1, "id", "1", "val1_i_dvo", map("set", i));
 +      version2 = addAndAssertVersion(version2, "id", "2", "val1_i_dvo", map("inc", 1));
 +      version3 = addAndAssertVersion(version3, "id", "3", "val1_i_dvo", map("set", i));
 +
 +      version1 = addAndAssertVersion(version1, "id", "1", "val2_l_dvo", map("set", i));
 +      version2 = addAndAssertVersion(version2, "id", "2", "val2_l_dvo", map("inc", 1));
 +      version3 = addAndAssertVersion(version3, "id", "3", "val2_l_dvo", map("set", i));
 +    }
 +    assertU(commit("softCommit", "true"));
 +
 +    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*,[docid]"),
 +        "//*[@numFound='3']",
 +        "//result/doc[1]/int[@name='val1_i_dvo'][.='"+(numPartialUpdates-1)+"']",
 +        "//result/doc[2]/int[@name='val1_i_dvo'][.='"+(numPartialUpdates+2)+"']",
 +        "//result/doc[3]/int[@name='val1_i_dvo'][.='"+(numPartialUpdates-1)+"']",
 +        "//result/doc[1]/long[@name='val2_l_dvo'][.='"+(numPartialUpdates-1)+"']",
 +        "//result/doc[2]/long[@name='val2_l_dvo'][.='"+(numPartialUpdates+2)+"']",
 +        "//result/doc[3]/long[@name='val2_l_dvo'][.='"+(numPartialUpdates-1)+"']",
 +        "//result/doc[1]/int[@name='[docid]'][.='"+docid1+"']",
 +        "//result/doc[2]/int[@name='[docid]'][.='"+docid2+"']",
 +        "//result/doc[3]/int[@name='[docid]'][.='"+docid3+"']",
 +        "//result/doc[1]/long[@name='_version_'][.='" + version1 + "']",
 +        "//result/doc[2]/long[@name='_version_'][.='" + version2 + "']",
 +        "//result/doc[3]/long[@name='_version_'][.='" + version3 + "']"
 +        );
 +  }
 +
 +  /**
 +   * Useful to store the state of an expected document into an in-memory model
 +   * representing the index.
 +   */
 +  private static class DocInfo {
 +    public final long version;
 +    public final Long value;
 +
 +    public DocInfo(long version, Long val) {
 +      this.version = version;
 +      this.value = val;
 +    }
 +
 +    @Override
 +    public String toString() {
 +      return "["+version+", "+value+"]";
 +    }
 +  }
 +
 +  /** @see #checkReplay */
 +  @Test
 +  public void testReplay_AfterInitialAddMixOfIncAndSet() throws Exception {
 +    checkReplay("val2_l_dvo",
 +        //
 +        sdoc("id", "0", "val2_l_dvo", 3000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 3)),
 +        HARDCOMMIT,
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 5)),
 +        sdoc("id", "1", "val2_l_dvo", 2000000000L),
 +        sdoc("id", "1", "val2_l_dvo", map("set", 2000000002L)),
 +        sdoc("id", "1", "val2_l_dvo", map("set", 3000000000L)),
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 7)),
 +        sdoc("id", "1", "val2_l_dvo", map("set", 7000000000L)),
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 11)),
 +        sdoc("id", "2", "val2_l_dvo", 2000000000L),
 +        HARDCOMMIT,
 +        sdoc("id", "2", "val2_l_dvo", map("set", 3000000000L)),
 +        HARDCOMMIT);
 +  }
 +
 +  /** @see #checkReplay */
 +  @Test
 +  public void testReplay_AfterInitialAddMixOfIncAndSetAndFullUpdates() throws Exception {
 +    checkReplay("val2_l_dvo",
 +        //
 +        sdoc("id", "0", "val2_l_dvo", 3000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000003L)),
 +        HARDCOMMIT,
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000008L)),
 +        sdoc("id", "1", "val2_l_dvo", 2000000000L),
 +        sdoc("id", "1", "val2_l_dvo", map("inc", 2)),
 +        sdoc("id", "1", "val2_l_dvo", 3000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000015L)),
 +        sdoc("id", "1", "val2_l_dvo", 7000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000026L)),
 +        sdoc("id", "2", "val2_l_dvo", 2000000000L),
 +        HARDCOMMIT,
 +        sdoc("id", "2", "val2_l_dvo", 3000000000L),
 +        HARDCOMMIT);
 +  }
 +
 +  /** @see #checkReplay */
 +  @Test
 +  public void testReplay_AllUpdatesAfterInitialAddAreInc() throws Exception {
 +    checkReplay("val2_l_dvo",
 +        //
 +        sdoc("id", "0", "val2_l_dvo", 3000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 3)),
 +        HARDCOMMIT,
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 5)),
 +        sdoc("id", "1", "val2_l_dvo", 2000000000L),
 +        sdoc("id", "1", "val2_l_dvo", map("inc", 2)),
 +        sdoc("id", "1", "val2_l_dvo", 3000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 7)),
 +        sdoc("id", "1", "val2_l_dvo", 7000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("inc", 11)),
 +        sdoc("id", "2", "val2_l_dvo", 2000000000L),
 +        HARDCOMMIT,
 +        sdoc("id", "2", "val2_l_dvo", 3000000000L),
 +        HARDCOMMIT);
 +  }
 +
 +  /** @see #checkReplay */
 +  @Test
 +  public void testReplay_AllUpdatesAfterInitialAddAreSets() throws Exception {
 +    checkReplay("val2_l_dvo",
 +        //
 +        sdoc("id", "0", "val2_l_dvo", 3000000000L),
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000003L)),
 +        HARDCOMMIT,
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000008L)),
 +        sdoc("id", "1", "val2_l_dvo", 2000000000L),
 +        sdoc("id", "1", "val2_l_dvo", map("set", 2000000002L)),
 +        sdoc("id", "1", "val2_l_dvo", map("set", 3000000000L)),
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000015L)),
 +        sdoc("id", "1", "val2_l_dvo", map("set", 7000000000L)),
 +        sdoc("id", "0", "val2_l_dvo", map("set", 3000000026L)),
 +        sdoc("id", "2", "val2_l_dvo", 2000000000L),
 +        HARDCOMMIT,
 +        sdoc("id", "2", "val2_l_dvo", map("set", 3000000000L)),
 +        HARDCOMMIT
 +        );
 +  }
 +  
 +  /** @see #checkReplay */
 +  @Test
 +  public void testReplay_MixOfInplaceAndNonInPlaceAtomicUpdates() throws Exception {
 +    checkReplay("inplace_l_dvo",
 +                //
 +                sdoc("id", "3", "inplace_l_dvo", map("inc", -13)),
 +                sdoc("id", "3", "inplace_l_dvo", map("inc", 19),    "regular_l", map("inc", -17)),
 +                sdoc("id", "1",                                     "regular_l", map("inc", -19)),
 +                sdoc("id", "3", "inplace_l_dvo", map("inc", -11)),
 +                sdoc("id", "2", "inplace_l_dvo", map("set", 28)),
 +                HARDCOMMIT,
 +                sdoc("id", "2", "inplace_l_dvo", map("inc", 45)),
 +                sdoc("id", "3", "inplace_l_dvo", map("set", 72)),
 +                sdoc("id", "2",                                     "regular_l", map("inc", -55)),
 +                sdoc("id", "2", "inplace_l_dvo", -48,               "regular_l", 159),
 +                sdoc("id", "3", "inplace_l_dvo", 52,                "regular_l", 895),
 +                sdoc("id", "2", "inplace_l_dvo", map("inc", 19)),
 +                sdoc("id", "3", "inplace_l_dvo", map("inc", -264),  "regular_l", map("inc", -207)),
 +                sdoc("id", "3", "inplace_l_dvo", -762,              "regular_l", 272),
 +                SOFTCOMMIT);
 +  }
 +  
 +  @Test
 +  public void testReplay_SetOverriddenWithNoValueThenInc() throws Exception {
 +    final String inplaceField = "inplace_l_dvo"; 
 +    checkReplay(inplaceField,
 +                //
 +                sdoc("id", "1", inplaceField, map("set", 555L)),
 +                SOFTCOMMIT,
 +                sdoc("id", "1", "regular_l", 666L), // NOTE: no inplaceField, regular add w/overwrite 
 +                sdoc("id", "1", inplaceField, map("inc", -77)),
 +                HARDCOMMIT);
 +  }
 +
 +  /** 
 +   * Simple enum for randomizing a type of update.
 +   * Each enum value has an associated probability, and the class has built in sanity checks 
 +   * that the total is 100%
 +   * 
 +   * @see RandomUpdate#pick
 +   * @see #checkRandomReplay
 +   */
 +  private static enum RandomUpdate {
 +    HARD_COMMIT(5), 
 +    SOFT_COMMIT(5),
 +
 +    /** doc w/o the inplaceField, atomic update on some other (non-inplace) field */
 +    ATOMIC_NOT_INPLACE(5),
 +    
 +    /** atomic update of a doc w/ inc on both inplaceField *AND* non-inplace field */
 +    ATOMIC_INPLACE_AND_NOT_INPLACE(10), 
 +
 +    
 +    /** atomic update of a doc w/ set inplaceField */
 +    ATOMIC_INPLACE_SET(25),
 +    /** atomic update of a doc w/ inc inplaceField */
 +    ATOMIC_INPLACE_INC(25), 
 +    
 +    /** doc w/o the inplaceField, normal add */
 +    ADD_NO_INPLACE_VALUE(5),
 +    /** a non atomic update of a doc w/ new inplaceField value */
 +    ADD_INPLACE_VALUE(20); 
 +    
 +    private RandomUpdate(int odds) {
 +      this.odds = odds;
 +    }
 +    public final int odds;
 +
 +    static { // sanity check odds add up to 100%
 +      int total = 0;
 +      for (RandomUpdate candidate : RandomUpdate.values()) {
 +        total += candidate.odds;
 +      }
 +      assertEquals("total odds doesn't equal 100", 100, total);
 +    }
 +
 +    /** pick a random type of RandomUpdate */
 +    public static final RandomUpdate pick(Random r) {
 +      final int target = TestUtil.nextInt(r, 1, 100);
 +      int cumulative_odds = 0;
 +      for (RandomUpdate candidate : RandomUpdate.values()) {
 +        cumulative_odds += candidate.odds;
 +        if (target <= cumulative_odds) {
 +          return candidate;
 +        }
 +      }
 +      fail("how did we not find a candidate? target=" + target + ", cumulative_odds=" + cumulative_odds);
 +      return null; // compiler mandated return
 +    }
 +  }
 +  
 +  /** @see #checkRandomReplay */
 +  @Test
 +  public void testReplay_Random_ManyDocsManyUpdates() throws Exception {
 +    
 +    // build up a random list of updates
 +    final int maxDocId = atLeast(50);
 +    final int numUpdates = maxDocId * 3;
 +    checkRandomReplay(maxDocId, numUpdates);
 +  }
 +  
 +  /** @see #checkRandomReplay */
 +  @Test
 +  public void testReplay_Random_FewDocsManyUpdates() throws Exception {
 +    
 +    // build up a random list of updates
 +    final int maxDocId = atLeast(3);
 +    final int numUpdates = maxDocId * 50;
 +    checkRandomReplay(maxDocId, numUpdates);
 +  }
 +  
 +  /** @see #checkRandomReplay */
 +  @Test
 +  public void testReplay_Random_FewDocsManyShortSequences() throws Exception {
 +    
 +    // build up a random list of updates
 +    final int numIters = atLeast(50);
 +    
 +    for (int i = 0; i < numIters; i++) {
 +      final int maxDocId = atLeast(3);
 +      final int numUpdates = maxDocId * 5;
 +      checkRandomReplay(maxDocId, numUpdates);
 +      deleteAllAndCommit();
 +    }
 +  }
 +
 +
 +  /** 
 +   * @see #checkReplay 
 +   * @see RandomUpdate
 +   */
 +  public void checkRandomReplay(final int maxDocId, final int numCmds) throws Exception {
 +    
 +    final String not_inplaceField = "regular_l";
 +    final String inplaceField = "inplace_l_dvo"; 
 +
 +    final Object[] cmds = new Object[numCmds];
 +    for (int iter = 0; iter < numCmds; iter++) {
 +      final int id = TestUtil.nextInt(random(), 1, maxDocId);
 +      final RandomUpdate update = RandomUpdate.pick(random());
 +
 +      switch (update) {
 +        
 +      case HARD_COMMIT:
 +        cmds[iter] = HARDCOMMIT;
 +        break;
 +        
 +      case SOFT_COMMIT:
 +        cmds[iter] = SOFTCOMMIT;
 +        break;
 +
 +      case ATOMIC_NOT_INPLACE:
 +        // atomic update on non_inplaceField, w/o any value specified for inplaceField
 +        cmds[iter] = sdoc("id", id,
 +                          not_inplaceField, map("inc", random().nextInt()));
 +        break;
 +        
 +      case ATOMIC_INPLACE_AND_NOT_INPLACE:
 +        // atomic update of a doc w/ inc on both inplaceField and not_inplaceField
 +        cmds[iter] = sdoc("id", id,
 +                          inplaceField, map("inc", random().nextInt()),
 +                          not_inplaceField, map("inc", random().nextInt()));
 +        break;
 +
 +      case ATOMIC_INPLACE_SET:
 +        // atomic update of a doc w/ set inplaceField
 +        cmds[iter] = sdoc("id", id,
 +                          inplaceField, map("set", random().nextLong()));
 +        break;
 +
 +      case ATOMIC_INPLACE_INC:
 +        // atomic update of a doc w/ inc inplaceField
 +        cmds[iter] = sdoc("id", id,
 +                          inplaceField, map("inc", random().nextInt()));
 +        break;
 +
 +      case ADD_NO_INPLACE_VALUE:
 +        // regular add of doc w/o the inplaceField, but does include non_inplaceField
 +        cmds[iter] = sdoc("id", id,
 +                          not_inplaceField, random().nextLong());
 +        break;
 +
 +      case ADD_INPLACE_VALUE:
 +        // a non atomic update of a doc w/ new inplaceField value
 +        cmds[iter] = sdoc("id", id,
 +                          inplaceField, random().nextLong(),
 +                          not_inplaceField, random().nextLong());
 +        break;
 +        
 +      default:
 +        fail("WTF is this? ... " + update);
 +      }
 +      
 +      assertNotNull(cmds[iter]); // sanity check switch
 +    }
 +
 +    checkReplay(inplaceField, cmds);
 +  }
 +  
 +  /** sentinal object for {@link #checkReplay} */
 +  public Object SOFTCOMMIT = new Object() { public String toString() { return "SOFTCOMMIT"; } };
 +  /** sentinal object for {@link #checkReplay} */
 +  public Object HARDCOMMIT = new Object() { public String toString() { return "HARDCOMMIT"; } };
 +
 +  /**
 +   * Executes a sequence of commands against Solr, while tracking the expected value of a specified 
 +   * <code>valField</code> Long field (presumably that only uses docvalues) against an in memory model 
 +   * maintained in parallel (for the purpose of testing the correctness of in-place updates..
 +   *
 +   * <p>
 +   * A few restrictions are placed on the {@link SolrInputDocument}s that can be included when using 
 +   * this method, in order to keep the in-memory model management simple:
 +   * </p>
 +   * <ul>
 +   *  <li><code>id</code> must be uniqueKey field</li>
 +   *  <li><code>id</code> may have any FieldType, but all values must be parsable as Integers</li>
 +   *  <li><code>valField</code> must be a single valued field</li>
 +   *  <li>All values in the <code>valField</code> must either be {@link Number}s, or Maps containing 
 +   *      atomic updates ("inc" or "set") where the atomic value is a {@link Number}</li>
 +   * </ul>
 +   * 
 +   * @param valField the field to model
 +   * @param commands A sequence of Commands which can either be SolrInputDocuments 
 +   *                 (regular or containing atomic update Maps)
 +   *                 or one of the {@link TestInPlaceUpdatesStandalone#HARDCOMMIT} or {@link TestInPlaceUpdatesStandalone#SOFTCOMMIT} sentinal objects.
 +   */
 +  public void checkReplay(final String valField, Object... commands) throws Exception {
 +    
 +    HashMap<Integer, DocInfo> model = new LinkedHashMap<>();
 +    HashMap<Integer, DocInfo> committedModel = new LinkedHashMap<>();
 +
 +    // by default, we only check the committed model after a commit
 +    // of if the number of total commands is relatively small.
 +    //
 +    // (in theory, there's no reason to check the committed model unless we know there's been a commit
 +    // but for smaller tests the overhead of doing so is tiny, so we might as well)
 +    //
 +    // if some test seed fails, and you want to force the committed model to be checked
 +    // after every command, just temporaribly force this variable to true...
 +    boolean checkCommittedModel = (commands.length < 50);
 +    
 +    for (Object cmd : commands) {
 +      if (cmd == SOFTCOMMIT) {
 +        assertU(commit("softCommit", "true"));
 +        committedModel = new LinkedHashMap(model);
 +        checkCommittedModel = true;
 +      } else if (cmd == HARDCOMMIT) {
 +        assertU(commit("softCommit", "false"));
 +        committedModel = new LinkedHashMap(model);
 +        checkCommittedModel = true;
 +      } else {
 +        assertNotNull("null command in checkReplay", cmd);
 +        assertTrue("cmd is neither sentinal (HARD|SOFT)COMMIT object, nor Solr doc: " + cmd.getClass(),
 +                   cmd instanceof SolrInputDocument);
 +        
 +        final SolrInputDocument sdoc = (SolrInputDocument) cmd;
 +        final int id = Integer.parseInt(sdoc.getFieldValue("id").toString());
 +        
 +        final DocInfo previousInfo = model.get(id);
 +        final Long previousValue = (null == previousInfo) ? null : previousInfo.value;
 +        
 +        final long version = addAndGetVersion(sdoc, null);
 +        
 +        final Object val = sdoc.getFieldValue(valField);
 +        if (val instanceof Map) {
 +          // atomic update of the field we're modeling
 +          
 +          Map<String,?> atomicUpdate = (Map) val;
 +          assertEquals(sdoc.toString(), 1, atomicUpdate.size());
 +          if (atomicUpdate.containsKey("inc")) {
 +            // Solr treats inc on a non-existing doc (or doc w/o existing value) as if existing value is 0
 +            final long base = (null == previousValue) ? 0L : previousValue;
 +            model.put(id, new DocInfo(version,
 +                                      base + ((Number)atomicUpdate.get("inc")).longValue()));
 +          } else if (atomicUpdate.containsKey("set")) {
 +            model.put(id, new DocInfo(version, ((Number)atomicUpdate.get("set")).longValue()));
 +          } else {
 +            fail("wtf update is this? ... " + sdoc);
 +          }
 +        } else if (null == val) {
 +          // the field we are modeling is not mentioned in this update, It's either...
 +          //
 +          // a) a regular update of some other fields (our model should have a null value)
 +          // b) an atomic update of some other field (keep existing value in model)
 +          //
 +          // for now, assume it's atomic and we're going to keep our existing value...
 +          Long newValue = (null == previousInfo) ? null : previousInfo.value;
 +          for (SolrInputField field : sdoc) {
 +            if (! ( "id".equals(field.getName()) || (field.getValue() instanceof Map)) ) {
 +              // not an atomic update, newValue in model should be null
 +              newValue = null;
 +              break;
 +            }
 +          }
 +          model.put(id, new DocInfo(version, newValue));
 +          
 +        } else {
 +          // regular replacement of the value in the field we're modeling
 +          
 +          assertTrue("Model field value is not a Number: " + val.getClass(), val instanceof Number);
 +          model.put(id, new DocInfo(version, ((Number)val).longValue()));
 +        }
 +      }
 +
 +      // after every op, check the model(s)
 +      
 +      // RTG to check the values for every id against the model
 +      for (Map.Entry<Integer, DocInfo> entry : model.entrySet()) {
 +        final Long expected = entry.getValue().value;
 +        assertEquals(expected, client.getById(String.valueOf(entry.getKey())).getFirstValue(valField));
 +      }
 +
 +      // search to check the values for every id in the committed model
 +      if (checkCommittedModel) {
 +        final int numCommitedDocs = committedModel.size();
 +        String[] xpaths = new String[1 + numCommitedDocs];
 +        int i = 0;
 +        for (Map.Entry<Integer, DocInfo> entry : committedModel.entrySet()) {
 +          Integer id = entry.getKey();
 +          Long expected = entry.getValue().value;
 +          if (null != expected) {
 +            xpaths[i] = "//result/doc[./str='"+id+"'][./long='"+expected+"']";
 +          } else {
 +            xpaths[i] = "//result/doc[./str='"+id+"'][not(./long)]";
 +          }           
 +          i++;
 +        }
 +        xpaths[i] = "//*[@numFound='"+numCommitedDocs+"']";
 +        assertQ(req("q", "*:*",
 +                    "fl", "id," + valField,
 +                    "rows", ""+numCommitedDocs),
 +                xpaths);
 +      }
 +    }
 +  }
 +
 +  @Test
 +  public void testMixedInPlaceAndNonInPlaceAtomicUpdates() throws Exception {
 +    SolrDocument rtgDoc = null;
 +    long version1 = addAndGetVersion(sdoc("id", "1", "inplace_updatable_float", "100", "stored_i", "100"), params());
 +
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", "1"), "stored_i", map("inc", "1"));
 +    rtgDoc = client.getById("1");
 +    assertEquals(101, rtgDoc.getFieldValue("stored_i"));
 +    assertEquals(101.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
 +    
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("inc", "1"));
 +    rtgDoc = client.getById("1");
 +    assertEquals(101, rtgDoc.getFieldValue("stored_i"));
 +    assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
 +
 +    version1 = addAndAssertVersion(version1, "id", "1", "stored_i", map("inc", "1"));
 +    rtgDoc = client.getById("1");
 +    assertEquals(102, rtgDoc.getFieldValue("stored_i"));
 +    assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
 +
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "*:*", "sort", "id asc", "fl", "*"),
 +            "//*[@numFound='1']",
 +            "//result/doc[1]/float[@name='inplace_updatable_float'][.='102.0']",
 +            "//result/doc[1]/int[@name='stored_i'][.='102']",
 +            "//result/doc[1]/long[@name='_version_'][.='" + version1 + "']"
 +            );
 +
 +    // recheck RTG after commit
 +    rtgDoc = client.getById("1");
 +    assertEquals(102, rtgDoc.getFieldValue("stored_i"));
 +    assertEquals(102.0f, rtgDoc.getFieldValue("inplace_updatable_float"));
 +  }
 +
 +  /** 
 +   * @see #callComputeInPlaceUpdatableFields
 +   * @see AtomicUpdateDocumentMerger#computeInPlaceUpdatableFields 
 +   */
 +  @Test
 +  public void testComputeInPlaceUpdatableFields() throws Exception {
 +    Set<String> inPlaceUpdatedFields = new HashSet<String>();
 +
 +    // these asserts should hold true regardless of type, or wether the field has a default
 +    List<String> fieldsToCheck = Arrays.asList("inplace_updatable_float",
 +                                               "inplace_updatable_int",
 +                                               "inplace_updatable_float_with_default",
 +                                               "inplace_updatable_int_with_default");
 +    Collections.shuffle(fieldsToCheck, random()); // ... and regardless of order checked
 +    for (String field : fieldsToCheck) {
 +      // In-place updatable field updated before it exists SHOULD NOT BE in-place updated:
 +      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                                    field, map("set", 10)));
 +      assertFalse(field, inPlaceUpdatedFields.contains(field));
 +      
 +      // In-place updatable field updated after it exists SHOULD BE in-place updated:
 +      addAndGetVersion(sdoc("id", "1", field, "0"), params()); // setting up the dv
 +      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                                    field, map("set", 10)));
 +      assertTrue(field, inPlaceUpdatedFields.contains(field));
 +
 +      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                                    field, map("inc", 10)));
 +      assertTrue(field, inPlaceUpdatedFields.contains(field));
 +
 +      final String altFieldWithDefault = field.contains("float") ?
 +        "inplace_updatable_int_with_default" : "inplace_updatable_int_with_default";
 +      
 +      // Updating an in-place updatable field (with a default) for the first time.
 +      // DV for it should have been already created when first document was indexed (above),
 +      // since it has a default value
 +      inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                                    altFieldWithDefault, map("set", 10)));
 +      assertTrue(field + " -> " + altFieldWithDefault, inPlaceUpdatedFields.contains(altFieldWithDefault));
 +      
 +      deleteAllAndCommit();
 +    }
 +  
 +    // Non in-place updates
 +    addAndGetVersion(sdoc("id", "1", "stored_i", "0"), params()); // setting up the dv
 +    assertTrue("stored field updated",
 +               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                      "stored_i", map("inc", 1))).isEmpty());
 +    
 +    assertTrue("full document update",
 +               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                      "inplace_updatable_int_with_default", "100")).isEmpty());
 +  
 +    assertTrue("non existent dynamic dv field updated first time",
 +               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                      "new_updatable_int_i_dvo", map("set", 10))).isEmpty());
 +    
 +    // After adding a full document with the dynamic dv field, in-place update should work
 +    addAndGetVersion(sdoc("id", "2", "new_updatable_int_i_dvo", "0"), params()); // setting up the dv
 +    if (random().nextBoolean()) {
 +      assertU(commit("softCommit", "false"));
 +    }
 +    inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "2", "_version_", 42L,
 +                                                                  "new_updatable_int_i_dvo", map("set", 10)));
 +    assertTrue(inPlaceUpdatedFields.contains("new_updatable_int_i_dvo"));
 +
 +    // for copy fields, regardless of wether the source & target support inplace updates,
 +    // it won't be inplace if the DVs don't exist yet...
 +    assertTrue("inplace fields should be empty when doc has no copyfield src values yet",
 +               callComputeInPlaceUpdatableFields(sdoc("id", "1", "_version_", 42L,
 +                                                      "copyfield1_src__both_updatable", map("set", 1),
 +                                                      "copyfield2_src__only_src_updatable", map("set", 2))).isEmpty());
 +
 +    // now add a doc that *does* have the src field for each copyfield...
 +    addAndGetVersion(sdoc("id", "3",
 +                          "copyfield1_src__both_updatable", -13,
 +                          "copyfield2_src__only_src_updatable", -15), params()); 
 +    if (random().nextBoolean()) {
 +      assertU(commit("softCommit", "false"));
 +    }
 +    
 +    // If a supported dv field has a copyField target which is supported, it should be an in-place update
 +    inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "3", "_version_", 42L,
 +                                                                  "copyfield1_src__both_updatable", map("set", 10)));
 +    assertTrue(inPlaceUpdatedFields.contains("copyfield1_src__both_updatable"));
 +
 +    // If a supported dv field has a copyField target which is not supported, it should not be an in-place update
 +    inPlaceUpdatedFields = callComputeInPlaceUpdatableFields(sdoc("id", "3", "_version_", 42L,
 +                                                                  "copyfield2_src__only_src_updatable", map("set", 10)));
 +    assertTrue(inPlaceUpdatedFields.isEmpty());
 +  }
 +
 +  @Test
 +  /**
 +   *  Test the @see {@link AtomicUpdateDocumentMerger#doInPlaceUpdateMerge(AddUpdateCommand,Set<String>)} 
 +   *  method is working fine
 +   */
 +  public void testDoInPlaceUpdateMerge() throws Exception {
 +    long version1 = addAndGetVersion(sdoc("id", "1", "title_s", "first"), null);
 +    long version2 = addAndGetVersion(sdoc("id", "2", "title_s", "second"), null);
 +    long version3 = addAndGetVersion(sdoc("id", "3", "title_s", "third"), null);
 +    assertU(commit("softCommit", "false"));
 +    assertQ(req("q", "*:*"), "//*[@numFound='3']");
 +
 +    // Adding a few in-place updates
 +    version1 = addAndAssertVersion(version1, "id", "1", "inplace_updatable_float", map("set", 200));
 +
 +    // Test the AUDM.doInPlaceUpdateMerge() method is working fine
 +    try (SolrQueryRequest req = req()) {
 +      AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc("id", "1", "_version_", 42L,
 +                                                             "inplace_updatable_float", map("inc", 10)));
 +      AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
 +      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.computeInPlaceUpdatableFields(cmd)));
 +      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
 +      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
 +      assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
 +      // in-place merged doc shouldn't have non-inplace fields from the index/tlog
 +      assertFalse(cmd.getSolrInputDocument().containsKey("title_s"));
 +      assertEquals(version1, cmd.prevVersion);
 +    }
 +    
 +    // do a commit, and the same results should be repeated
 +    assertU(commit("softCommit", "false"));
 +
 +    // Test the AUDM.doInPlaceUpdateMerge() method is working fine
 +    try (SolrQueryRequest req = req()) {
 +      AddUpdateCommand cmd = buildAddUpdateCommand(req, sdoc("id", "1", "_version_", 42L,
 +                                                             "inplace_updatable_float", map("inc", 10)));
 +      AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req);
 +      assertTrue(docMerger.doInPlaceUpdateMerge(cmd, AtomicUpdateDocumentMerger.computeInPlaceUpdatableFields(cmd)));
 +      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
 +      assertEquals(42L, cmd.getSolrInputDocument().getFieldValue("_version_"));
 +      assertEquals(210f, cmd.getSolrInputDocument().getFieldValue("inplace_updatable_float"));
 +      // in-place merged doc shouldn't have non-inplace fields from the index/tlog
 +      assertFalse(cmd.getSolrInputDocument().containsKey("title_s")); 
 +      assertEquals(version1, cmd.prevVersion);
 +    }
 +  }
 +  
 +  /** 
 +   * Helper method that sets up a req/cmd to run {@link AtomicUpdateDocumentMerger#computeInPlaceUpdatableFields} 
 +   * on the specified solr input document.
 +   */
 +  private static Set<String> callComputeInPlaceUpdatableFields(final SolrInputDocument sdoc) throws Exception {
 +    try (SolrQueryRequest req = req()) {
 +      AddUpdateCommand cmd = new AddUpdateCommand(req);
 +      cmd.solrDoc = sdoc;
 +      assertTrue(cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD));
 +      cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString()));
 +      return AtomicUpdateDocumentMerger.computeInPlaceUpdatableFields(cmd);
 +    }
 +  }
 +}


[41/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7627: Add #intersect(CompiledAutomaton) to Sorted*DocValues

Posted by ho...@apache.org.
LUCENE-7627: Add #intersect(CompiledAutomaton) to Sorted*DocValues


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

Branch: refs/heads/jira/solr-5944
Commit: 8fa0a8dd1e5eb3a5e2553c346372d203d00e575b
Parents: 53d5af1
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Jan 11 12:07:11 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 14 09:40:19 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../apache/lucene/index/SortedDocValues.java    | 22 ++++++++++++++
 .../apache/lucene/index/SortedSetDocValues.java | 22 ++++++++++++++
 .../index/BaseDocValuesFormatTestCase.java      | 32 ++++++++++++++++++++
 4 files changed, 80 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 30943d2..58201d6 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -126,6 +126,10 @@ New features
   concurrently across all segments in the index (Emmanuel Keller via
   Mike McCandless)
 
+* LUCENE-7627: Added .intersect methods to SortedDocValues and 
+  SortedSetDocValues to allow filtering their TermsEnums with a
+  CompiledAutomaton (Alan Woodward, Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
index e2d7dfd..087e487 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A per-document byte[] with presorted values.  This is fundamentally an
@@ -110,4 +111,25 @@ public abstract class SortedDocValues extends BinaryDocValues {
     return new SortedDocValuesTermsEnum(this);
   }
 
+  /**
+   * Returns a {@link TermsEnum} over the values, filtered by a {@link CompiledAutomaton}
+   * The enum supports {@link TermsEnum#ord()}.
+   */
+  public TermsEnum intersect(CompiledAutomaton automaton) throws IOException {
+    TermsEnum in = termsEnum();
+    switch (automaton.type) {
+      case NONE:
+        return TermsEnum.EMPTY;
+      case ALL:
+        return in;
+      case SINGLE:
+        return new SingleTermsEnum(in, automaton.term);
+      case NORMAL:
+        return new AutomatonTermsEnum(in, automaton);
+      default:
+        // unreachable
+        throw new RuntimeException("unhandled case");
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
index 6d02c25..9e1c6a3 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A multi-valued version of {@link SortedDocValues}.
@@ -102,4 +103,25 @@ public abstract class SortedSetDocValues extends DocValuesIterator {
   public TermsEnum termsEnum() throws IOException {
     return new SortedSetDocValuesTermsEnum(this);
   }
+
+  /**
+   * Returns a {@link TermsEnum} over the values, filtered by a {@link CompiledAutomaton}
+   * The enum supports {@link TermsEnum#ord()}.
+   */
+  public TermsEnum intersect(CompiledAutomaton automaton) throws IOException {
+    TermsEnum in = termsEnum();
+    switch (automaton.type) {
+      case NONE:
+        return TermsEnum.EMPTY;
+      case ALL:
+        return in;
+      case SINGLE:
+        return new SingleTermsEnum(in, automaton.term);
+      case NORMAL:
+        return new AutomatonTermsEnum(in, automaton);
+      default:
+        // unreachable
+        throw new RuntimeException("unhandled case");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8fa0a8dd/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
index d55f212..8cb6665 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
@@ -67,6 +67,8 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
 
 import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
@@ -906,6 +908,21 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     termsEnum.seekExact(2);
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
+
+    // NORMAL automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp(".*l.*").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertEquals("world", termsEnum.next().utf8ToString());
+    assertEquals(2, termsEnum.ord());
+    assertNull(termsEnum.next());
+
+    // SINGLE automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp("hello").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertNull(termsEnum.next());
+
     ireader.close();
     directory.close();
   }
@@ -2057,6 +2074,21 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     termsEnum.seekExact(2);
     assertEquals("world", termsEnum.term().utf8ToString());
     assertEquals(2, termsEnum.ord());
+
+    // NORMAL automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp(".*l.*").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertEquals("world", termsEnum.next().utf8ToString());
+    assertEquals(2, termsEnum.ord());
+    assertNull(termsEnum.next());
+
+    // SINGLE automaton
+    termsEnum = dv.intersect(new CompiledAutomaton(new RegExp("hello").toAutomaton()));
+    assertEquals("hello", termsEnum.next().utf8ToString());
+    assertEquals(1, termsEnum.ord());
+    assertNull(termsEnum.next());
+
     ireader.close();
     directory.close();
   }


[02/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7611: Remove queries dependency from suggester module

Posted by ho...@apache.org.
LUCENE-7611: Remove queries dependency from suggester module


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

Branch: refs/heads/jira/solr-5944
Commit: 8f4fee3ad1c0027587d0de96f59cf61b2df67bc8
Parents: ce8b678
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 7 12:32:17 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 13:07:13 2017 +0000

----------------------------------------------------------------------
 dev-tools/idea/lucene/suggest/suggest.iml       |  1 -
 .../lucene/search/DoubleValuesSource.java       | 27 ++++++++++++
 lucene/suggest/build.xml                        |  3 +-
 .../suggest/DocumentValueSourceDictionary.java  | 45 --------------------
 .../DocumentValueSourceDictionaryTest.java      | 25 +++++------
 5 files changed, 38 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/dev-tools/idea/lucene/suggest/suggest.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/suggest/suggest.iml b/dev-tools/idea/lucene/suggest/suggest.iml
index 576ea8b..5e58bc2 100644
--- a/dev-tools/idea/lucene/suggest/suggest.iml
+++ b/dev-tools/idea/lucene/suggest/suggest.iml
@@ -13,7 +13,6 @@
     <orderEntry type="sourceFolder" forTests="false" />
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
-    <orderEntry type="module" module-name="queries" />
     <orderEntry type="module" module-name="analysis-common" />
     <orderEntry type="module" module-name="lucene-core" />
   </component>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
index 4ac8fc1..d4be4e9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
@@ -150,6 +150,33 @@ public abstract class DoubleValuesSource {
   };
 
   /**
+   * Creates a DoubleValuesSource that always returns a constant value
+   */
+  public static DoubleValuesSource constant(double value) {
+    return new DoubleValuesSource() {
+      @Override
+      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        return new DoubleValues() {
+          @Override
+          public double doubleValue() throws IOException {
+            return value;
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return true;
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return false;
+      }
+    };
+  }
+
+  /**
    * Returns a DoubleValues instance that wraps scores returned by a Scorer
    */
   public static DoubleValues fromScorer(Scorer scorer) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/suggest/build.xml
----------------------------------------------------------------------
diff --git a/lucene/suggest/build.xml b/lucene/suggest/build.xml
index 5babe06..bc4bed5 100644
--- a/lucene/suggest/build.xml
+++ b/lucene/suggest/build.xml
@@ -30,11 +30,10 @@
 
   <path id="classpath">
     <pathelement path="${analyzers-common.jar}"/>
-    <pathelement path="${queries.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
-  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 656dc04..2291ac9 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -23,7 +23,6 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.LongValues;
 import org.apache.lucene.search.LongValuesSource;
 
@@ -63,21 +62,6 @@ import org.apache.lucene.search.LongValuesSource;
 public class DocumentValueSourceDictionary extends DocumentDictionary {
   
   private final LongValuesSource weightsValueSource;
-  
-  /**
-   * Creates a new dictionary with the contents of the fields named <code>field</code>
-   * for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
-   * for the associated contexts and uses the <code>weightsValueSource</code> supplied 
-   * to determine the score.
-   *
-   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String, String)}
-   */
-  @Deprecated
-  public DocumentValueSourceDictionary(IndexReader reader, String field,
-                                       ValueSource weightsValueSource, String payload, String contexts) {
-    super(reader, field, null, payload, contexts);
-    this.weightsValueSource = weightsValueSource.asLongValuesSource();
-  }
 
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
@@ -94,21 +78,6 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>
    * for the terms, <code>payloadField</code> for the corresponding payloads
-   * and uses the <code>weightsValueSource</code> supplied to determine the 
-   * score.
-   *
-   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource, String)}
-   */
-  @Deprecated
-  public DocumentValueSourceDictionary(IndexReader reader, String field,
-                                       ValueSource weightsValueSource, String payload) {
-    super(reader, field, null, payload);
-    this.weightsValueSource = weightsValueSource.asLongValuesSource();
-  }
-
-  /**
-   * Creates a new dictionary with the contents of the fields named <code>field</code>
-   * for the terms, <code>payloadField</code> for the corresponding payloads
    * and uses the <code>weightsValueSource</code> supplied to determine the
    * score.
    */
@@ -117,20 +86,6 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
     super(reader, field, null, payload);
     this.weightsValueSource = weightsValueSource;
   }
-  
-  /** 
-   * Creates a new dictionary with the contents of the fields named <code>field</code>
-   * for the terms and uses the <code>weightsValueSource</code> supplied to determine the 
-   * score.
-   *
-   * @deprecated Use {@link #DocumentValueSourceDictionary(IndexReader, String, LongValuesSource)}
-   */
-  @Deprecated
-  public DocumentValueSourceDictionary(IndexReader reader, String field,
-                                       ValueSource weightsValueSource) {
-    super(reader, field, null, null);
-    this.weightsValueSource = weightsValueSource.asLongValuesSource();
-  }
 
   /**
    * Creates a new dictionary with the contents of the fields named <code>field</code>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8f4fee3a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index 92799cd..55970e4 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -39,10 +39,6 @@ import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
-import org.apache.lucene.queries.function.valuesource.SumFloatFunction;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.LongValues;
 import org.apache.lucene.search.LongValuesSource;
@@ -73,7 +69,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.commit();
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
 
     assertNull(inputIterator.next());
@@ -119,8 +115,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s, PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -227,8 +223,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s, PAYLOAD_FIELD_NAME, CONTEXTS_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -305,8 +301,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new SumFloatFunction(toAdd));
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2, WEIGHT_FIELD_NAME_3);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -390,9 +386,8 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IndexReader ir = DirectoryReader.open(dir);
     assertTrue("NumDocs should be > 0 but was " + ir.numDocs(), ir.numDocs() > 0);
     assertEquals(ir.numDocs(), docs.size());
-    ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2)};
-
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
+    LongValuesSource s = sum(WEIGHT_FIELD_NAME_1, WEIGHT_FIELD_NAME_2);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, s, PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
@@ -478,7 +473,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
 
     IndexReader ir = DirectoryReader.open(dir);
-    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
+    Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, LongValuesSource.constant(10), PAYLOAD_FIELD_NAME);
     InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {


[39/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7626: IndexWriter no longer accepts broken offsets

Posted by ho...@apache.org.
LUCENE-7626: IndexWriter no longer accepts broken offsets


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

Branch: refs/heads/jira/solr-5944
Commit: 64b86331c29d074fa7b257d65d3fda3b662bf96a
Parents: 5b3565e
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Jan 13 17:46:02 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Jan 13 17:46:02 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../miscellaneous/FixBrokenOffsetsFilter.java   |  78 ++++++++++++
 .../FixBrokenOffsetsFilterFactory.java          |  39 ++++++
 ...ache.lucene.analysis.util.TokenFilterFactory |   1 +
 .../TestFixBrokenOffsetsFilter.java             |  50 ++++++++
 .../apache/lucene/index/FixBrokenOffsets.java   | 125 +++++++++++++++++++
 .../java/org/apache/lucene/index/package.html   |  27 ++++
 .../lucene/index/TestFixBrokenOffsets.java      | 114 +++++++++++++++++
 .../lucene/index/index.630.brokenoffsets.zip    | Bin 0 -> 3203 bytes
 .../org/apache/lucene/index/CheckIndex.java     |  29 +++--
 .../lucene/index/DefaultIndexingChain.java      |  20 ++-
 .../org/apache/lucene/index/TestCheckIndex.java |   5 -
 .../search/highlight/TokenSourcesTest.java      |   2 +-
 .../lucene/search/TestTermAutomatonQuery.java   |   3 +
 .../index/BaseTermVectorsFormatTestCase.java    |  17 +--
 .../apache/lucene/index/BaseTestCheckIndex.java |  19 ---
 .../java/org/apache/lucene/util/TestUtil.java   |   4 +-
 .../apache/solr/schema/PreAnalyzedField.java    |  11 ++
 .../solr/index/hdfs/CheckHdfsIndexTest.java     |   5 -
 19 files changed, 480 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 040f4e0..30943d2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -29,6 +29,9 @@ API Changes
 
 Bug Fixes
 
+* LUCENE-7626: IndexWriter will no longer accept broken token offsets
+  (Mike McCandless)
+
 Improvements
 
 * LUCENE-7489: Better storage of sparse doc-values fields with the default

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilter.java
new file mode 100644
index 0000000..b0a6b1d
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilter.java
@@ -0,0 +1,78 @@
+/*
+ * 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.lucene.analysis.miscellaneous;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+
+/** 
+ * A filter to correct offsets that illegally go backwards.
+ *
+ * @deprecated Fix the token filters that create broken offsets in the first place.
+ */
+@Deprecated
+public final class FixBrokenOffsetsFilter extends TokenFilter {
+
+  private int lastStartOffset;
+  private int lastEndOffset;
+
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  public FixBrokenOffsetsFilter(TokenStream in) {
+    super(in);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (input.incrementToken() == false) {
+      return false;
+    }
+    fixOffsets();
+    return true;
+  }
+
+  @Override
+  public void end() throws IOException {
+    super.end();
+    fixOffsets();
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    lastStartOffset = 0;
+    lastEndOffset = 0;
+  }
+
+  private void fixOffsets() {
+    int startOffset = offsetAtt.startOffset();
+    int endOffset = offsetAtt.endOffset();
+    if (startOffset < lastStartOffset) {
+      startOffset = lastStartOffset;
+    }
+    if (endOffset < startOffset) {
+      endOffset = startOffset;
+    }
+    offsetAtt.setOffset(startOffset, endOffset);
+    lastStartOffset = startOffset;
+    lastEndOffset = endOffset;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilterFactory.java
new file mode 100644
index 0000000..8484b8c
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/FixBrokenOffsetsFilterFactory.java
@@ -0,0 +1,39 @@
+/*
+ * 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.lucene.analysis.miscellaneous;
+
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link FixBrokenOffsetsFilter}.
+ */
+public class FixBrokenOffsetsFilterFactory extends TokenFilterFactory {
+
+  /** Sole constructor */
+  public FixBrokenOffsetsFilterFactory(Map<String,String> args) {
+    super(args);
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new FixBrokenOffsetsFilter(input);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
index 73986d7..5f8894c 100644
--- a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
+++ b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -64,6 +64,7 @@ org.apache.lucene.analysis.miscellaneous.CapitalizationFilterFactory
 org.apache.lucene.analysis.miscellaneous.CodepointCountFilterFactory
 org.apache.lucene.analysis.miscellaneous.DateRecognizerFilterFactory
 org.apache.lucene.analysis.miscellaneous.FingerprintFilterFactory
+org.apache.lucene.analysis.miscellaneous.FixBrokenOffsetsFilterFactory
 org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilterFactory
 org.apache.lucene.analysis.miscellaneous.KeepWordFilterFactory
 org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java
new file mode 100644
index 0000000..ada5014
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestFixBrokenOffsetsFilter.java
@@ -0,0 +1,50 @@
+/*
+ * 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.lucene.analysis.miscellaneous;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.Directory;
+
+public class TestFixBrokenOffsetsFilter extends BaseTokenStreamTestCase {
+
+  public void testBogusTermVectors() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+    ft.setStoreTermVectors(true);
+    ft.setStoreTermVectorOffsets(true);
+    Field field = new Field("foo", "", ft);
+    field.setTokenStream(new FixBrokenOffsetsFilter(new CannedTokenStream(
+        new Token("bar", 5, 10), new Token("bar", 1, 4)
+        )));
+    doc.add(field);
+    iw.addDocument(doc);
+    iw.close();
+    dir.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java b/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java
new file mode 100644
index 0000000..d4d6f85
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java
@@ -0,0 +1,125 @@
+/*
+ * 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.lucene.index;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.SuppressForbidden;
+
+/**
+ * Command-line tool that reads from a source index and
+ * writes to a dest index, correcting any broken offsets
+ * in the process.
+ *
+ * @lucene.experimental
+ */
+public class FixBrokenOffsets {
+  public SegmentInfos infos;
+
+  FSDirectory fsDir;
+
+  Path dir;
+
+  @SuppressForbidden(reason = "System.out required: command line tool")
+  public static void main(String[] args) throws IOException {
+    if (args.length < 2) {
+      System.err.println("Usage: FixBrokenOffsetse <srcDir> <destDir>");
+      return;
+    }
+    Path srcPath = Paths.get(args[0]);
+    if (!Files.exists(srcPath)) {
+      throw new RuntimeException("srcPath " + srcPath.toAbsolutePath() + " doesn't exist");
+    }
+    Path destPath = Paths.get(args[1]);
+    if (Files.exists(destPath)) {
+      throw new RuntimeException("destPath " + destPath.toAbsolutePath() + " already exists; please remove it and re-run");
+    }
+    Directory srcDir = FSDirectory.open(srcPath);
+    DirectoryReader reader = DirectoryReader.open(srcDir);
+
+    List<LeafReaderContext> leaves = reader.leaves();
+    CodecReader[] filtered = new CodecReader[leaves.size()];
+    for(int i=0;i<leaves.size();i++) {
+      filtered[i] = SlowCodecReaderWrapper.wrap(new FilterLeafReader(leaves.get(i).reader()) {
+          @Override
+          public Fields getTermVectors(int docID) throws IOException {
+            Fields termVectors = in.getTermVectors(docID);
+            if (termVectors == null) {
+              return null;
+            }
+            return new FilterFields(termVectors) {
+              @Override
+              public Terms terms(String field) throws IOException {
+                return new FilterTerms(super.terms(field)) {
+                  @Override
+                  public TermsEnum iterator() throws IOException {
+                    return new FilterTermsEnum(super.iterator()) {
+                      @Override
+                      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+                        return new FilterPostingsEnum(super.postings(reuse, flags)) {
+                          int nextLastStartOffset = 0;
+                          int lastStartOffset = 0;
+
+                          @Override
+                          public int nextPosition() throws IOException {
+                            int pos = super.nextPosition();
+                            lastStartOffset = nextLastStartOffset;
+                            nextLastStartOffset = startOffset();
+                            return pos;
+                          }
+                          
+                          @Override
+                          public int startOffset() throws IOException {
+                            int offset = super.startOffset();
+                            if (offset < lastStartOffset) {
+                              offset = lastStartOffset;
+                            }
+                            return offset;
+                          }
+                          
+                          @Override
+                          public int endOffset() throws IOException {
+                            int offset = super.endOffset();
+                            if (offset < lastStartOffset) {
+                              offset = lastStartOffset;
+                            }
+                            return offset;
+                          }
+                        };
+                      }
+                    };
+                  }
+                };
+              }
+            };
+          }
+        });
+    }
+
+    Directory destDir = FSDirectory.open(destPath);
+    IndexWriter writer = new IndexWriter(destDir, new IndexWriterConfig());
+    writer.addIndexes(filtered);
+    IOUtils.close(writer, reader, srcDir, destDir);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html
new file mode 100644
index 0000000..42ff91a
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/index/package.html
@@ -0,0 +1,27 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<!-- not a package-info.java, because we already defined this package in core/ -->
+<html>
+<head>
+  <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+  <title>Tools for handling backwards compatibility issues with indices.</title>
+</head>
+<body>
+Tools for handling backwards compatibility issues with indices.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
new file mode 100644
index 0000000..bcd5a65
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
@@ -0,0 +1,114 @@
+/*
+ * 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.lucene.index;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestFixBrokenOffsets extends LuceneTestCase {
+
+  // Run this in Lucene 6.x:
+  //
+  //     ant test -Dtestcase=TestFixBrokenOffsets -Dtestmethod=testCreateBrokenOffsetsIndex -Dtests.codec=default -Dtests.useSecurityManager=false
+  /*
+  public void testCreateBrokenOffsetsIndex() throws IOException {
+
+    Path indexDir = Paths.get("/tmp/brokenoffsets");
+    Files.deleteIfExists(indexDir);
+    Directory dir = newFSDirectory(indexDir);
+    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+
+    Document doc = new Document();
+    FieldType fieldType = new FieldType(TextField.TYPE_STORED);
+    fieldType.setStoreTermVectors(true);
+    fieldType.setStoreTermVectorPositions(true);
+    fieldType.setStoreTermVectorOffsets(true);
+    Field field = new Field("foo", "bar", fieldType);
+    field.setTokenStream(new CannedTokenStream(new Token("foo", 10, 13), new Token("foo", 7, 9)));
+    doc.add(field);
+    writer.addDocument(doc);
+    writer.commit();
+
+    // 2nd segment
+    doc = new Document();
+    field = new Field("foo", "bar", fieldType);
+    field.setTokenStream(new CannedTokenStream(new Token("bar", 15, 17), new Token("bar", 1, 5)));
+    doc.add(field);
+    writer.addDocument(doc);
+    
+    writer.close();
+
+    dir.close();
+  }
+  */
+
+  public void testFixBrokenOffsetsIndex() throws IOException {
+    InputStream resource = getClass().getResourceAsStream("index.630.brokenoffsets.zip");
+    assertNotNull("Broken offsets index not found", resource);
+    Path path = createTempDir("brokenoffsets");
+    TestUtil.unzip(resource, path);
+    Directory dir = FSDirectory.open(path);
+
+    // OK: index is 6.3.0 so offsets not checked:
+    TestUtil.checkIndex(dir);
+    
+    MockDirectoryWrapper tmpDir = newMockDirectory();
+    tmpDir.setCheckIndexOnClose(false);
+    IndexWriter w = new IndexWriter(tmpDir, new IndexWriterConfig());
+    w.addIndexes(dir);
+    w.close();
+    // OK: addIndexes(Directory...) also keeps version as 6.3.0, so offsets not checked:
+    TestUtil.checkIndex(tmpDir);
+    tmpDir.close();
+
+    final MockDirectoryWrapper tmpDir2 = newMockDirectory();
+    tmpDir2.setCheckIndexOnClose(false);
+    w = new IndexWriter(tmpDir2, new IndexWriterConfig());
+    DirectoryReader reader = DirectoryReader.open(dir);
+    List<LeafReaderContext> leaves = reader.leaves();
+    CodecReader[] codecReaders = new CodecReader[leaves.size()];
+    for(int i=0;i<leaves.size();i++) {
+      codecReaders[i] = (CodecReader) leaves.get(i).reader();
+    }
+    w.addIndexes(codecReaders);
+    w.close();
+
+    // NOT OK: broken offsets were copied into a 7.0 segment:
+    ByteArrayOutputStream output = new ByteArrayOutputStream(1024);    
+    RuntimeException re = expectThrows(RuntimeException.class, () -> {TestUtil.checkIndex(tmpDir2, false, true, output);});
+    assertEquals("term [66 6f 6f]: doc 0: pos 1: startOffset 7 < lastStartOffset 10; consider using the FixBrokenOffsets tool in Lucene's backward-codecs module to correct your index", re.getMessage());
+    tmpDir2.close();
+
+    // Now run the tool and confirm the broken offsets are fixed:
+    Path path2 = createTempDir("fixedbrokenoffsets").resolve("subdir");
+    FixBrokenOffsets.main(new String[] {path.toString(), path2.toString()});
+    Directory tmpDir3 = FSDirectory.open(path2);
+    TestUtil.checkIndex(tmpDir3);
+    tmpDir3.close();
+    
+    dir.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip
new file mode 100644
index 0000000..3cf476a
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.630.brokenoffsets.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index fd8011d..3bb10d3 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -740,13 +740,13 @@ public final class CheckIndex implements Closeable {
           segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, failFast);
+          segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, failFast, version);
 
           // Test Stored Fields
           segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors, failFast);
+          segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors, failFast, version);
 
           // Test Docvalues
           segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
@@ -1205,7 +1205,7 @@ public final class CheckIndex implements Closeable {
    * checks Fields api is consistent with itself.
    * searcher is optional, to verify with queries. Can be null.
    */
-  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose) throws IOException {
+  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose, Version version) throws IOException {
     // TODO: we should probably return our own stats thing...?!
     long startNS;
     if (doPrint) {
@@ -1461,14 +1461,13 @@ public final class CheckIndex implements Closeable {
               if (hasOffsets) {
                 int startOffset = postings.startOffset();
                 int endOffset = postings.endOffset();
-                // NOTE: we cannot enforce any bounds whatsoever on vectors... they were a free-for-all before?
-                // but for offsets in the postings lists these checks are fine: they were always enforced by IndexWriter
-                if (!isVectors) {
+                // In Lucene 7 we fixed IndexWriter to also enforce term vector offsets
+                if (isVectors == false || version.onOrAfter(Version.LUCENE_7_0_0)) {
                   if (startOffset < 0) {
                     throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " is out of bounds");
                   }
                   if (startOffset < lastOffset) {
-                    throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " < lastStartOffset " + lastOffset);
+                    throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": startOffset " + startOffset + " < lastStartOffset " + lastOffset + "; consider using the FixBrokenOffsets tool in Lucene's backward-codecs module to correct your index");
                   }
                   if (endOffset < 0) {
                     throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + ": endOffset " + endOffset + " is out of bounds");
@@ -1742,15 +1741,15 @@ public final class CheckIndex implements Closeable {
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream) throws IOException {
-    return testPostings(reader, infoStream, false, false);
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, Version version) throws IOException {
+    return testPostings(reader, infoStream, false, false, version);
   }
   
   /**
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
+  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream, boolean verbose, boolean failFast, Version version) throws IOException {
 
     // TODO: we should go and verify term vectors match, if
     // crossCheckTermVectors is on...
@@ -1765,7 +1764,7 @@ public final class CheckIndex implements Closeable {
 
       final Fields fields = reader.getPostingsReader().getMergeInstance();
       final FieldInfos fieldInfos = reader.getFieldInfos();
-      status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose);
+      status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose, version);
     } catch (Throwable e) {
       if (failFast) {
         IOUtils.reThrow(e);
@@ -2339,15 +2338,15 @@ public final class CheckIndex implements Closeable {
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream) throws IOException {
-    return testTermVectors(reader, infoStream, false, false, false);
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, Version version) throws IOException {
+    return testTermVectors(reader, infoStream, false, false, false, version);
   }
 
   /**
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
+  public static Status.TermVectorStatus testTermVectors(CodecReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast, Version version) throws IOException {
     long startNS = System.nanoTime();
     final Status.TermVectorStatus status = new Status.TermVectorStatus();
     final FieldInfos fieldInfos = reader.getFieldInfos();
@@ -2387,7 +2386,7 @@ public final class CheckIndex implements Closeable {
           
           if (tfv != null) {
             // First run with no deletions:
-            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
+            checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose, version);
             
             // Only agg stats if the doc is live:
             final boolean doStats = liveDocs == null || liveDocs.get(j);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index 79c285b..197ab31 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.NormsConsumer;
@@ -728,10 +729,6 @@ final class DefaultIndexingChain extends DocConsumer {
 
       final boolean analyzed = fieldType.tokenized() && docState.analyzer != null;
         
-      // only bother checking offsets if something will consume them.
-      // TODO: after we fix analyzers, also check if termVectorOffsets will be indexed.
-      final boolean checkOffsets = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
-
       /*
        * To assist people in tracking down problems in analysis components, we wish to write the field name to the infostream
        * when we fail. We expect some caller to eventually deal with the real exception, so we don't want any 'catch' clauses,
@@ -743,6 +740,7 @@ final class DefaultIndexingChain extends DocConsumer {
         stream.reset();
         invertState.setAttributeSource(stream);
         termsHashPerField.start(field, first);
+        CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class);
 
         while (stream.incrementToken()) {
 
@@ -771,15 +769,13 @@ final class DefaultIndexingChain extends DocConsumer {
             invertState.numOverlap++;
           }
               
-          if (checkOffsets) {
-            int startOffset = invertState.offset + invertState.offsetAttribute.startOffset();
-            int endOffset = invertState.offset + invertState.offsetAttribute.endOffset();
-            if (startOffset < invertState.lastStartOffset || endOffset < startOffset) {
-              throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, and offsets must not go backwards "
-                                                 + "startOffset=" + startOffset + ",endOffset=" + endOffset + ",lastStartOffset=" + invertState.lastStartOffset + " for field '" + field.name() + "'");
-            }
-            invertState.lastStartOffset = startOffset;
+          int startOffset = invertState.offset + invertState.offsetAttribute.startOffset();
+          int endOffset = invertState.offset + invertState.offsetAttribute.endOffset();
+          if (startOffset < invertState.lastStartOffset || endOffset < startOffset) {
+            throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, and offsets must not go backwards "
+                                               + "startOffset=" + startOffset + ",endOffset=" + endOffset + ",lastStartOffset=" + invertState.lastStartOffset + " for field '" + field.name() + "'");
           }
+          invertState.lastStartOffset = startOffset;
 
           invertState.length++;
           if (invertState.length < 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java b/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
index 7b71d3c..2559ce4 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
@@ -43,11 +43,6 @@ public class TestCheckIndex extends BaseTestCheckIndex {
   }
   
   @Test
-  public void testBogusTermVectors() throws IOException {
-    testBogusTermVectors(directory);
-  }
-  
-  @Test
   public void testChecksumsOnly() throws IOException {
     testChecksumsOnly(directory);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java
index 581ff2f..d49434a 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java
@@ -377,7 +377,7 @@ public class TokenSourcesTest extends BaseTokenStreamTestCase {
     }
 
     final BaseTermVectorsFormatTestCase.RandomTokenStream rTokenStream =
-        new BaseTermVectorsFormatTestCase.RandomTokenStream(TestUtil.nextInt(random(), 1, 10), terms, termBytes, false);
+        new BaseTermVectorsFormatTestCase.RandomTokenStream(TestUtil.nextInt(random(), 1, 10), terms, termBytes);
     //check to see if the token streams might have non-deterministic testable result
     final boolean storeTermVectorPositions = random().nextBoolean();
     final int[] startOffsets = rTokenStream.getStartOffsets();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
index 6055e00..6ef9baf 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
@@ -45,6 +45,7 @@ import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
@@ -431,7 +432,9 @@ public class TestTermAutomatonQuery extends LuceneTestCase {
     @Override
     public boolean incrementToken() throws IOException {
       if (synNext) {
+        AttributeSource.State state = captureState();
         clearAttributes();
+        restoreState(state);
         posIncAtt.setPositionIncrement(0);
         termAtt.append(""+((char) 97 + random().nextInt(3)));
         synNext = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
index 5e6809f..7acee87 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseTermVectorsFormatTestCase.java
@@ -200,10 +200,6 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
     int i = 0;
 
     public RandomTokenStream(int len, String[] sampleTerms, BytesRef[] sampleTermBytes) {
-      this(len, sampleTerms, sampleTermBytes, rarely());
-    }
-
-    public RandomTokenStream(int len, String[] sampleTerms, BytesRef[] sampleTermBytes, boolean offsetsGoBackwards) {
       terms = new String[len];
       termBytes = new BytesRef[len];
       positionsIncrements = new int[len];
@@ -216,17 +212,12 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
         terms[i] = sampleTerms[o];
         termBytes[i] = sampleTermBytes[o];
         positionsIncrements[i] = TestUtil.nextInt(random(), i == 0 ? 1 : 0, 10);
-        if (offsetsGoBackwards) {
-          startOffsets[i] = random().nextInt();
-          endOffsets[i] = random().nextInt();
+        if (i == 0) {
+          startOffsets[i] = TestUtil.nextInt(random(), 0, 1 << 16);
         } else {
-          if (i == 0) {
-            startOffsets[i] = TestUtil.nextInt(random(), 0, 1 << 16);
-          } else {
-            startOffsets[i] = startOffsets[i-1] + TestUtil.nextInt(random(), 0, rarely() ? 1 << 16 : 20);
-          }
-          endOffsets[i] = startOffsets[i] + TestUtil.nextInt(random(), 0, rarely() ? 1 << 10 : 20);
+          startOffsets[i] = startOffsets[i-1] + TestUtil.nextInt(random(), 0, rarely() ? 1 << 16 : 20);
         }
+        endOffsets[i] = startOffsets[i] + TestUtil.nextInt(random(), 0, rarely() ? 1 << 10 : 20);
       }
 
       for (int i = 0; i < len; ++i) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/test-framework/src/java/org/apache/lucene/index/BaseTestCheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseTestCheckIndex.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseTestCheckIndex.java
index cdec720..21ccf3b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseTestCheckIndex.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseTestCheckIndex.java
@@ -22,11 +22,8 @@ import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
@@ -105,22 +102,6 @@ public class BaseTestCheckIndex extends LuceneTestCase {
     checker.close();
   }
   
-  // LUCENE-4221: we have to let these thru, for now
-  public void testBogusTermVectors(Directory dir) throws IOException {
-    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
-    Document doc = new Document();
-    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
-    ft.setStoreTermVectors(true);
-    ft.setStoreTermVectorOffsets(true);
-    Field field = new Field("foo", "", ft);
-    field.setTokenStream(new CannedTokenStream(
-        new Token("bar", 5, 10), new Token("bar", 1, 4)
-    ));
-    doc.add(field);
-    iw.addDocument(doc);
-    iw.close();
-  }
-  
   public void testChecksumsOnly(Directory dir) throws IOException {
     LineFileDocs lf = new LineFileDocs(random());
     MockAnalyzer analyzer = new MockAnalyzer(random());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index d3351ab..0ea90fc 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -334,9 +334,9 @@ public final class TestUtil {
     CheckIndex.testLiveDocs(codecReader, infoStream, true);
     CheckIndex.testFieldInfos(codecReader, infoStream, true);
     CheckIndex.testFieldNorms(codecReader, infoStream, true);
-    CheckIndex.testPostings(codecReader, infoStream, false, true);
+    CheckIndex.testPostings(codecReader, infoStream, false, true, Version.LUCENE_7_0_0);
     CheckIndex.testStoredFields(codecReader, infoStream, true);
-    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true);
+    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true, Version.LUCENE_7_0_0);
     CheckIndex.testDocValues(codecReader, infoStream, true);
     CheckIndex.testPoints(codecReader, infoStream, true);
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java b/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java
index 87d4094..5f125d9 100644
--- a/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java
+++ b/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java
@@ -27,6 +27,7 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableField;
@@ -284,6 +285,7 @@ public class PreAnalyzedField extends TextField implements HasImplicitIndexAnaly
     private byte[] binaryValue = null;
     private PreAnalyzedParser parser;
     private IOException readerConsumptionException;
+    private int lastEndOffset;
 
     public PreAnalyzedTokenizer(PreAnalyzedParser parser) {
       // we don't pack attributes: since we are used for (de)serialization and dont want bloat.
@@ -311,6 +313,8 @@ public class PreAnalyzedField extends TextField implements HasImplicitIndexAnaly
       
       AttributeSource.State state = it.next();
       restoreState(state.clone());
+      // TODO: why can't I lookup the OffsetAttribute up in ctor instead?
+      lastEndOffset = addAttribute(OffsetAttribute.class).endOffset();
       return true;
     }
 
@@ -329,6 +333,13 @@ public class PreAnalyzedField extends TextField implements HasImplicitIndexAnaly
       it = cachedStates.iterator();
     }
 
+    @Override
+    public void end() throws IOException {
+      super.end();
+      // we must set the end offset correctly so multi-valued fields don't try to send offsets backwards:
+      addAttribute(OffsetAttribute.class).setOffset(lastEndOffset, lastEndOffset);
+    }
+
     private void setReaderConsumptionException(IOException e) {
       readerConsumptionException = e;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/64b86331/solr/core/src/test/org/apache/solr/index/hdfs/CheckHdfsIndexTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/index/hdfs/CheckHdfsIndexTest.java b/solr/core/src/test/org/apache/solr/index/hdfs/CheckHdfsIndexTest.java
index b4f6931..61b4305 100644
--- a/solr/core/src/test/org/apache/solr/index/hdfs/CheckHdfsIndexTest.java
+++ b/solr/core/src/test/org/apache/solr/index/hdfs/CheckHdfsIndexTest.java
@@ -121,11 +121,6 @@ public class CheckHdfsIndexTest extends AbstractFullDistribZkTestBase {
   }
 
   @Test
-  public void testBogusTermVectors() throws IOException {
-    testCheckIndex.testBogusTermVectors(directory);
-  }
-
-  @Test
   public void testChecksumsOnly() throws IOException {
     testCheckIndex.testChecksumsOnly(directory);
   }


[24/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9644: Fixed SimpleMLTQParser and CloudMLTQParser to handle boosts properly and CloudMLTQParser to only extract actual values from IndexableField type fields to the filtered document.

Posted by ho...@apache.org.
SOLR-9644: Fixed SimpleMLTQParser and CloudMLTQParser to handle boosts properly and CloudMLTQParser to only extract actual values from IndexableField type fields to the filtered document.


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

Branch: refs/heads/jira/solr-5944
Commit: 2b4e3dd941a7a88274f2a86f18ea57a9d95e4364
Parents: b8383db
Author: anshum <an...@apache.org>
Authored: Mon Jan 9 13:05:21 2017 -0800
Committer: anshum <an...@apache.org>
Committed: Mon Jan 9 13:06:24 2017 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 ++
 .../apache/solr/search/mlt/CloudMLTQParser.java | 49 ++++++++++++--------
 .../solr/search/mlt/SimpleMLTQParser.java       | 30 ++++++------
 .../solr/search/mlt/CloudMLTQParserTest.java    | 23 ++++++++-
 .../solr/search/mlt/SimpleMLTQParserTest.java   | 33 +++++++++++--
 5 files changed, 102 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2b4e3dd9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c79b3c6..2b79f04 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -345,6 +345,10 @@ Bug Fixes
 * SOLR-9883: Example schemaless solr config files can lead to invalid tlog replays: when updates are buffered,
   update processors ordered before DistributedUpdateProcessor, e.g. field normalization, are never run. (Steve Rowe)
 
+* SOLR-9644: SimpleMLTQParser and CloudMLTQParser did not handle field boosts properly
+  and CloudMLTQParser included extra strings from the field definitions in the query.
+  (Ere Maijala via Anshum Gupta)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2b4e3dd9/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java b/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java
index 0f46725..945047b 100644
--- a/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/mlt/CloudMLTQParser.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.regex.Pattern;
 
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.mlt.MoreLikeThis;
@@ -64,73 +65,83 @@ public class CloudMLTQParser extends QParser {
           SolrException.ErrorCode.BAD_REQUEST, "Error completing MLT request. Could not fetch " +
           "document with id [" + id + "]");
     }
-    
+
     String[] qf = localParams.getParams("qf");
     Map<String,Float> boostFields = new HashMap<>();
     MoreLikeThis mlt = new MoreLikeThis(req.getSearcher().getIndexReader());
-    
-    mlt.setMinTermFreq(localParams.getInt("mintf", MoreLikeThis.DEFAULT_MIN_TERM_FREQ));
 
+    mlt.setMinTermFreq(localParams.getInt("mintf", MoreLikeThis.DEFAULT_MIN_TERM_FREQ));
     mlt.setMinDocFreq(localParams.getInt("mindf", 0));
-
     mlt.setMinWordLen(localParams.getInt("minwl", MoreLikeThis.DEFAULT_MIN_WORD_LENGTH));
-
     mlt.setMaxWordLen(localParams.getInt("maxwl", MoreLikeThis.DEFAULT_MAX_WORD_LENGTH));
-
     mlt.setMaxQueryTerms(localParams.getInt("maxqt", MoreLikeThis.DEFAULT_MAX_QUERY_TERMS));
-
     mlt.setMaxNumTokensParsed(localParams.getInt("maxntp", MoreLikeThis.DEFAULT_MAX_NUM_TOKENS_PARSED));
-    
     mlt.setMaxDocFreq(localParams.getInt("maxdf", MoreLikeThis.DEFAULT_MAX_DOC_FREQ));
 
-    if(localParams.get("boost") != null) {
-      mlt.setBoost(localParams.getBool("boost"));
-      boostFields = SolrPluginUtils.parseFieldBoosts(qf);
-    }
+    Boolean boost = localParams.getBool("boost", MoreLikeThis.DEFAULT_BOOST);
+    mlt.setBoost(boost);
 
     mlt.setAnalyzer(req.getSchema().getIndexAnalyzer());
 
     Map<String, Collection<Object>> filteredDocument = new HashMap<>();
-    ArrayList<String> fieldNames = new ArrayList<>();
+    String[] fieldNames;
 
     if (qf != null) {
+      ArrayList<String> fields = new ArrayList();
       for (String fieldName : qf) {
         if (!StringUtils.isEmpty(fieldName))  {
           String[] strings = splitList.split(fieldName);
           for (String string : strings) {
             if (!StringUtils.isEmpty(string)) {
-              fieldNames.add(string);
+              fields.add(string);
             }
           }
         }
       }
+      // Parse field names and boosts from the fields
+      boostFields = SolrPluginUtils.parseFieldBoosts(fields.toArray(new String[0]));
+      fieldNames = boostFields.keySet().toArray(new String[0]);
     } else {
+      ArrayList<String> fields = new ArrayList();
       for (String field : doc.getFieldNames()) {
         // Only use fields that are stored and have an explicit analyzer.
         // This makes sense as the query uses tf/idf/.. for query construction.
         // We might want to relook and change this in the future though.
         SchemaField f = req.getSchema().getFieldOrNull(field);
         if (f != null && f.stored() && f.getType().isExplicitAnalyzer()) {
-          fieldNames.add(field);
+          fields.add(field);
         }
       }
+      fieldNames = fields.toArray(new String[0]);
     }
 
-    if( fieldNames.size() < 1 ) {
+    if (fieldNames.length < 1) {
       throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
           "MoreLikeThis requires at least one similarity field: qf" );
     }
 
-    mlt.setFieldNames(fieldNames.toArray(new String[fieldNames.size()]));
+    mlt.setFieldNames(fieldNames);
     for (String field : fieldNames) {
-      filteredDocument.put(field, doc.getFieldValues(field));
+      Collection<Object> fieldValues = doc.getFieldValues(field);
+      if (fieldValues != null) {
+        Collection<Object> values = new ArrayList();
+        for (Object val : fieldValues) {
+          if (val instanceof IndexableField) {
+            values.add(((IndexableField)val).stringValue());
+          }
+          else {
+            values.add(val);
+          }
+        }
+        filteredDocument.put(field, values);
+      }
     }
 
     try {
       Query rawMLTQuery = mlt.like(filteredDocument);
       BooleanQuery boostedMLTQuery = (BooleanQuery) rawMLTQuery;
 
-      if (boostFields.size() > 0) {
+      if (boost && boostFields.size() > 0) {
         BooleanQuery.Builder newQ = new BooleanQuery.Builder();
         newQ.setMinimumNumberShouldMatch(boostedMLTQuery.getMinimumNumberShouldMatch());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2b4e3dd9/solr/core/src/java/org/apache/solr/search/mlt/SimpleMLTQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/mlt/SimpleMLTQParser.java b/solr/core/src/java/org/apache/solr/search/mlt/SimpleMLTQParser.java
index 50803df..de6eb58 100644
--- a/solr/core/src/java/org/apache/solr/search/mlt/SimpleMLTQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/mlt/SimpleMLTQParser.java
@@ -76,16 +76,13 @@ public class SimpleMLTQParser extends QParser {
       mlt.setMaxQueryTerms(localParams.getInt("maxqt", MoreLikeThis.DEFAULT_MAX_QUERY_TERMS));
       mlt.setMaxNumTokensParsed(localParams.getInt("maxntp", MoreLikeThis.DEFAULT_MAX_NUM_TOKENS_PARSED));
       mlt.setMaxDocFreq(localParams.getInt("maxdf", MoreLikeThis.DEFAULT_MAX_DOC_FREQ));
+      Boolean boost = localParams.getBool("boost", false);
+      mlt.setBoost(boost);
 
-      // what happens if value is explicitly set to false?
-      if(localParams.get("boost") != null) {
-        mlt.setBoost(localParams.getBool("boost", false));
-        boostFields = SolrPluginUtils.parseFieldBoosts(qf);
-      }
+      String[] fieldNames;
       
-      ArrayList<String> fields = new ArrayList<>();
-
       if (qf != null) {
+        ArrayList<String> fields = new ArrayList<>();
         for (String fieldName : qf) {
           if (!StringUtils.isEmpty(fieldName))  {
             String[] strings = splitList.split(fieldName);
@@ -96,26 +93,31 @@ public class SimpleMLTQParser extends QParser {
             }
           }
         }
+        // Parse field names and boosts from the fields
+        boostFields = SolrPluginUtils.parseFieldBoosts(fields.toArray(new String[0]));
+        fieldNames = boostFields.keySet().toArray(new String[0]);
       } else {
-        Map<String, SchemaField> fieldNames = req.getSearcher().getSchema().getFields();
-        for (String fieldName : fieldNames.keySet()) {
-          if (fieldNames.get(fieldName).indexed() && fieldNames.get(fieldName).stored())
-            if (fieldNames.get(fieldName).getType().getNumericType() == null)
+        Map<String, SchemaField> fieldDefinitions = req.getSearcher().getSchema().getFields();
+        ArrayList<String> fields = new ArrayList();
+        for (String fieldName : fieldDefinitions.keySet()) {
+          if (fieldDefinitions.get(fieldName).indexed() && fieldDefinitions.get(fieldName).stored())
+            if (fieldDefinitions.get(fieldName).getType().getNumericType() == null)
               fields.add(fieldName);
         }
+        fieldNames = fields.toArray(new String[0]);
       }
-      if( fields.size() < 1 ) {
+      if (fieldNames.length < 1) {
         throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
             "MoreLikeThis requires at least one similarity field: qf" );
       }
 
-      mlt.setFieldNames(fields.toArray(new String[fields.size()]));
+      mlt.setFieldNames(fieldNames);
       mlt.setAnalyzer(req.getSchema().getIndexAnalyzer());
 
       Query rawMLTQuery = mlt.like(scoreDocs[0].doc);
       BooleanQuery boostedMLTQuery = (BooleanQuery) rawMLTQuery;
 
-      if (boostFields.size() > 0) {
+      if (boost && boostFields.size() > 0) {
         BooleanQuery.Builder newQ = new BooleanQuery.Builder();
         newQ.setMinimumNumberShouldMatch(boostedMLTQuery.getMinimumNumberShouldMatch());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2b4e3dd9/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java b/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java
index ffcde2f..e3a8d7b 100644
--- a/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java
+++ b/solr/core/src/test/org/apache/solr/search/mlt/CloudMLTQParserTest.java
@@ -121,6 +121,27 @@ public class CloudMLTQParserTest extends SolrCloudTestCase {
     }
     assertArrayEquals(expectedIds, actualIds);
 
+    queryResponse = cluster.getSolrClient().query(COLLECTION, new SolrQuery("{!mlt qf=lowerfilt_u^10,lowerfilt1_u^1000 boost=false mintf=0 mindf=0}30"));
+    solrDocuments = queryResponse.getResults();
+    expectedIds = new int[]{31, 18, 23, 13, 14, 20, 22, 32, 19, 21};
+    actualIds = new int[solrDocuments.size()];
+    i = 0;
+    for (SolrDocument solrDocument : solrDocuments) {
+      actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id")));
+    }
+    System.out.println("DEBUG ACTUAL IDS 1: " + Arrays.toString(actualIds));
+    assertArrayEquals(expectedIds, actualIds);
+
+    queryResponse = cluster.getSolrClient().query(COLLECTION, new SolrQuery("{!mlt qf=lowerfilt_u^10,lowerfilt1_u^1000 boost=true mintf=0 mindf=0}30"));
+    solrDocuments = queryResponse.getResults();
+    expectedIds = new int[]{29, 31, 32, 18, 23, 13, 14, 20, 22, 19};
+    actualIds = new int[solrDocuments.size()];
+    i = 0;
+    for (SolrDocument solrDocument : solrDocuments) {
+      actualIds[i++] = Integer.valueOf(String.valueOf(solrDocument.getFieldValue("id")));
+    }
+    System.out.println("DEBUG ACTUAL IDS 2: " + Arrays.toString(actualIds));
+    assertArrayEquals(expectedIds, actualIds);
   }
 
   @Test
@@ -220,7 +241,7 @@ public class CloudMLTQParserTest extends SolrCloudTestCase {
     }
     assertArrayEquals(expectedIds, actualIds);
   }
-  
+
   public void testInvalidSourceDocument() throws IOException {
     SolrException e = expectThrows(SolrException.class, () -> {
       cluster.getSolrClient().query(COLLECTION, new SolrQuery("{!mlt qf=lowerfilt_u}999999"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2b4e3dd9/solr/core/src/test/org/apache/solr/search/mlt/SimpleMLTQParserTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/mlt/SimpleMLTQParserTest.java b/solr/core/src/test/org/apache/solr/search/mlt/SimpleMLTQParserTest.java
index 6f3570f..026c594 100644
--- a/solr/core/src/test/org/apache/solr/search/mlt/SimpleMLTQParserTest.java
+++ b/solr/core/src/test/org/apache/solr/search/mlt/SimpleMLTQParserTest.java
@@ -108,8 +108,37 @@ public class SimpleMLTQParserTest extends SolrTestCaseJ4 {
     );
 
     params = new ModifiableSolrParams();
+    params.set(CommonParams.Q, "{!mlt qf=lowerfilt,lowerfilt1^1000 boost=false mintf=0 mindf=0}30");
+    assertQ(req(params),
+        "//result/doc[1]/int[@name='id'][.='31']",
+        "//result/doc[2]/int[@name='id'][.='13']",
+        "//result/doc[3]/int[@name='id'][.='14']",
+        "//result/doc[4]/int[@name='id'][.='18']",
+        "//result/doc[5]/int[@name='id'][.='20']",
+        "//result/doc[6]/int[@name='id'][.='22']",
+        "//result/doc[7]/int[@name='id'][.='23']",
+        "//result/doc[8]/int[@name='id'][.='32']",
+        "//result/doc[9]/int[@name='id'][.='15']",
+        "//result/doc[10]/int[@name='id'][.='16']"
+    );
+
+    params = new ModifiableSolrParams();
+    params.set(CommonParams.Q, "{!mlt qf=lowerfilt,lowerfilt1^1000 boost=true mintf=0 mindf=0}30");
+    assertQ(req(params),
+        "//result/doc[1]/int[@name='id'][.='29']",
+        "//result/doc[2]/int[@name='id'][.='31']",
+        "//result/doc[3]/int[@name='id'][.='32']",
+        "//result/doc[4]/int[@name='id'][.='13']",
+        "//result/doc[5]/int[@name='id'][.='14']",
+        "//result/doc[6]/int[@name='id'][.='18']",
+        "//result/doc[7]/int[@name='id'][.='20']",
+        "//result/doc[8]/int[@name='id'][.='22']",
+        "//result/doc[9]/int[@name='id'][.='23']",
+        "//result/doc[10]/int[@name='id'][.='15']"
+    );
+
+    params = new ModifiableSolrParams();
     params.set(CommonParams.Q, "{!mlt qf=lowerfilt mindf=0 mintf=1}26");
-    params.set(CommonParams.DEBUG, "true");
     assertQ(req(params),
         "//result/doc[1]/int[@name='id'][.='29']",
         "//result/doc[2]/int[@name='id'][.='27']",
@@ -118,14 +147,12 @@ public class SimpleMLTQParserTest extends SolrTestCaseJ4 {
 
     params = new ModifiableSolrParams();
     params.set(CommonParams.Q, "{!mlt qf=lowerfilt mindf=10 mintf=1}26");
-    params.set(CommonParams.DEBUG, "true");
     assertQ(req(params),
         "//result[@numFound='0']"
     );
 
     params = new ModifiableSolrParams();
     params.set(CommonParams.Q, "{!mlt qf=lowerfilt minwl=3 mintf=1 mindf=1}26");
-    params.set(CommonParams.DEBUG, "true");
     assertQ(req(params),
         "//result[@numFound='3']"
     );


[18/50] [abbrv] lucene-solr:jira/solr-5944: improve exception message

Posted by ho...@apache.org.
improve exception message


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

Branch: refs/heads/jira/solr-5944
Commit: d7beb0f1479d2a9c4f37d780aa31edcd1012ac69
Parents: b9827bc
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Jan 9 05:48:29 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Jan 9 05:48:54 2017 -0500

----------------------------------------------------------------------
 lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d7beb0f1/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
index 0487400..b994d0d 100644
--- a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
@@ -386,7 +386,7 @@ public class MMapDirectory extends FSDirectory {
           " [Please grant at least the following permissions: RuntimePermission(\"accessClassInPackage.sun.misc\") " +
           " and ReflectPermission(\"suppressAccessChecks\")]";
     } catch (ReflectiveOperationException | RuntimeException e) {
-      return "Unmapping is not supported on this platform, because internal Java APIs are not compatible to this Lucene version: " + e; 
+      return "Unmapping is not supported on this platform, because internal Java APIs are not compatible with this Lucene version: " + e; 
     }
   }
   


[40/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9960 MetricsHandler should support multiple prefixes.

Posted by ho...@apache.org.
SOLR-9960 MetricsHandler should support multiple prefixes.


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

Branch: refs/heads/jira/solr-5944
Commit: 53d5af17dac5e2c27f1304b029cbd09461ea197b
Parents: 64b8633
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Fri Jan 13 19:24:40 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Sat Jan 14 09:52:15 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                | 14 +--
 .../solr/handler/admin/MetricsHandler.java      | 92 +++++++++++++-------
 .../apache/solr/metrics/SolrMetricManager.java  | 30 ++++---
 .../solr/handler/admin/MetricsHandlerTest.java  | 26 +++++-
 4 files changed, 109 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53d5af17/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b6055fe..ccc5d7c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -196,14 +196,16 @@ New Features
 
 * SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)
 
-* SOLR-9812: SOLR-9911: Added a new /admin/metrics API to return all metrics collected by Solr via API.
-  API supports three optional parameters:
+* SOLR-9812: SOLR-9911, SOLR-9960: Added a new /admin/metrics API to return all metrics collected by Solr via API.
+  API supports four optional multi-valued parameters:
   * 'group' (all,jvm,jetty,node,core),
-  * 'type' (all,counter,timer,gauge,histogram) both of which are multi-valued
-  * 'prefix' that filters the returned metrics
+  * 'type' (all,counter,timer,gauge,histogram),
+  * 'prefix' that filters the returned metrics,
+  * 'registry' that selects one or more registries by prefix (eg. solr.jvm,solr.core.collection1)
   Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
-  Example: http://localhost:8983/solr/admin/metrics?group=jvm&prefix=buffers
-  (shalin)
+  Example: http://localhost:8983/solr/admin/metrics?group=jvm&prefix=buffers,os
+  Example: http://localhost:8983/solr/admin/metrics?registry=solr.node,solr.core&prefix=ADMIN
+  (shalin, ab)
 
 * SOLR-9884: Add version to segments handler output (Steven Bower via Erick Erickson)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53d5af17/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
index 0c87875..385317b 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -18,9 +18,10 @@
 package org.apache.solr.handler.admin;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 import com.codahale.metrics.Counter;
@@ -34,7 +35,6 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.SolrQueryRequest;
@@ -74,59 +74,85 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
     MetricFilter mustMatchFilter = parseMustMatchFilter(req);
     List<MetricType> metricTypes = parseMetricTypes(req);
     List<MetricFilter> metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList());
-    List<Group> requestedGroups = parseGroups(req);
+    Set<String> requestedRegistries = parseRegistries(req);
 
     NamedList response = new NamedList();
-    for (Group group : requestedGroups) {
-      String registryName = SolrMetricManager.getRegistryName(group);
-      if (group == Group.core) {
-        // this requires special handling because of the way we create registry name for a core (deeply nested)
-        container.getAllCoreNames().forEach(s -> {
-          String coreRegistryName;
-          try (SolrCore core = container.getCore(s)) {
-            coreRegistryName = core.getCoreMetricManager().getRegistryName();
-          }
-          MetricRegistry registry = metricManager.registry(coreRegistryName);
-          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter));
-        });
-      } else {
-        MetricRegistry registry = metricManager.registry(registryName);
-        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter));
-      }
+    for (String registryName : requestedRegistries) {
+      MetricRegistry registry = metricManager.registry(registryName);
+      response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter));
     }
     rsp.getValues().add("metrics", response);
   }
 
   private MetricFilter parseMustMatchFilter(SolrQueryRequest req) {
-    String prefix = req.getParams().get("prefix");
+    String[] prefixes = req.getParams().getParams("prefix");
     MetricFilter mustMatchFilter;
-    if (prefix != null) {
-      mustMatchFilter = new SolrMetricManager.PrefixFilter(prefix.trim());
+    if (prefixes != null && prefixes.length > 0) {
+      Set<String> prefixSet = new HashSet<>();
+      for (String prefix : prefixes) {
+        prefixSet.addAll(StrUtils.splitSmart(prefix, ','));
+      }
+      mustMatchFilter = new SolrMetricManager.PrefixFilter((String[])prefixSet.toArray(new String[prefixSet.size()]));
     } else  {
       mustMatchFilter = MetricFilter.ALL;
     }
     return mustMatchFilter;
   }
 
-  private List<Group> parseGroups(SolrQueryRequest req) {
+  private Set<String> parseRegistries(SolrQueryRequest req) {
     String[] groupStr = req.getParams().getParams("group");
-    List<String> groups = Collections.emptyList();
+    String[] registryStr = req.getParams().getParams("registry");
+    if ((groupStr == null || groupStr.length == 0) && (registryStr == null || registryStr.length == 0)) {
+      // return all registries
+      return container.getMetricManager().registryNames();
+    }
+    boolean allRegistries = false;
+    Set<String> initialPrefixes = Collections.emptySet();
     if (groupStr != null && groupStr.length > 0) {
-      groups = new ArrayList<>();
+      initialPrefixes = new HashSet<>();
       for (String g : groupStr) {
-        groups.addAll(StrUtils.splitSmart(g, ','));
+        List<String> split = StrUtils.splitSmart(g, ',');
+        for (String s : split) {
+          if (s.trim().equals("all")) {
+            allRegistries = true;
+            break;
+          }
+          initialPrefixes.add(SolrMetricManager.overridableRegistryName(s.trim()));
+        }
+        if (allRegistries) {
+          return container.getMetricManager().registryNames();
+        }
       }
     }
 
-    List<Group> requestedGroups = Arrays.asList(Group.values()); // by default we return all groups
-    try {
-      if (groups.size() > 0 && !groups.contains("all")) {
-        requestedGroups = groups.stream().map(String::trim).map(Group::valueOf).collect(Collectors.toList());
+    if (registryStr != null && registryStr.length > 0) {
+      if (initialPrefixes.isEmpty()) {
+        initialPrefixes = new HashSet<>();
+      }
+      for (String r : registryStr) {
+        List<String> split = StrUtils.splitSmart(r, ',');
+        for (String s : split) {
+          if (s.trim().equals("all")) {
+            allRegistries = true;
+            break;
+          }
+          initialPrefixes.add(SolrMetricManager.overridableRegistryName(s.trim()));
+        }
+        if (allRegistries) {
+          return container.getMetricManager().registryNames();
+        }
+      }
+    }
+    Set<String> validRegistries = new HashSet<>();
+    for (String r : container.getMetricManager().registryNames()) {
+      for (String prefix : initialPrefixes) {
+        if (r.startsWith(prefix)) {
+          validRegistries.add(r);
+          break;
+        }
       }
-    } catch (IllegalArgumentException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid group in: " + groups + " specified. Must be one of (all, jvm, jetty, http, node, core)", e);
     }
-    return requestedGroups;
+    return validRegistries;
   }
 
   private List<MetricType> parseMetricTypes(SolrQueryRequest req) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53d5af17/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index 9f126ed..caa8906 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -93,31 +93,37 @@ public class SolrMetricManager {
    * with names that start with a prefix.
    */
   public static class PrefixFilter implements MetricFilter {
-    private final String prefix;
+    private final String[] prefixes;
     private final Set<String> matched = new HashSet<>();
+    private boolean allMatch = false;
 
     /**
      * Create a filter that uses the provided prefix.
-     * @param prefix prefix to use, must not be null. If empty then any
-     *               name will match.
+     * @param prefixes prefixes to use, must not be null. If empty then any
+     *               name will match, if not empty then match on any prefix will
+     *                 succeed (logical OR).
      */
-    public PrefixFilter(String prefix) {
-      Objects.requireNonNull(prefix);
-      this.prefix = prefix;
+    public PrefixFilter(String... prefixes) {
+      Objects.requireNonNull(prefixes);
+      this.prefixes = prefixes;
+      if (prefixes.length == 0) {
+        allMatch = true;
+      }
     }
 
     @Override
     public boolean matches(String name, Metric metric) {
-      if (prefix.isEmpty()) {
+      if (allMatch) {
         matched.add(name);
         return true;
       }
-      if (name.startsWith(prefix)) {
-        matched.add(name);
-        return true;
-      } else {
-        return false;
+      for (String prefix : prefixes) {
+        if (name.startsWith(prefix)) {
+          matched.add(name);
+          return true;
+        }
       }
+      return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53d5af17/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
index edab3ce..a1b29db 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -66,6 +66,26 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(values.get("solr.jvm"));
 
     resp = new SolrQueryResponse();
+    // "collection" works too, because it's a prefix for "collection1"
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "registry", "solr.core.collection,solr.jvm"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.core.collection1"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    // "collection" works too, because it's a prefix for "collection1"
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "registry", "solr.core.collection", "registry", "solr.jvm"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.core.collection1"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
     values = resp.getValues();
     assertNotNull(values.get("metrics"));
@@ -94,7 +114,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNull(values.get("ADMIN./admin/authorization.errors")); // this is a timer node
 
     resp = new SolrQueryResponse();
-    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "prefix", "CONTAINER.cores"), resp);
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "prefix", "CONTAINER.cores,CONTAINER.threadPool"), resp);
     values = resp.getValues();
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
@@ -102,10 +122,12 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertEquals(0, ((NamedList)values.get("solr.jvm")).size());
     assertEquals(0, ((NamedList)values.get("solr.jetty")).size());
     assertEquals(0, ((NamedList)values.get("solr.core.collection1")).size());
-    assertEquals(3, ((NamedList)values.get("solr.node")).size());
+    assertEquals(11, ((NamedList)values.get("solr.node")).size());
     assertNotNull(values.get("solr.node"));
     values = (NamedList) values.get("solr.node");
     assertNotNull(values.get("CONTAINER.cores.lazy")); // this is a gauge node
+    assertNotNull(values.get("CONTAINER.threadPool.coreContainerWorkExecutor.completed"));
+    assertNotNull(values.get("CONTAINER.threadPool.coreLoadExecutor.completed"));
 
     resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "prefix", "CONTAINER.cores"), resp);


[38/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7596: Update Groovy to version 2.4.8 to allow building with Java 9 build 148+. Also update JGit version for working-copy checks.

Posted by ho...@apache.org.
LUCENE-7596: Update Groovy to version 2.4.8 to allow building with Java 9 build 148+. Also update JGit version for working-copy checks.


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

Branch: refs/heads/jira/solr-5944
Commit: 5b3565ed7e6fe5a1d027f9f60e151a6fa2e2208b
Parents: 987e265
Author: Uwe Schindler <us...@apache.org>
Authored: Fri Jan 13 20:20:31 2017 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Fri Jan 13 20:20:31 2017 +0100

----------------------------------------------------------------------
 build.xml               | 2 +-
 lucene/CHANGES.txt      | 3 +++
 lucene/common-build.xml | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b3565ed/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index c06273c..07c01c2 100644
--- a/build.xml
+++ b/build.xml
@@ -20,7 +20,7 @@
 <project name="lucene-solr" default="-projecthelp" basedir=".">
   <import file="lucene/common-build.xml"/>
 
-  <property name="jgit-version" value="4.4.1.201607150455-r"/>
+  <property name="jgit-version" value="4.6.0.201612231935-r"/>
   
   <property name="tests.heap-dump-dir" location="heapdumps"/>
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b3565ed/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4912920..040f4e0 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -270,6 +270,9 @@ Build
   Lucene and Solr DOAP RDF files into the Git source repository under
   dev-tools/doap/ and then pulling release dates from those files, rather than
   from JIRA. (Mano Kovacs, hossman, Steve Rowe)
+  
+* LUCENE-7596: Update Groovy to version 2.4.8 to allow building with Java 9
+  build 148+. Also update JGit version for working-copy checks. (Uwe Schindler)
 
 ======================= Lucene 6.3.0 =======================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b3565ed/lucene/common-build.xml
----------------------------------------------------------------------
diff --git a/lucene/common-build.xml b/lucene/common-build.xml
index 42cd140..87d2e0a 100644
--- a/lucene/common-build.xml
+++ b/lucene/common-build.xml
@@ -2332,7 +2332,7 @@ ${ant.project.name}.test.dependencies=${test.classpath.list}
 
   <!-- GROOVY scripting engine for ANT tasks -->
   <target name="resolve-groovy" unless="groovy.loaded" depends="ivy-availability-check,ivy-configure">
-    <ivy:cachepath organisation="org.codehaus.groovy" module="groovy-all" revision="2.4.6"
+    <ivy:cachepath organisation="org.codehaus.groovy" module="groovy-all" revision="2.4.8"
       inline="true" conf="default" type="jar" transitive="true" pathid="groovy.classpath"/>
     <taskdef name="groovy"
       classname="org.codehaus.groovy.ant.Groovy"


[42/50] [abbrv] lucene-solr:jira/solr-5944: Add getMatchingChildren() method to Scorer

Posted by ho...@apache.org.
Add getMatchingChildren() method to Scorer


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

Branch: refs/heads/jira/solr-5944
Commit: 9403372fbc36aced848bf8aa498bc71b7b94567b
Parents: 8fa0a8d
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 14 09:08:02 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 14 10:28:51 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +++
 .../apache/lucene/search/DisjunctionScorer.java |  8 +++++
 .../lucene/search/MinShouldMatchSumScorer.java  |  9 ++++++
 .../java/org/apache/lucene/search/Scorer.java   |  8 +++++
 .../search/TestBooleanQueryVisitSubscorers.java | 33 ++++++++++++++++++++
 5 files changed, 62 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 58201d6..540188e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -130,6 +130,10 @@ New features
   SortedSetDocValues to allow filtering their TermsEnums with a
   CompiledAutomaton (Alan Woodward, Mike McCandless)
 
+* LUCENE-7628: Scorer now has a getMatchingChildren() method that will 
+  return all child scorers positioned on the current document.  (Alan
+  Woodward)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
index c53942a..a76999e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
@@ -202,4 +202,12 @@ abstract class DisjunctionScorer extends Scorer {
     return children;
   }
 
+  @Override
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    List<ChildScorer> children = new ArrayList<>();
+    for (DisiWrapper w = getSubMatches(); w != null; w = w.next) {
+      children.add(new ChildScorer(w.scorer, "SHOULD"));
+    }
+    return children;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index 032b5fe..b977400 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -133,6 +133,15 @@ final class MinShouldMatchSumScorer extends Scorer {
   }
 
   @Override
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    List<ChildScorer> children = new ArrayList<>();
+    for (DisiWrapper s = lead; s != null; s = s.next) {
+      children.add(new ChildScorer(s.scorer, "SHOULD"));
+    }
+    return children;
+  }
+
+  @Override
   public DocIdSetIterator iterator() {
     return new DocIdSetIterator() {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/java/org/apache/lucene/search/Scorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
index f434327..2e35e91 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
@@ -82,6 +82,14 @@ public abstract class Scorer {
   public Collection<ChildScorer> getChildren() {
     return Collections.emptyList();
   }
+
+  /**
+   * Returns child sub-scorers that match the current document
+   * @lucene.experimental
+   */
+  public Collection<ChildScorer> getMatchingChildren() throws IOException {
+    return getChildren();
+  }
   
   /** A child Scorer and its relationship to its parent.
    * the meaning of the relationship depends upon the parent query. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9403372f/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index 60ba528..0909d5d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -108,6 +108,39 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
     assertEquals(2, tfs.get(1).intValue()); // f2:search + f2:lucene
     assertEquals(2, tfs.get(2).intValue()); // f2:search + f2:lucene
   }
+
+  public void testDisjunctionMatches() throws IOException {
+    BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
+    bq1.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq1.add(new PhraseQuery(F2, "search", "engine"), Occur.SHOULD);
+
+    Weight w1 = scorerSearcher.createNormalizedWeight(bq1.build(), true);
+    Scorer s1 = w1.scorer(reader.leaves().get(0));
+    assertEquals(0, s1.iterator().nextDoc());
+    assertEquals(2, s1.getMatchingChildren().size());
+
+    BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
+    bq2.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq2.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
+
+    Weight w2 = scorerSearcher.createNormalizedWeight(bq2.build(), true);
+    Scorer s2 = w2.scorer(reader.leaves().get(0));
+    assertEquals(0, s2.iterator().nextDoc());
+    assertEquals(1, s2.getMatchingChildren().size());
+  }
+
+  public void testMinShouldMatchMatches() throws IOException {
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
+    bq.add(new TermQuery(new Term(F2, "lucene")), Occur.SHOULD);
+    bq.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
+    bq.setMinimumNumberShouldMatch(2);
+
+    Weight w = scorerSearcher.createNormalizedWeight(bq.build(), true);
+    Scorer s = w.scorer(reader.leaves().get(0));
+    assertEquals(0, s.iterator().nextDoc());
+    assertEquals(2, s.getMatchingChildren().size());
+  }
   
   public void testConjunctions() throws IOException {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();


[22/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9934: SolrTestCase.clearIndex has been improved to take advantage of low level test specific logic that clears the index metadata more completely then a normal *:* DBQ can due to update versioning

Posted by ho...@apache.org.
SOLR-9934: SolrTestCase.clearIndex has been improved to take advantage of low level test specific logic that clears the index metadata more completely then a normal *:* DBQ can due to update versioning


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

Branch: refs/heads/jira/solr-5944
Commit: 1d7379b680062eca766f0410e3db7ff9e9b34cb0
Parents: 17cd0f0
Author: Chris Hostetter <ho...@apache.org>
Authored: Mon Jan 9 09:53:55 2017 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Mon Jan 9 09:54:56 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +++
 .../org/apache/solr/search/TestRTGBase.java     | 12 -----------
 .../org/apache/solr/search/TestRecovery.java    | 13 ------------
 .../apache/solr/search/TestRecoveryHdfs.java    | 11 ----------
 .../apache/solr/update/CdcrUpdateLogTest.java   | 17 ---------------
 .../java/org/apache/solr/SolrTestCaseJ4.java    | 22 ++++++++++++++++++--
 6 files changed, 23 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d7379b6/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 49d24df..b28df9c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -408,6 +408,9 @@ Other Changes
 
 * SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching (Ishan Chattopadhyaya)
 
+* SOLR-9934: SolrTestCase.clearIndex has been improved to take advantage of low level test specific logic that
+  clears the index metadata more completely then a normal *:* DBQ can due to update versioning.  (hossman)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d7379b6/solr/core/src/test/org/apache/solr/search/TestRTGBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRTGBase.java b/solr/core/src/test/org/apache/solr/search/TestRTGBase.java
index 13ecb32..bb1b08a 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRTGBase.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRTGBase.java
@@ -36,24 +36,12 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.update.UpdateLog;
 
 import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
-import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
 public class TestRTGBase extends SolrTestCaseJ4 {
 
   // means we've seen the leader and have version info (i.e. we are a non-leader replica)
   public static String FROM_LEADER = DistribPhase.FROMLEADER.toString();
 
-  // since we make up fake versions in these tests, we can get messed up by a DBQ with a real version
-  // since Solr can think following updates were reordered.
-  @Override
-  public void clearIndex() {
-    try {
-      deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM,FROM_LEADER));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
   protected final ConcurrentHashMap<Integer,DocInfo> model = new ConcurrentHashMap<>();
   protected Map<Integer,DocInfo> committedModel = new HashMap<>();
   protected long snapshotCount;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d7379b6/solr/core/src/test/org/apache/solr/search/TestRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 12d3ec3..9b88ec4 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -72,19 +72,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
     }
   }
 
-
-  // since we make up fake versions in these tests, we can get messed up by a DBQ with a real version
-  // since Solr can think following updates were reordered.
-  @Override
-  public void clearIndex() {
-    try {
-      deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM,FROM_LEADER));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-
   @Test
   public void testLogReplay() throws Exception {
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d7379b6/solr/core/src/test/org/apache/solr/search/TestRecoveryHdfs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecoveryHdfs.java b/solr/core/src/test/org/apache/solr/search/TestRecoveryHdfs.java
index e7e60ec..e6bb9a6 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecoveryHdfs.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecoveryHdfs.java
@@ -109,17 +109,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
     dfsCluster = null;
   }
 
-  // since we make up fake versions in these tests, we can get messed up by a DBQ with a real version
-  // since Solr can think following updates were reordered.
-  @Override
-  public void clearIndex() {
-    try {
-      deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM,FROM_LEADER));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
   @Test
   public void testReplicationFactor() throws Exception {
     clearIndex(); 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d7379b6/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java b/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java
index 60dc9c3..b7409d1 100644
--- a/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java
+++ b/solr/core/src/test/org/apache/solr/update/CdcrUpdateLogTest.java
@@ -31,20 +31,14 @@ import org.apache.lucene.util.LuceneTestCase.Nightly;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.noggit.ObjectBuilder;
 
-import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
-
 @Nightly
 public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
 
-  // means that we've seen the leader and have version info (i.e. we are a non-leader replica)
-  private static String FROM_LEADER = DistributedUpdateProcessor.DistribPhase.FROMLEADER.toString();
-
   private static int timeout = 60;  // acquire timeout in seconds.  change this to a huge number when debugging to prevent threads from advancing.
 
   // TODO: fix this test to not require FSDirectory
@@ -66,17 +60,6 @@ public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
     }
   }
 
-  // since we make up fake versions in these tests, we can get messed up by a DBQ with a real version
-  // since Solr can think following updates were reordered.
-  @Override
-  public void clearIndex() {
-    try {
-      deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM, FROM_LEADER));
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
   private void clearCore() throws IOException {
     clearIndex();
     assertU(commit());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d7379b6/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 437679d..61de56d 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -132,6 +132,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
+import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
 import static java.util.Objects.requireNonNull;
 
 /**
@@ -1148,9 +1151,24 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     @Override
     public String toString() { return xml; }
   }
-  
+
+  /**
+   * Does a low level delete of all docs in the index. 
+   *
+   * The behavior of this method is slightly different then doing a normal <code>*:*</code> DBQ because it
+   * takes advantage of internal methods to ensure all index data is wiped, regardless of optimistic 
+   * concurrency version constraints -- making it suitable for tests that create synthetic versions, 
+   * and/or require a completely pristine index w/o any field metdata.
+   *
+   * @see #deleteByQueryAndGetVersion
+   */
   public void clearIndex() {
-    assertU(delQ("*:*"));
+    try {
+      deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE),
+                                               DISTRIB_UPDATE_PARAM,DistribPhase.FROMLEADER.toString()));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
   }
 
   /** Send JSON update commands */


[16/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching

Posted by ho...@apache.org.
SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching


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

Branch: refs/heads/jira/solr-5944
Commit: b0177312032e039673bfbbd42cd1dca09fb93833
Parents: 322ad88
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Jan 9 08:33:06 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Jan 9 08:33:06 2017 +0530

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0177312/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 11151d7..49d24df 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -406,6 +406,8 @@ Other Changes
 
 * SOLR-9944: Map the nodes function name to the GatherNodesStream (Joel Bernstein)
 
+* SOLR-9777: IndexFingerprinting should use getCombinedCoreAndDeletesKey() instead of getCoreCacheKey() for per-segment caching (Ishan Chattopadhyaya)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0177312/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 08072e1..697e008 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1687,7 +1687,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   public IndexFingerprint getIndexFingerprint(SolrIndexSearcher searcher, LeafReaderContext ctx, long maxVersion)
       throws IOException {
     IndexFingerprint f = null;
-    f = perSegmentFingerprintCache.get(ctx.reader().getCoreCacheKey());
+    f = perSegmentFingerprintCache.get(ctx.reader().getCombinedCoreAndDeletesKey());
     // fingerprint is either not cached or
     // if we want fingerprint only up to a version less than maxVersionEncountered in the segment, or
     // documents were deleted from segment for which fingerprint was cached
@@ -1698,7 +1698,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       // cache fingerprint for the segment only if all the versions in the segment are included in the fingerprint
       if (f.getMaxVersionEncountered() == f.getMaxInHash()) {
         log.info("Caching fingerprint for searcher:{} leafReaderContext:{} mavVersion:{}", searcher, ctx, maxVersion);
-        perSegmentFingerprintCache.put(ctx.reader().getCoreCacheKey(), f);
+        perSegmentFingerprintCache.put(ctx.reader().getCombinedCoreAndDeletesKey(), f);
       }
 
     } else {


[11/50] [abbrv] lucene-solr:jira/solr-5944: TokenStreamToAutomaton failed to handle certain holes correctly

Posted by ho...@apache.org.
TokenStreamToAutomaton failed to handle certain holes correctly


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

Branch: refs/heads/jira/solr-5944
Commit: e64111c6545d8e05241bc87eb4623c5ed44db312
Parents: 1aa9c42
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 8 06:26:08 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 8 06:26:08 2017 -0500

----------------------------------------------------------------------
 .../apache/lucene/analysis/TokenStreamToAutomaton.java  | 11 ++++++++++-
 .../org/apache/lucene/analysis/TestGraphTokenizers.java | 12 ++++++++++++
 2 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e64111c6/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java b/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
index 071fa4a..64bac66 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
@@ -113,6 +113,7 @@ public class TokenStreamToAutomaton {
     final RollingBuffer<Position> positions = new Positions();
 
     int pos = -1;
+    int freedPos = 0;
     Position posData = null;
     int maxOffset = 0;
     while (in.incrementToken()) {
@@ -150,7 +151,15 @@ public class TokenStreamToAutomaton {
             addHoles(builder, positions, pos);
           }
         }
-        positions.freeBefore(pos);
+        while (freedPos <= pos) {
+          Position freePosData = positions.get(freedPos);
+          // don't free this position yet if we may still need to fill holes over it:
+          if (freePosData.arriving == -1 || freePosData.leaving == -1) {
+            break;
+          }
+          positions.freeBefore(freedPos);
+          freedPos++;
+        }
       }
 
       final int endPos = pos + posLengthAtt.getPositionLength();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e64111c6/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
index 78fb127..8899dd1 100644
--- a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
+++ b/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
@@ -585,4 +585,16 @@ public class TestGraphTokenizers extends BaseTokenStreamTestCase {
       Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
       Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
   }
+
+  public void testTokenStreamGraphWithHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 1, 8),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    assertSameLanguage(Operations.union(join(s2a("abc"), SEP_A, s2a("xyz")),
+                                        join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"), SEP_A, s2a("ghi"))), ts);
+  }
 }


[31/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9918: Remove unused import to make precommit happy

Posted by ho...@apache.org.
SOLR-9918: Remove unused import to make precommit happy


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

Branch: refs/heads/jira/solr-5944
Commit: 2437204730130dc8c03efb111ec7d4db456189ed
Parents: d66bfba
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Wed Jan 11 12:20:42 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Wed Jan 11 12:20:42 2017 +0530

----------------------------------------------------------------------
 .../update/processor/SkipExistingDocumentsProcessorFactoryTest.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/24372047/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
index 2afe35c..7e17f71 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
@@ -20,7 +20,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
 import java.io.IOException;


[44/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9893: Update Mockito to version 2.6.2 for Java 9 support. Disable all legacy EasyMock tests when running on Java 9 until they were migrated to Mockito

Posted by ho...@apache.org.
SOLR-9893: Update Mockito to version 2.6.2 for Java 9 support. Disable all legacy EasyMock tests when running on Java 9 until they were migrated to Mockito


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

Branch: refs/heads/jira/solr-5944
Commit: 4eafdb337ab88553506e15bc186cd06464ea14f9
Parents: e2c41af
Author: Uwe Schindler <us...@apache.org>
Authored: Sat Jan 14 12:43:24 2017 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Sat Jan 14 12:43:24 2017 +0100

----------------------------------------------------------------------
 lucene/ivy-versions.properties                  |   5 +-
 solr/CHANGES.txt                                |   3 +
 .../handler/dataimport/TestJdbcDataSource.java  |   7 +
 solr/contrib/morphlines-core/ivy.xml            |   8 +-
 solr/core/ivy.xml                               |   4 +-
 .../org/apache/solr/cloud/ClusterStateTest.java |   8 +
 ...verseerCollectionConfigSetProcessorTest.java |   4 +-
 .../solr/core/BlobRepositoryMockingTest.java    |   9 +-
 .../org/apache/solr/core/CoreSorterTest.java    |   7 +
 .../security/TestPKIAuthenticationPlugin.java   |   8 +
 .../solr/servlet/SolrRequestParserTest.java     |   3 +
 solr/licenses/byte-buddy-1.6.2.jar.sha1         |   1 +
 solr/licenses/byte-buddy-LICENSE-ASL.txt        | 202 +++++++++++++++++++
 solr/licenses/byte-buddy-NOTICE.txt             |   4 +
 solr/licenses/mockito-core-1.9.5.jar.sha1       |   1 -
 solr/licenses/mockito-core-2.6.2.jar.sha1       |   1 +
 solr/licenses/objenesis-1.2.jar.sha1            |   1 -
 solr/licenses/objenesis-2.4.jar.sha1            |   1 +
 .../solrj/impl/CloudSolrClientCacheTest.java    |   7 +
 19 files changed, 275 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 297d46c..770649c 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -96,6 +96,7 @@ io.netty.netty-all.version = 4.0.36.Final
 /mecab/mecab-ipadic = 2.7.0-20070801
 /mecab/mecab-naist-jdic = 0.6.3b-20111013
 /net.arnx/jsonic = 1.2.7
+/net.bytebuddy/byte-buddy = 1.6.2
 /net.sf.ehcache/ehcache-core = 2.4.4
 /net.sf.saxon/Saxon-HE = 9.6.0-2
 /net.sourceforge.argparse4j/argparse4j = 0.4.3
@@ -287,14 +288,14 @@ org.kitesdk.kite-morphlines.version = 1.1.0
 
 /org.locationtech.spatial4j/spatial4j = 0.6
 
-/org.mockito/mockito-core = 1.9.5
+/org.mockito/mockito-core = 2.6.2
 
 org.mortbay.jetty.version = 6.1.26
 /org.mortbay.jetty/jetty = ${org.mortbay.jetty.version}
 /org.mortbay.jetty/jetty-util = ${org.mortbay.jetty.version}
 
 /org.noggit/noggit = 0.6
-/org.objenesis/objenesis = 1.2
+/org.objenesis/objenesis = 2.4
 
 org.ow2.asm.version = 5.1
 /org.ow2.asm/asm = ${org.ow2.asm.version}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ccc5d7c..f8ecf23 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -437,6 +437,9 @@ Other Changes
 * SOLR-9934: SolrTestCase.clearIndex has been improved to take advantage of low level test specific logic that
   clears the index metadata more completely then a normal *:* DBQ can due to update versioning.  (hossman)
 
+* SOLR-9893: Update Mockito to version 2.6.2 for Java 9 support. Disable all legacy EasyMock tests when running
+  on Java 9 until they were migrated to Mockito. (Uwe Schindler)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
index 01340bc..12e34be 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
@@ -35,11 +35,13 @@ import java.util.Properties;
 
 import javax.sql.DataSource;
 
+import org.apache.lucene.util.Constants;
 import org.apache.solr.handler.dataimport.JdbcDataSource.ResultSetIterator;
 import org.easymock.EasyMock;
 import org.easymock.IMocksControl;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -68,6 +70,11 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
   Properties props = new Properties();
 
   String sysProp = System.getProperty("java.naming.factory.initial");
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+  }
 
   @Override
   @Before

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/contrib/morphlines-core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/ivy.xml b/solr/contrib/morphlines-core/ivy.xml
index f7fd005..ad47aec 100644
--- a/solr/contrib/morphlines-core/ivy.xml
+++ b/solr/contrib/morphlines-core/ivy.xml
@@ -69,13 +69,17 @@
 
     <dependency org="aopalliance" name="aopalliance" rev="${/aopalliance/aopalliance}" conf="test" />
     <dependency org="com.sun.xml.bind" name="jaxb-impl" rev="${/com.sun.xml.bind/jaxb-impl}" conf="test" />
-    <dependency org="org.objenesis" name="objenesis" rev="${/org.objenesis/objenesis}" conf="test" />
     <dependency org="io.netty" name="netty-all" rev="${/io.netty/netty-all}" conf="test" />
     <dependency org="org.apache.mrunit" name="mrunit" rev="${/org.apache.mrunit/mrunit}" conf="test">
       <artifact name="mrunit" maven:classifier="hadoop2" />
       <exclude org="log4j" module="log4j" />
     </dependency>
-    <dependency org="org.mockito" name="mockito-core" rev="${/org.mockito/mockito-core}" conf="test" />
+    
+    <!-- Mocking -->
+    <dependency org="org.mockito" name="mockito-core" rev="${/org.mockito/mockito-core}" conf="test"/>
+    <dependency org="net.bytebuddy" name="byte-buddy" rev="${/net.bytebuddy/byte-buddy}" conf="test"/>
+    <dependency org="org.objenesis" name="objenesis" rev="${/org.objenesis/objenesis}" conf="test"/>
+    
     <dependency org="commons-collections" name="commons-collections" rev="${/commons-collections/commons-collections}" conf="test" />
 
     <!-- FasterXml Jackson Dependencies -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 67e4379..074e35f 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -51,11 +51,13 @@
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="compile"/>
     <dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="compile"/>
 
+    <!-- TODO: Nuke those 2 deps, please!!!! -->
     <dependency org="org.easymock" name="easymock" rev="${/org.easymock/easymock}" conf="test"/>
     <dependency org="cglib" name="cglib-nodep" rev="${/cglib/cglib-nodep}" conf="test"/>
-    <dependency org="org.objenesis" name="objenesis" rev="${/org.objenesis/objenesis}" conf="test"/>
 
     <dependency org="org.mockito" name="mockito-core" rev="${/org.mockito/mockito-core}" conf="test"/>
+    <dependency org="net.bytebuddy" name="byte-buddy" rev="${/net.bytebuddy/byte-buddy}" conf="test"/>
+    <dependency org="org.objenesis" name="objenesis" rev="${/org.objenesis/objenesis}" conf="test"/>
 
     <dependency org="com.fasterxml.jackson.core" name="jackson-core" rev="${/com.fasterxml.jackson.core/jackson-core}" conf="compile"/>
     <dependency org="com.fasterxml.jackson.core" name="jackson-databind" rev="${/com.fasterxml.jackson.core/jackson-databind}" conf="test"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
index 704c877..5911cbe 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java
@@ -21,6 +21,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -30,11 +31,18 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
 import org.easymock.EasyMock;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.easymock.EasyMock.createMock;
 
 public class ClusterStateTest extends SolrTestCaseJ4 {
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+  }
+
   @Test
   public void testStoreAndRead() throws Exception {
     Map<String,DocCollection> collectionStates = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index 6a7906d..d9f53af 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -23,6 +23,7 @@ import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 
+import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.response.QueryResponse;
@@ -103,9 +104,10 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     }
     
   }
-  
+
   @BeforeClass
   public static void setUpOnce() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
     workQueueMock = createMock(OverseerTaskQueue.class);
     runningMapMock = createMock(DistributedMap.class);
     completedMapMock = createMock(DistributedMap.class);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
index 6da1367..355cb61 100644
--- a/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
+++ b/solr/core/src/test/org/apache/solr/core/BlobRepositoryMockingTest.java
@@ -25,10 +25,13 @@ import java.nio.charset.Charset;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.common.SolrException;
 import org.easymock.EasyMock;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.easymock.EasyMock.anyObject;
@@ -64,7 +67,11 @@ public class BlobRepositoryMockingTest {
   boolean blobFetched = false;
   String blobKey = "";
 
-
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    LuceneTestCase.assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+  }
+  
   @Before
   public void setUp() throws IllegalAccessException, NoSuchFieldException {
     blobFetched = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
index dda437a..f0c4b8c 100644
--- a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
+++ b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
+import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
@@ -35,6 +36,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreSorter.CountsForEachShard;
 import org.apache.solr.util.MockCoreContainer;
+import org.junit.BeforeClass;
 
 import static java.util.stream.Collectors.toList;
 import static org.apache.solr.core.CoreSorter.getShardName;
@@ -47,6 +49,11 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
   Map<String, Boolean> nodes = new LinkedHashMap<>();
   Set<String> liveNodes = new HashSet<>();
 
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+  }
+  
   public void testComparator() {
     List<CountsForEachShard> l = new ArrayList<>();
     //                           DOWN LIVE  MY

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
index 1eb1d21..f602b1b 100644
--- a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.http.Header;
 import org.apache.http.auth.BasicUserPrincipal;
 import org.apache.http.message.BasicHttpRequest;
+import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.CoreContainer;
@@ -36,6 +37,8 @@ import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.CryptoKeys;
 import org.easymock.EasyMock;
+import org.junit.BeforeClass;
+
 import static org.easymock.EasyMock.getCurrentArguments;
 
 public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
@@ -70,6 +73,11 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     }
   }
 
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+  }
+  
   public void test() throws Exception {
     AtomicReference<Principal> principal = new AtomicReference<>();
     String nodeName = "node_x_233";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java b/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
index 53ef7a6..b9e1e4a 100644
--- a/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
+++ b/solr/core/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
@@ -42,6 +42,7 @@ import javax.servlet.http.HttpServletRequest;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
+import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
@@ -66,6 +67,8 @@ public class SolrRequestParserTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+    
     initCore("solrconfig.xml", "schema.xml");
     parser = new SolrRequestParsers( h.getCore().getSolrConfig() );
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/byte-buddy-1.6.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/byte-buddy-1.6.2.jar.sha1 b/solr/licenses/byte-buddy-1.6.2.jar.sha1
new file mode 100644
index 0000000..af3a381
--- /dev/null
+++ b/solr/licenses/byte-buddy-1.6.2.jar.sha1
@@ -0,0 +1 @@
+f58a01d36e24a94241d44d52c78e380396d5adb2

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/byte-buddy-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/byte-buddy-LICENSE-ASL.txt b/solr/licenses/byte-buddy-LICENSE-ASL.txt
new file mode 100644
index 0000000..e06d208
--- /dev/null
+++ b/solr/licenses/byte-buddy-LICENSE-ASL.txt
@@ -0,0 +1,202 @@
+Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "{}"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright {yyyy} {name of copyright owner}
+
+   Licensed 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.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/byte-buddy-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/byte-buddy-NOTICE.txt b/solr/licenses/byte-buddy-NOTICE.txt
new file mode 100644
index 0000000..731a995
--- /dev/null
+++ b/solr/licenses/byte-buddy-NOTICE.txt
@@ -0,0 +1,4 @@
+Byte Buddy is a code generation and manipulation library for creating and modifying Java
+classes during the runtime of a Java application and without the help of a compiler.
+
+Copyright 2014 Rafael Winterhalter

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/mockito-core-1.9.5.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/mockito-core-1.9.5.jar.sha1 b/solr/licenses/mockito-core-1.9.5.jar.sha1
deleted file mode 100644
index 5de9041..0000000
--- a/solr/licenses/mockito-core-1.9.5.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-c3264abeea62c4d2f367e21484fbb40c7e256393

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/mockito-core-2.6.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/mockito-core-2.6.2.jar.sha1 b/solr/licenses/mockito-core-2.6.2.jar.sha1
new file mode 100644
index 0000000..f130b90
--- /dev/null
+++ b/solr/licenses/mockito-core-2.6.2.jar.sha1
@@ -0,0 +1 @@
+9eeaa7c2a971cd4738e1b9391a38ba4f21f05763

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/objenesis-1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/objenesis-1.2.jar.sha1 b/solr/licenses/objenesis-1.2.jar.sha1
deleted file mode 100644
index 1252cc6..0000000
--- a/solr/licenses/objenesis-1.2.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-bfcb0539a071a4c5a30690388903ac48c0667f2a

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/licenses/objenesis-2.4.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/objenesis-2.4.jar.sha1 b/solr/licenses/objenesis-2.4.jar.sha1
new file mode 100644
index 0000000..278f7dd
--- /dev/null
+++ b/solr/licenses/objenesis-2.4.jar.sha1
@@ -0,0 +1 @@
+2916b6c96b50c5b3ec4452ed99401db745aabb27

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4eafdb33/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
index 415c658..c144890 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
@@ -31,17 +31,24 @@ import java.util.function.Function;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.http.NoHttpResponseException;
+import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.util.NamedList;
 import org.easymock.EasyMock;
+import org.junit.BeforeClass;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 public class CloudSolrClientCacheTest extends SolrTestCaseJ4 {
 
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    assumeFalse("SOLR-9893: EasyMock does not work with Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+  }
+  
   public void testCaching() throws Exception {
     String collName = "gettingstarted";
     Set<String> livenodes = new HashSet<>();


[26/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9950 Check the difference in counts - meter may not be zero at this point.

Posted by ho...@apache.org.
SOLR-9950 Check the difference in counts - meter may not be zero at this point.


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

Branch: refs/heads/jira/solr-5944
Commit: 98422e0dc0c7de4635e1bc80bcd5ca70a8d2761a
Parents: 2048b82
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Jan 10 11:41:40 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Jan 10 11:44:16 2017 +0100

----------------------------------------------------------------------
 .../test/org/apache/solr/search/TestRecovery.java    | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/98422e0d/solr/core/src/test/org/apache/solr/search/TestRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 7bd0951..f3e98dd 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -139,6 +139,8 @@ public class TestRecovery extends SolrTestCaseJ4 {
       assertTrue(replayingLogs.getValue().intValue() > 0);
       Gauge<Long> replayingDocs = (Gauge<Long>)metrics.get("TLOG.replay.remaining.bytes");
       assertTrue(replayingDocs.getValue().longValue() > 0);
+      Meter replayDocs = (Meter)metrics.get("TLOG.replay.ops");
+      long initialOps = replayDocs.getCount();
 
       // unblock recovery
       logReplay.release(1000);
@@ -151,8 +153,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
 
       assertJQ(req("q","*:*") ,"/response/numFound==3");
 
-      Meter replayDocs = (Meter)metrics.get("TLOG.replay.ops");
-      assertEquals(5L, replayDocs.getCount());
+      assertEquals(5L, replayDocs.getCount() - initialOps);
       assertEquals(UpdateLog.State.ACTIVE.ordinal(), state.getValue().intValue());
 
       // make sure we can still access versions after recovery
@@ -236,6 +237,10 @@ public class TestRecovery extends SolrTestCaseJ4 {
       assertEquals(UpdateLog.State.BUFFERING, ulog.getState());
       Gauge<Integer> state = (Gauge<Integer>)metrics.get("TLOG.state");
       assertEquals(UpdateLog.State.BUFFERING.ordinal(), state.getValue().intValue());
+      Gauge<Integer> bufferedOps = (Gauge<Integer>)metrics.get("TLOG.buffered.ops");
+      int initialOps = bufferedOps.getValue();
+      Meter applyingBuffered = (Meter)metrics.get("TLOG.applying_buffered.ops");
+      long initialApplyingOps = applyingBuffered.getCount();
 
       // simulate updates from a leader
       updateJ(jsonAdd(sdoc("id","B1", "_version_","1010")), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
@@ -267,8 +272,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
           ,"=={'doc':null}"
       );
 
-      Gauge<Integer> bufferedOps = (Gauge<Integer>)metrics.get("TLOG.buffered.ops");
-      assertEquals(6, bufferedOps.getValue().intValue());
+      assertEquals(6, bufferedOps.getValue().intValue() - initialOps);
 
       rinfoFuture = ulog.applyBufferedUpdates();
       assertTrue(rinfoFuture != null);
@@ -280,8 +284,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
       UpdateLog.RecoveryInfo rinfo = rinfoFuture.get();
       assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
 
-      Meter applyingBuffered = (Meter)metrics.get("TLOG.applying_buffered.ops");
-      assertEquals(6L, applyingBuffered.getCount());
+      assertEquals(6L, applyingBuffered.getCount() - initialApplyingOps);
 
       assertJQ(req("qt","/get", "getVersions","6")
           ,"=={'versions':[-2010,1030,1020,-1017,1015,1010]}"


[28/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9584: Support Solr being proxied with another endpoint than default /solr This closes #86 - see original commit e0b4caccd3312b011cdfbb3951ea43812486ca98

Posted by ho...@apache.org.
SOLR-9584: Support Solr being proxied with another endpoint than default /solr
This closes #86 - see original commit e0b4caccd3312b011cdfbb3951ea43812486ca98


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

Branch: refs/heads/jira/solr-5944
Commit: f99c9676325c1749e570b9337a8c67a089d1fb28
Parents: e0b4cac
Author: Jan H�ydahl <ja...@apache.org>
Authored: Tue Jan 10 14:32:38 2017 +0100
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Tue Jan 10 14:32:38 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f99c9676/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2a5d5bb..0131b7b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -71,6 +71,8 @@ Optimizations
   string fields from the FieldCache, resulting in up to 56% better throughput for those cases.
   (yonik)
 
+* SOLR-9584: Support Solr being proxied with another endpoint than default /solr, by using relative links
+  in AdminUI javascripts (Yun Jie Zhou via janhoy)
 
 ==================  6.4.0 ==================
 


[04/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7611: Remove unnecessary Exception wrapping from DocumentValueSourceDictionary

Posted by ho...@apache.org.
LUCENE-7611: Remove unnecessary Exception wrapping from DocumentValueSourceDictionary


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

Branch: refs/heads/jira/solr-5944
Commit: 67261d2fb515f255e05c138281ab6c6b71d66716
Parents: 8f4fee3
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 7 16:06:29 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 16:06:29 2017 +0000

----------------------------------------------------------------------
 .../suggest/DocumentValueSourceDictionary.java  | 21 +++++++-------------
 1 file changed, 7 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67261d2f/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 2291ac9..9356975 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -132,27 +132,20 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
      * by the <code>weightsValueSource</code>
      * */
     @Override
-    protected long getWeight(Document doc, int docId) {
+    protected long getWeight(Document doc, int docId) throws IOException {
       if (currentWeightValues == null) {
         return 0;
       }
       int subIndex = ReaderUtil.subIndex(docId, starts);
       if (subIndex != currentLeafIndex) {
         currentLeafIndex = subIndex;
-        try {
-          currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-      try {
-        if (currentWeightValues.advanceExact(docId - starts[subIndex]))
-          return currentWeightValues.longValue();
-        else
-          return 0;
-      } catch (IOException e) {
-        throw new RuntimeException(e);
+        currentWeightValues = weightsValueSource.getValues(leaves.get(currentLeafIndex), null);
       }
+      if (currentWeightValues.advanceExact(docId - starts[subIndex]))
+        return currentWeightValues.longValue();
+      else
+        return 0;
+
     }
 
   }


[13/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9859: Don't log error on NoSuchFileException (Cao Manh Dat)

Posted by ho...@apache.org.
SOLR-9859: Don't log error on NoSuchFileException (Cao Manh Dat)


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

Branch: refs/heads/jira/solr-5944
Commit: 25290ab5d6af25c05cbbb4738f49329273a7d693
Parents: f985fca
Author: markrmiller <ma...@apache.org>
Authored: Sun Jan 8 09:21:43 2017 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Sun Jan 8 09:21:43 2017 -0500

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/core/DirectoryFactory.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/25290ab5/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index 136a0a6..ac18d7e 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -22,6 +22,7 @@ import java.io.FileFilter;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.NoSuchFileException;
 import java.util.Collection;
 import java.util.Collections;
 
@@ -191,7 +192,7 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
     try {
       dir.deleteFile(toName);
-    } catch (FileNotFoundException e) {
+    } catch (FileNotFoundException | NoSuchFileException e) {
 
     } catch (Exception e) {
       log.error("Exception deleting file", e);


[34/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9948: Add a way to configure granularity of metrics for http connections

Posted by ho...@apache.org.
SOLR-9948: Add a way to configure granularity of metrics for http connections


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

Branch: refs/heads/jira/solr-5944
Commit: d2664b100463ada22162d53aad1c6d306d2cc9c1
Parents: 7435ab1
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Jan 12 13:00:00 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Jan 12 13:00:00 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |  9 ++-
 .../component/HttpShardHandlerFactory.java      | 15 +++-
 .../apache/solr/update/UpdateShardHandler.java  | 14 +++-
 .../solr/update/UpdateShardHandlerConfig.java   | 14 +++-
 .../stats/HttpClientMetricNameStrategy.java     | 28 +++++++
 .../stats/InstrumentedHttpRequestExecutor.java  | 81 +++++++++++++++-----
 .../java/org/apache/solr/util/TestHarness.java  |  3 +-
 8 files changed, 143 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index afcd10b..0cf50d4 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -209,7 +209,9 @@ New Features
 
 * SOLR-9725: Substitute properties into JdbcDataSource configuration ( Jamie Jackson, Yuri Sashevsky via Mikhail Khludnev)
 
-* SOLR-9877: SOLR-9923: Use instrumented http client and connection pool. (shalin)
+* SOLR-9877: SOLR-9923: SOLR-9948: Use instrumented http client and connection pool in HttpShardHandler and
+  UpdateShardHandler. The metrics are collected per query-less URL and method by default but it can be configured
+  to host/method and per-method as well. (shalin)
 
 * SOLR-9880: Add Ganglia, Graphite and SLF4J metrics reporters. (ab)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 49d9ae5..e41cd8d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -282,6 +282,7 @@ public class SolrXmlConfig {
     int maxUpdateConnectionsPerHost = UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST;
     int distributedSocketTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATESOTIMEOUT;
     int distributedConnectionTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
+    String metricNameStrategy = UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY;
 
     Object muc = nl.remove("maxUpdateConnections");
     if (muc != null) {
@@ -307,10 +308,16 @@ public class SolrXmlConfig {
       defined = true;
     }
 
+    Object mns = nl.remove("metricNameStrategy");
+    if (mns != null)  {
+      metricNameStrategy = mns.toString();
+      defined = true;
+    }
+
     if (!defined && !alwaysDefine)
       return null;
 
-    return new UpdateShardHandlerConfig(maxUpdateConnections, maxUpdateConnectionsPerHost, distributedSocketTimeout, distributedConnectionTimeout);
+    return new UpdateShardHandlerConfig(maxUpdateConnections, maxUpdateConnectionsPerHost, distributedSocketTimeout, distributedConnectionTimeout, metricNameStrategy);
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 258be97..be6e763 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -41,6 +42,7 @@ import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
 import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager;
 import org.apache.solr.util.stats.MetricUtils;
@@ -61,6 +63,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
+
 
 public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized, SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -97,6 +101,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
 
   private String scheme = null;
 
+  private HttpClientMetricNameStrategy metricNameStrategy;
+
   private final Random r = new Random();
 
   private final ReplicaListTransformer shufflingReplicaListTransformer = new ShufflingReplicaListTransformer(r);
@@ -148,6 +154,13 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
       this.scheme = StringUtils.removeEnd(this.scheme, "://");
     }
 
+    String strategy = getParameter(args, "metricNameStrategy", UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY, sb);
+    this.metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(strategy);
+    if (this.metricNameStrategy == null)  {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unknown metricNameStrategy: " + strategy + " found. Must be one of: " + KNOWN_METRIC_NAME_STRATEGIES.keySet());
+    }
+
     this.connectionTimeout = getParameter(args, HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout, sb);
     this.maxConnectionsPerHost = getParameter(args, HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, maxConnectionsPerHost,sb);
     this.maxConnections = getParameter(args, HttpClientUtil.PROP_MAX_CONNECTIONS, maxConnections,sb);
@@ -177,7 +190,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
     );
 
     ModifiableSolrParams clientParams = getClientParams();
-    httpRequestExecutor = new InstrumentedHttpRequestExecutor();
+    httpRequestExecutor = new InstrumentedHttpRequestExecutor(this.metricNameStrategy);
     clientConnectionManager = new InstrumentedPoolingHttpClientConnectionManager(HttpClientUtil.getSchemaRegisteryProvider().getSchemaRegistry());
     this.defaultClient = HttpClientUtil.createClient(clientParams, clientConnectionManager, false, httpRequestExecutor);
     this.loadbalancer = createLoadbalancer(defaultClient);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index f13cfb5..67447a3 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -34,11 +34,14 @@ import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
 import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
+
 public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
   
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -74,7 +77,16 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
       clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT, cfg.getDistributedSocketTimeout());
       clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, cfg.getDistributedConnectionTimeout());
     }
-    httpRequestExecutor = new InstrumentedHttpRequestExecutor();
+    HttpClientMetricNameStrategy metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
+    if (cfg != null)  {
+      metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(cfg.getMetricNameStrategy());
+      if (metricNameStrategy == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Unknown metricNameStrategy: " + cfg.getMetricNameStrategy() + " found. Must be one of: " + KNOWN_METRIC_NAME_STRATEGIES.keySet());
+      }
+    }
+
+    httpRequestExecutor = new InstrumentedHttpRequestExecutor(metricNameStrategy);
     client = HttpClientUtil.createClient(clientParams, clientConnectionManager, false, httpRequestExecutor);
 
     // following is done only for logging complete configuration.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
index ffb06c4..d31ce50 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
@@ -22,10 +22,12 @@ public class UpdateShardHandlerConfig {
   public static final int DEFAULT_DISTRIBUPDATESOTIMEOUT = 600000;
   public static final int DEFAULT_MAXUPDATECONNECTIONS = 100000;
   public static final int DEFAULT_MAXUPDATECONNECTIONSPERHOST = 100000;
+  public static final String DEFAULT_METRICNAMESTRATEGY = "queryLessURLAndMethod";
 
   public static final UpdateShardHandlerConfig DEFAULT
       = new UpdateShardHandlerConfig(DEFAULT_MAXUPDATECONNECTIONS, DEFAULT_MAXUPDATECONNECTIONSPERHOST,
-                                     DEFAULT_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT);
+                                     DEFAULT_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT,
+                                      DEFAULT_METRICNAMESTRATEGY);
 
   private final int maxUpdateConnections;
 
@@ -35,11 +37,15 @@ public class UpdateShardHandlerConfig {
 
   private final int distributedConnectionTimeout;
 
-  public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout) {
+  private final String metricNameStrategy;
+
+  public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout,
+                                  String metricNameStrategy) {
     this.maxUpdateConnections = maxUpdateConnections;
     this.maxUpdateConnectionsPerHost = maxUpdateConnectionsPerHost;
     this.distributedSocketTimeout = distributedSocketTimeout;
     this.distributedConnectionTimeout = distributedConnectionTimeout;
+    this.metricNameStrategy = metricNameStrategy;
   }
 
   public int getMaxUpdateConnectionsPerHost() {
@@ -57,4 +63,8 @@ public class UpdateShardHandlerConfig {
   public int getDistributedConnectionTimeout() {
     return distributedConnectionTimeout;
   }
+
+  public String getMetricNameStrategy() {
+    return metricNameStrategy;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java b/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java
new file mode 100644
index 0000000..930c5f7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java
@@ -0,0 +1,28 @@
+/*
+ * 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.util.stats;
+
+import org.apache.http.HttpRequest;
+
+/**
+ * Strategy for creating metric names for HttpClient
+ * Copied from metrics-httpclient library
+ */
+public interface HttpClientMetricNameStrategy {
+  String getNameFor(String scope, HttpRequest request);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
index 0caa2d1..bd8d368 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
@@ -19,7 +19,9 @@ package org.apache.solr.util.stats;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.HashMap;
 import java.util.Locale;
+import java.util.Map;
 
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
@@ -35,13 +37,72 @@ import org.apache.http.protocol.HttpRequestExecutor;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 
+import static org.apache.solr.metrics.SolrMetricManager.mkName;
+
 /**
  * Sub-class of HttpRequestExecutor which tracks metrics interesting to solr
  * Inspired and partially copied from dropwizard httpclient library
  */
 public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor implements SolrMetricProducer {
+  public static final HttpClientMetricNameStrategy QUERYLESS_URL_AND_METHOD =
+      (scope, request) -> {
+        try {
+          final RequestLine requestLine = request.getRequestLine();
+          String schemeHostPort = null;
+          if (request instanceof HttpRequestWrapper) {
+            HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
+            if (wrapper.getTarget() != null) {
+              schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" + wrapper.getTarget().getPort();
+            }
+          }
+          final URIBuilder url = new URIBuilder(requestLine.getUri());
+          return mkName((schemeHostPort != null ? schemeHostPort : "") + url.removeQuery().build().toString() + "." + methodNameString(request), scope);
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(e);
+        }
+      };
+
+  public static final HttpClientMetricNameStrategy METHOD_ONLY =
+      (scope, request) -> mkName(methodNameString(request), scope);
+
+  public static final HttpClientMetricNameStrategy HOST_AND_METHOD =
+      (scope, request) -> {
+        try {
+          final RequestLine requestLine = request.getRequestLine();
+          String schemeHostPort = null;
+          if (request instanceof HttpRequestWrapper) {
+            HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
+            if (wrapper.getTarget() != null) {
+              schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" + wrapper.getTarget().getPort();
+            }
+          }
+          final URIBuilder url = new URIBuilder(requestLine.getUri());
+          return mkName((schemeHostPort != null ? schemeHostPort : "") + "." + methodNameString(request), scope);
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(e);
+        }
+      };
+
+  public static final Map<String, HttpClientMetricNameStrategy> KNOWN_METRIC_NAME_STRATEGIES = new HashMap<>(3);
+
+  static  {
+    KNOWN_METRIC_NAME_STRATEGIES.put("queryLessURLAndMethod", QUERYLESS_URL_AND_METHOD);
+    KNOWN_METRIC_NAME_STRATEGIES.put("hostAndMethod", HOST_AND_METHOD);
+    KNOWN_METRIC_NAME_STRATEGIES.put("methodOnly", METHOD_ONLY);
+  }
+
   protected MetricRegistry metricsRegistry;
   protected String scope;
+  protected HttpClientMetricNameStrategy nameStrategy;
+
+  public InstrumentedHttpRequestExecutor(int waitForContinue, HttpClientMetricNameStrategy nameStrategy) {
+    super(waitForContinue);
+    this.nameStrategy = nameStrategy;
+  }
+
+  public InstrumentedHttpRequestExecutor(HttpClientMetricNameStrategy nameStrategy) {
+    this.nameStrategy = nameStrategy;
+  }
 
   private static String methodNameString(HttpRequest request) {
     return request.getRequestLine().getMethod().toLowerCase(Locale.ROOT) + ".requests";
@@ -50,7 +111,7 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
   @Override
   public HttpResponse execute(HttpRequest request, HttpClientConnection conn, HttpContext context) throws IOException, HttpException {
     Timer.Context timerContext = null;
-    if (metricsRegistry != null)  {
+    if (metricsRegistry != null) {
       timerContext = timer(request).time();
     }
     try {
@@ -63,7 +124,7 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
   }
 
   private Timer timer(HttpRequest request) {
-    return metricsRegistry.timer(getNameFor(request));
+    return metricsRegistry.timer(nameStrategy.getNameFor(scope, request));
   }
 
   @Override
@@ -72,20 +133,4 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
     this.scope = scope;
   }
 
-  private String getNameFor(HttpRequest request) {
-    try {
-      final RequestLine requestLine = request.getRequestLine();
-      String schemeHostPort = null;
-      if (request instanceof HttpRequestWrapper) {
-        HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
-        if (wrapper.getTarget() != null)  {
-          schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" +  wrapper.getTarget().getPort();
-        }
-      }
-      final URIBuilder url = new URIBuilder(requestLine.getUri());
-      return SolrMetricManager.mkName((schemeHostPort != null ? schemeHostPort : "") + url.removeQuery().build().toString() + "." + methodNameString(request), scope);
-    } catch (URISyntaxException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index be8a24c..2386681 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -190,7 +190,8 @@ public class TestHarness extends BaseTestHarness {
     UpdateShardHandlerConfig updateShardHandlerConfig
         = new UpdateShardHandlerConfig(UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONS,
                                        UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST,
-                                       30000, 30000);
+                                       30000, 30000,
+                                        UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
     // universal default metric reporter
     Map<String,String> attributes = new HashMap<>();
     attributes.put("name", "default");


[20/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7624: Move TermsQuery into core as TermInSetQuery

Posted by ho...@apache.org.
LUCENE-7624: Move TermsQuery into core as TermInSetQuery


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

Branch: refs/heads/jira/solr-5944
Commit: 22940f5c49297b606d710c6775309d67ff064f2f
Parents: 5e9f927
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Jan 9 14:01:33 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Jan 9 14:25:05 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/TermInSetQuery.java    | 369 +++++++++++++++++++
 .../search/UsageTrackingQueryCachingPolicy.java |   5 +-
 .../lucene/search/TermInSetQueryTest.java       | 328 +++++++++++++++++
 .../apache/lucene/facet/MultiFacetQuery.java    |  12 +-
 .../org/apache/lucene/queries/TermsQuery.java   | 332 +----------------
 .../apache/lucene/queries/TermsQueryTest.java   | 339 -----------------
 .../prefix/TermQueryPrefixTreeStrategy.java     |  10 +-
 .../spatial/prefix/NumberRangeFacetsTest.java   |   6 +-
 .../solr/handler/component/ExpandComponent.java |  25 +-
 .../java/org/apache/solr/schema/FieldType.java  |   4 +-
 .../apache/solr/search/TermsQParserPlugin.java  |   4 +-
 .../org/apache/solr/search/join/GraphQuery.java |   4 +-
 .../apache/solr/search/TestSolrQueryParser.java |  12 +-
 14 files changed, 748 insertions(+), 705 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4bbf9ee..109a534 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -85,6 +85,9 @@ API Changes
 * LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
   as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
 
+* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
+  (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
new file mode 100644
index 0000000..e1a1575
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -0,0 +1,369 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.PrefixCodedTerms;
+import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Specialization for a disjunction over many terms that behaves like a
+ * {@link ConstantScoreQuery} over a {@link BooleanQuery} containing only
+ * {@link org.apache.lucene.search.BooleanClause.Occur#SHOULD} clauses.
+ * <p>For instance in the following example, both @{code q1} and {@code q2}
+ * would yield the same scores:
+ * <pre class="prettyprint">
+ * Query q1 = new TermInSetQuery(new Term("field", "foo"), new Term("field", "bar"));
+ *
+ * BooleanQuery bq = new BooleanQuery();
+ * bq.add(new TermQuery(new Term("field", "foo")), Occur.SHOULD);
+ * bq.add(new TermQuery(new Term("field", "bar")), Occur.SHOULD);
+ * Query q2 = new ConstantScoreQuery(bq);
+ * </pre>
+ * <p>When there are few terms, this query executes like a regular disjunction.
+ * However, when there are many terms, instead of merging iterators on the fly,
+ * it will populate a bit set with matching docs and return a {@link Scorer}
+ * over this bit set.
+ * <p>NOTE: This query produces scores that are equal to its boost
+ */
+public class TermInSetQuery extends Query implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermInSetQuery.class);
+  // Same threshold as MultiTermQueryConstantScoreWrapper
+  static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
+
+  private final boolean singleField; // whether all terms are from the same field
+  private final PrefixCodedTerms termData;
+  private final int termDataHashCode; // cached hashcode of termData
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given collection. It
+   * can contain duplicate terms and multiple fields.
+   */
+  public TermInSetQuery(Collection<Term> terms) {
+    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
+    // already sorted if we are a SortedSet with natural order
+    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
+    if (!sorted) {
+      ArrayUtil.timSort(sortedTerms);
+    }
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    Set<String> fields = new HashSet<>();
+    Term previous = null;
+    for (Term term : sortedTerms) {
+      if (term.equals(previous) == false) {
+        fields.add(term.field());
+        builder.add(term);
+      }
+      previous = term;
+    }
+    singleField = fields.size() == 1;
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given collection for
+   * a single field. It can contain duplicate terms.
+   */
+  public TermInSetQuery(String field, Collection<BytesRef> terms) {
+    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
+    // already sorted if we are a SortedSet with natural order
+    boolean sorted = terms instanceof SortedSet && ((SortedSet<BytesRef>)terms).comparator() == null;
+    if (!sorted) {
+      ArrayUtil.timSort(sortedTerms);
+    }
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    BytesRefBuilder previous = null;
+    for (BytesRef term : sortedTerms) {
+      if (previous == null) {
+        previous = new BytesRefBuilder();
+      } else if (previous.get().equals(term)) {
+        continue; // deduplicate
+      }
+      builder.add(field, term);
+      previous.copyBytes(term);
+    }
+    singleField = true;
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given {@link BytesRef} array for
+   * a single field.
+   */
+  public TermInSetQuery(String field, BytesRef...terms) {
+    this(field, Arrays.asList(terms));
+  }
+
+  /**
+   * Creates a new {@link TermInSetQuery} from the given array. The array can
+   * contain duplicate terms and multiple fields.
+   */
+  public TermInSetQuery(final Term... terms) {
+    this(Arrays.asList(terms));
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+    if (termData.size() <= threshold) {
+      BooleanQuery.Builder bq = new BooleanQuery.Builder();
+      TermIterator iterator = termData.iterator();
+      for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+        bq.add(new TermQuery(new Term(iterator.field(), BytesRef.deepCopyOf(term))), Occur.SHOULD);
+      }
+      return new ConstantScoreQuery(bq.build());
+    }
+    return super.rewrite(reader);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return sameClassAs(other) &&
+        equalsTo(getClass().cast(other));
+  }
+
+  private boolean equalsTo(TermInSetQuery other) {
+    // termData might be heavy to compare so check the hash code first
+    return termDataHashCode == other.termDataHashCode &&
+        termData.equals(other.termData);
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * classHash() + termDataHashCode;
+  }
+
+  /** Returns the terms wrapped in a PrefixCodedTerms. */
+  public PrefixCodedTerms getTermData() {
+    return termData;
+  }
+
+  @Override
+  public String toString(String defaultField) {
+    StringBuilder builder = new StringBuilder();
+    boolean first = true;
+    TermIterator iterator = termData.iterator();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if (!first) {
+        builder.append(' ');
+      }
+      first = false;
+      builder.append(new Term(iterator.field(), term).toString());
+    }
+
+    return builder.toString();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    return Collections.emptyList();
+  }
+
+  private static class TermAndState {
+    final String field;
+    final TermsEnum termsEnum;
+    final BytesRef term;
+    final TermState state;
+    final int docFreq;
+    final long totalTermFreq;
+
+    TermAndState(String field, TermsEnum termsEnum) throws IOException {
+      this.field = field;
+      this.termsEnum = termsEnum;
+      this.term = BytesRef.deepCopyOf(termsEnum.term());
+      this.state = termsEnum.termState();
+      this.docFreq = termsEnum.docFreq();
+      this.totalTermFreq = termsEnum.totalTermFreq();
+    }
+  }
+
+  private static class WeightOrDocIdSet {
+    final Weight weight;
+    final DocIdSet set;
+
+    WeightOrDocIdSet(Weight weight) {
+      this.weight = Objects.requireNonNull(weight);
+      this.set = null;
+    }
+
+    WeightOrDocIdSet(DocIdSet bitset) {
+      this.set = bitset;
+      this.weight = null;
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    return new ConstantScoreWeight(this, boost) {
+
+      @Override
+      public void extractTerms(Set<Term> terms) {
+        // no-op
+        // This query is for abuse cases when the number of terms is too high to
+        // run efficiently as a BooleanQuery. So likewise we hide its terms in
+        // order to protect highlighters
+      }
+
+      /**
+       * On the given leaf context, try to either rewrite to a disjunction if
+       * there are few matching terms, or build a bitset containing matching docs.
+       */
+      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
+        final LeafReader reader = context.reader();
+
+        // We will first try to collect up to 'threshold' terms into 'matchingTerms'
+        // if there are two many terms, we will fall back to building the 'builder'
+        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+        assert termData.size() > threshold : "Query should have been rewritten";
+        List<TermAndState> matchingTerms = new ArrayList<>(threshold);
+        DocIdSetBuilder builder = null;
+
+        final Fields fields = reader.fields();
+        String lastField = null;
+        Terms terms = null;
+        TermsEnum termsEnum = null;
+        PostingsEnum docs = null;
+        TermIterator iterator = termData.iterator();
+        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+          String field = iterator.field();
+          // comparing references is fine here
+          if (field != lastField) {
+            terms = fields.terms(field);
+            if (terms == null) {
+              termsEnum = null;
+            } else {
+              termsEnum = terms.iterator();
+            }
+            lastField = field;
+          }
+          if (termsEnum != null && termsEnum.seekExact(term)) {
+            if (matchingTerms == null) {
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+            } else if (matchingTerms.size() < threshold) {
+              matchingTerms.add(new TermAndState(field, termsEnum));
+            } else {
+              assert matchingTerms.size() == threshold;
+              if (singleField) {
+                // common case: all terms are in the same field
+                // use an optimized builder that leverages terms stats to be more efficient
+                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
+              } else {
+                // corner case: different fields
+                // don't make assumptions about the docs we will get
+                builder = new DocIdSetBuilder(reader.maxDoc());
+              }
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+              for (TermAndState t : matchingTerms) {
+                t.termsEnum.seekExact(t.term, t.state);
+                docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
+                builder.add(docs);
+              }
+              matchingTerms = null;
+            }
+          }
+        }
+        if (matchingTerms != null) {
+          assert builder == null;
+          BooleanQuery.Builder bq = new BooleanQuery.Builder();
+          for (TermAndState t : matchingTerms) {
+            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
+            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
+            bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD);
+          }
+          Query q = new ConstantScoreQuery(bq.build());
+          final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores, score());
+          return new WeightOrDocIdSet(weight);
+        } else {
+          assert builder != null;
+          return new WeightOrDocIdSet(builder.build());
+        }
+      }
+
+      private Scorer scorer(DocIdSet set) throws IOException {
+        if (set == null) {
+          return null;
+        }
+        final DocIdSetIterator disi = set.iterator();
+        if (disi == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+        if (weightOrBitSet.weight != null) {
+          return weightOrBitSet.weight.bulkScorer(context);
+        } else {
+          final Scorer scorer = scorer(weightOrBitSet.set);
+          if (scorer == null) {
+            return null;
+          }
+          return new DefaultBulkScorer(scorer);
+        }
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+        if (weightOrBitSet.weight != null) {
+          return weightOrBitSet.weight.scorer(context);
+        } else {
+          return scorer(weightOrBitSet.set);
+        }
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index ab68eeb..035947f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -50,9 +50,8 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
     // DocIdSet in the first place
     return query instanceof MultiTermQuery ||
         query instanceof MultiTermQueryConstantScoreWrapper ||
-        isPointQuery(query) ||
-        // can't refer to TermsQuery directly as it is in another module
-        "TermsQuery".equals(query.getClass().getSimpleName());
+        query instanceof TermInSetQuery ||
+        isPointQuery(query);
   }
 
   static boolean isCheap(Query query) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
new file mode 100644
index 0000000..e694d97
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -0,0 +1,328 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FilterDirectoryReader;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageTester;
+import org.apache.lucene.util.TestUtil;
+
+public class TermInSetQueryTest extends LuceneTestCase {
+
+  public void testDuel() throws IOException {
+    final int iters = atLeast(2);
+    for (int iter = 0; iter < iters; ++iter) {
+      final List<Term> allTerms = new ArrayList<>();
+      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
+      for (int i = 0; i < numTerms; ++i) {
+        final String field = usually() ? "f" : "g";
+        final String value = TestUtil.randomAnalysisString(random(), 10, true);
+        allTerms.add(new Term(field, value));
+      }
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final Term term = allTerms.get(random().nextInt(allTerms.size()));
+        doc.add(new StringField(term.field(), term.text(), Store.NO));
+        iw.addDocument(doc);
+      }
+      if (numTerms > 1 && random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      if (reader.numDocs() == 0) {
+        // may occasionally happen if all documents got the same term
+        IOUtils.close(reader, dir);
+        continue;
+      }
+
+      for (int i = 0; i < 100; ++i) {
+        final float boost = random().nextFloat() * 10;
+        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
+        List<Term> queryTerms = new ArrayList<>();
+        for (int j = 0; j < numQueryTerms; ++j) {
+          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
+        }
+        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        for (Term t : queryTerms) {
+          bq.add(new TermQuery(t), Occur.SHOULD);
+        }
+        final Query q1 = new ConstantScoreQuery(bq.build());
+        final Query q2 = new TermInSetQuery(queryTerms);
+        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+
+  private TermInSetQuery termsQuery(boolean singleField, Term...terms) {
+    return termsQuery(singleField, Arrays.asList(terms));
+  }
+
+  private TermInSetQuery termsQuery(boolean singleField, Collection<Term> termList) {
+    if (!singleField) {
+      return new TermInSetQuery(new ArrayList<>(termList));
+    }
+    final TermInSetQuery filter;
+    List<BytesRef> bytes = new ArrayList<>();
+    String field = null;
+    for (Term term : termList) {
+        bytes.add(term.bytes());
+        if (field != null) {
+          assertEquals(term.field(), field);
+        }
+        field = term.field();
+    }
+    assertNotNull(field);
+    filter = new TermInSetQuery(field, bytes);
+    return filter;
+  }
+
+  public void testHashCodeAndEquals() {
+    int num = atLeast(100);
+    final boolean singleField = random().nextBoolean();
+    List<Term> terms = new ArrayList<>();
+    Set<Term> uniqueTerms = new HashSet<>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + (singleField ? "1" : random().nextInt(100));
+      String string = TestUtil.randomRealisticUnicodeString(random());
+      terms.add(new Term(field, string));
+      uniqueTerms.add(new Term(field, string));
+      TermInSetQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
+      Collections.shuffle(terms, random());
+      TermInSetQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
+      assertEquals(right, left);
+      assertEquals(right.hashCode(), left.hashCode());
+      if (uniqueTerms.size() > 1) {
+        List<Term> asList = new ArrayList<>(uniqueTerms);
+        asList.remove(0);
+        TermInSetQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
+        assertFalse(left.equals(notEqual));
+        assertFalse(right.equals(notEqual));
+      }
+    }
+
+    TermInSetQuery tq1 = new TermInSetQuery(new Term("thing", "apple"));
+    TermInSetQuery tq2 = new TermInSetQuery(new Term("thing", "orange"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+
+    // different fields with the same term should have differing hashcodes
+    tq1 = new TermInSetQuery(new Term("thing1", "apple"));
+    tq2 = new TermInSetQuery(new Term("thing2", "apple"));
+    assertFalse(tq1.hashCode() == tq2.hashCode());
+  }
+
+  public void testSingleFieldEquals() {
+    // Two terms with the same hash code
+    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
+    TermInSetQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
+    TermInSetQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
+    assertFalse(left.equals(right));
+  }
+
+  public void testToString() {
+    TermInSetQuery termsQuery = new TermInSetQuery(new Term("field1", "a"),
+                                              new Term("field1", "b"),
+                                              new Term("field1", "c"));
+    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
+  }
+
+  public void testDedup() {
+    Query query1 = new TermInSetQuery(new Term("foo", "bar"));
+    Query query2 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testOrderDoesNotMatter() {
+    // order of terms if different
+    Query query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    Query query2 = new TermInSetQuery(new Term("foo", "baz"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+
+    // order of fields is different
+    query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("bar", "bar"));
+    query2 = new TermInSetQuery(new Term("bar", "bar"), new Term("foo", "bar"));
+    QueryUtils.checkEqual(query1, query2);
+  }
+
+  public void testRamBytesUsed() {
+    List<Term> terms = new ArrayList<>();
+    final int numTerms = 1000 + random().nextInt(1000);
+    for (int i = 0; i < numTerms; ++i) {
+      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
+    }
+    TermInSetQuery query = new TermInSetQuery(terms);
+    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
+    final long expectedRamBytesUsed = query.ramBytesUsed();
+    // error margin within 5%
+    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
+  }
+
+  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
+
+    private final AtomicInteger counter;
+    
+    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
+      super(in, new TermsCountingSubReaderWrapper(counter));
+      this.counter = counter;
+    }
+
+    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
+      private final AtomicInteger counter;
+
+      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
+        this.counter = counter;
+      }
+
+      @Override
+      public LeafReader wrap(LeafReader reader) {
+        return new TermsCountingLeafReaderWrapper(reader, counter);
+      }
+    }
+
+    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
+
+      private final AtomicInteger counter;
+
+      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
+        super(in);
+        this.counter = counter;
+      }
+
+      @Override
+      public Fields fields() throws IOException {
+        return new FilterFields(in.fields()) {
+          @Override
+          public Terms terms(String field) throws IOException {
+            final Terms in = this.in.terms(field);
+            if (in == null) {
+              return null;
+            }
+            return new FilterTerms(in) {
+              @Override
+              public TermsEnum iterator() throws IOException {
+                counter.incrementAndGet();
+                return super.iterator();
+              }
+            };
+          }
+        };
+      }
+      
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
+      return new TermsCountingDirectoryReaderWrapper(in, counter);
+    }
+
+  }
+
+  public void testPullOneTermsEnumPerField() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new StringField("foo", "1", Store.NO));
+    doc.add(new StringField("bar", "2", Store.NO));
+    doc.add(new StringField("baz", "3", Store.NO));
+    w.addDocument(doc);
+    DirectoryReader reader = w.getReader();
+    w.close();
+    final AtomicInteger counter = new AtomicInteger();
+    DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
+
+    final List<Term> terms = new ArrayList<>();
+    final Set<String> fields = new HashSet<>();
+    // enough terms to avoid the rewrite
+    final int numTerms = TestUtil.nextInt(random(), TermInSetQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
+    for (int i = 0; i < numTerms; ++i) {
+      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
+      final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
+      fields.add(field);
+      terms.add(new Term(field, term));
+    }
+
+    new IndexSearcher(wrapped).count(new TermInSetQuery(terms));
+    assertEquals(fields.size(), counter.get());
+    wrapped.close();
+    dir.close();
+  }
+  
+  public void testBinaryToString() {
+    TermInSetQuery query = new TermInSetQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
+    assertEquals("field:[ff fe]", query.toString());
+  }
+
+  public void testIsConsideredCostlyByQueryCache() throws IOException {
+    TermInSetQuery query = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    assertFalse(policy.shouldCache(query));
+    policy.onUse(query);
+    policy.onUse(query);
+    // cached after two uses
+    assertTrue(policy.shouldCache(query));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
index dd212c6..a010709 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
@@ -16,13 +16,13 @@
  */
 package org.apache.lucene.facet;
 
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
-import org.apache.lucene.search.Query;
-
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
+
 /**
  * A multi-terms {@link Query} over a {@link FacetField}.
  * <p>
@@ -30,9 +30,9 @@ import java.util.Collection;
  * especially in cases where you don't intend to use {@link DrillSideways}
  *
  * @lucene.experimental
- * @see org.apache.lucene.queries.TermsQuery
+ * @see org.apache.lucene.search.TermInSetQuery
  */
-public class MultiFacetQuery extends TermsQuery {
+public class MultiFacetQuery extends TermInSetQuery {
 
   /**
    * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
index 7b7f094..5effa83 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
@@ -16,130 +16,33 @@
  */
 package org.apache.lucene.queries;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.SortedSet;
 
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.PrefixCodedTerms;
-import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-import org.apache.lucene.index.TermState;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BulkScorer;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.RamUsageEstimator;
 
 /**
- * Specialization for a disjunction over many terms that behaves like a
- * {@link ConstantScoreQuery} over a {@link BooleanQuery} containing only
- * {@link org.apache.lucene.search.BooleanClause.Occur#SHOULD} clauses.
- * <p>For instance in the following example, both @{code q1} and {@code q2}
- * would yield the same scores:
- * <pre class="prettyprint">
- * Query q1 = new TermsQuery(new Term("field", "foo"), new Term("field", "bar"));
- *
- * BooleanQuery bq = new BooleanQuery();
- * bq.add(new TermQuery(new Term("field", "foo")), Occur.SHOULD);
- * bq.add(new TermQuery(new Term("field", "bar")), Occur.SHOULD);
- * Query q2 = new ConstantScoreQuery(bq);
- * </pre>
- * <p>When there are few terms, this query executes like a regular disjunction.
- * However, when there are many terms, instead of merging iterators on the fly,
- * it will populate a bit set with matching docs and return a {@link Scorer}
- * over this bit set.
- * <p>NOTE: This query produces scores that are equal to its boost
+ * @deprecated Use {@link org.apache.lucene.search.TermInSetQuery}
  */
-public class TermsQuery extends Query implements Accountable {
-
-  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(TermsQuery.class);
-  // Same threshold as MultiTermQueryConstantScoreWrapper
-  static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
-
-  private final boolean singleField; // whether all terms are from the same field
-  private final PrefixCodedTerms termData;
-  private final int termDataHashCode; // cached hashcode of termData
+@Deprecated
+public class TermsQuery extends TermInSetQuery {
 
   /**
    * Creates a new {@link TermsQuery} from the given collection. It
    * can contain duplicate terms and multiple fields.
    */
   public TermsQuery(Collection<Term> terms) {
-    Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    Set<String> fields = new HashSet<>();
-    Term previous = null;
-    for (Term term : sortedTerms) {
-      if (term.equals(previous) == false) {
-        fields.add(term.field());
-        builder.add(term);
-      }
-      previous = term;
-    }
-    singleField = fields.size() == 1;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
+    super(terms);
   }
-  
+
   /**
    * Creates a new {@link TermsQuery} from the given collection for
    * a single field. It can contain duplicate terms.
    */
   public TermsQuery(String field, Collection<BytesRef> terms) {
-    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
-    // already sorted if we are a SortedSet with natural order
-    boolean sorted = terms instanceof SortedSet && ((SortedSet<BytesRef>)terms).comparator() == null;
-    if (!sorted) {
-      ArrayUtil.timSort(sortedTerms);
-    }
-    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
-    BytesRefBuilder previous = null;
-    for (BytesRef term : sortedTerms) {
-      if (previous == null) {
-        previous = new BytesRefBuilder();
-      } else if (previous.get().equals(term)) {
-        continue; // deduplicate
-      }
-      builder.add(field, term);
-      previous.copyBytes(term);
-    }
-    singleField = true;
-    termData = builder.finish();
-    termDataHashCode = termData.hashCode();
+    super(field, terms);
   }
 
   /**
@@ -147,7 +50,7 @@ public class TermsQuery extends Query implements Accountable {
    * a single field.
    */
   public TermsQuery(String field, BytesRef...terms) {
-   this(field, Arrays.asList(terms));
+    this(field, Arrays.asList(terms));
   }
 
   /**
@@ -158,224 +61,5 @@ public class TermsQuery extends Query implements Accountable {
     this(Arrays.asList(terms));
   }
 
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
-    if (termData.size() <= threshold) {
-      BooleanQuery.Builder bq = new BooleanQuery.Builder();
-      TermIterator iterator = termData.iterator();
-      for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-        bq.add(new TermQuery(new Term(iterator.field(), BytesRef.deepCopyOf(term))), Occur.SHOULD);
-      }
-      return new ConstantScoreQuery(bq.build());
-    }
-    return super.rewrite(reader);
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    return sameClassAs(other) &&
-           equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(TermsQuery other) {
-    // termData might be heavy to compare so check the hash code first
-    return termDataHashCode == other.termDataHashCode && 
-           termData.equals(other.termData);
-  }
-
-  @Override
-  public int hashCode() {
-    return 31 * classHash() + termDataHashCode;
-  }
-
-  /** Returns the terms wrapped in a PrefixCodedTerms. */
-  public PrefixCodedTerms getTermData() {
-    return termData;
-  }
-
-  @Override
-  public String toString(String defaultField) {
-    StringBuilder builder = new StringBuilder();
-    boolean first = true;
-    TermIterator iterator = termData.iterator();
-    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-      if (!first) {
-        builder.append(' ');
-      }
-      first = false;
-      builder.append(new Term(iterator.field(), term).toString());
-    }
-
-    return builder.toString();
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
-  }
 
-  @Override
-  public Collection<Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
-
-  private static class TermAndState {
-    final String field;
-    final TermsEnum termsEnum;
-    final BytesRef term;
-    final TermState state;
-    final int docFreq;
-    final long totalTermFreq;
-
-    TermAndState(String field, TermsEnum termsEnum) throws IOException {
-      this.field = field;
-      this.termsEnum = termsEnum;
-      this.term = BytesRef.deepCopyOf(termsEnum.term());
-      this.state = termsEnum.termState();
-      this.docFreq = termsEnum.docFreq();
-      this.totalTermFreq = termsEnum.totalTermFreq();
-    }
-  }
-
-  private static class WeightOrDocIdSet {
-    final Weight weight;
-    final DocIdSet set;
-
-    WeightOrDocIdSet(Weight weight) {
-      this.weight = Objects.requireNonNull(weight);
-      this.set = null;
-    }
-
-    WeightOrDocIdSet(DocIdSet bitset) {
-      this.set = bitset;
-      this.weight = null;
-    }
-  }
-
-  @Override
-  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new ConstantScoreWeight(this, boost) {
-
-      @Override
-      public void extractTerms(Set<Term> terms) {
-        // no-op
-        // This query is for abuse cases when the number of terms is too high to
-        // run efficiently as a BooleanQuery. So likewise we hide its terms in
-        // order to protect highlighters
-      }
-
-      /**
-       * On the given leaf context, try to either rewrite to a disjunction if
-       * there are few matching terms, or build a bitset containing matching docs.
-       */
-      private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
-        final LeafReader reader = context.reader();
-
-        // We will first try to collect up to 'threshold' terms into 'matchingTerms'
-        // if there are two many terms, we will fall back to building the 'builder'
-        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
-        assert termData.size() > threshold : "Query should have been rewritten";
-        List<TermAndState> matchingTerms = new ArrayList<>(threshold);
-        DocIdSetBuilder builder = null;
-
-        final Fields fields = reader.fields();
-        String lastField = null;
-        Terms terms = null;
-        TermsEnum termsEnum = null;
-        PostingsEnum docs = null;
-        TermIterator iterator = termData.iterator();
-        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-          String field = iterator.field();
-          // comparing references is fine here
-          if (field != lastField) {
-            terms = fields.terms(field);
-            if (terms == null) {
-              termsEnum = null;
-            } else {
-              termsEnum = terms.iterator();
-            }
-            lastField = field;
-          }
-          if (termsEnum != null && termsEnum.seekExact(term)) {
-            if (matchingTerms == null) {
-              docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.add(docs);
-            } else if (matchingTerms.size() < threshold) {
-              matchingTerms.add(new TermAndState(field, termsEnum));
-            } else {
-              assert matchingTerms.size() == threshold;
-              if (singleField) {
-                // common case: all terms are in the same field
-                // use an optimized builder that leverages terms stats to be more efficient
-                builder = new DocIdSetBuilder(reader.maxDoc(), terms);
-              } else {
-                // corner case: different fields
-                // don't make assumptions about the docs we will get
-                builder = new DocIdSetBuilder(reader.maxDoc());
-              }
-              docs = termsEnum.postings(docs, PostingsEnum.NONE);
-              builder.add(docs);
-              for (TermAndState t : matchingTerms) {
-                t.termsEnum.seekExact(t.term, t.state);
-                docs = t.termsEnum.postings(docs, PostingsEnum.NONE);
-                builder.add(docs);
-              }
-              matchingTerms = null;
-            }
-          }
-        }
-        if (matchingTerms != null) {
-          assert builder == null;
-          BooleanQuery.Builder bq = new BooleanQuery.Builder();
-          for (TermAndState t : matchingTerms) {
-            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
-            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
-            bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD);
-          }
-          Query q = new ConstantScoreQuery(bq.build());
-          final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores, score());
-          return new WeightOrDocIdSet(weight);
-        } else {
-          assert builder != null;
-          return new WeightOrDocIdSet(builder.build());
-        }
-      }
-
-      private Scorer scorer(DocIdSet set) throws IOException {
-        if (set == null) {
-          return null;
-        }
-        final DocIdSetIterator disi = set.iterator();
-        if (disi == null) {
-          return null;
-        }
-        return new ConstantScoreScorer(this, score(), disi);
-      }
-
-      @Override
-      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
-          return weightOrBitSet.weight.bulkScorer(context);
-        } else {
-          final Scorer scorer = scorer(weightOrBitSet.set);
-          if (scorer == null) {
-            return null;
-          }
-          return new DefaultBulkScorer(scorer);
-        }
-      }
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        final WeightOrDocIdSet weightOrBitSet = rewrite(context);
-        if (weightOrBitSet.weight != null) {
-          return weightOrBitSet.weight.scorer(context);
-        } else {
-          return scorer(weightOrBitSet.set);
-        }
-      }
-    };
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
deleted file mode 100644
index f8b10ef..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * 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.lucene.queries;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FilterDirectoryReader;
-import org.apache.lucene.index.FilterLeafReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryUtils;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.RamUsageTester;
-import org.apache.lucene.util.TestUtil;
-
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-import com.carrotsearch.randomizedtesting.generators.RandomStrings;
-
-public class TermsQueryTest extends LuceneTestCase {
-
-  public void testDuel() throws IOException {
-    final int iters = atLeast(2);
-    for (int iter = 0; iter < iters; ++iter) {
-      final List<Term> allTerms = new ArrayList<>();
-      final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
-      for (int i = 0; i < numTerms; ++i) {
-        final String field = usually() ? "f" : "g";
-        final String value = TestUtil.randomAnalysisString(random(), 10, true);
-        allTerms.add(new Term(field, value));
-      }
-      Directory dir = newDirectory();
-      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
-      final int numDocs = atLeast(100);
-      for (int i = 0; i < numDocs; ++i) {
-        Document doc = new Document();
-        final Term term = allTerms.get(random().nextInt(allTerms.size()));
-        doc.add(new StringField(term.field(), term.text(), Store.NO));
-        iw.addDocument(doc);
-      }
-      if (numTerms > 1 && random().nextBoolean()) {
-        iw.deleteDocuments(new TermQuery(allTerms.get(0)));
-      }
-      iw.commit();
-      final IndexReader reader = iw.getReader();
-      final IndexSearcher searcher = newSearcher(reader);
-      iw.close();
-
-      if (reader.numDocs() == 0) {
-        // may occasionally happen if all documents got the same term
-        IOUtils.close(reader, dir);
-        continue;
-      }
-
-      for (int i = 0; i < 100; ++i) {
-        final float boost = random().nextFloat() * 10;
-        final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
-        List<Term> queryTerms = new ArrayList<>();
-        for (int j = 0; j < numQueryTerms; ++j) {
-          queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
-        }
-        final BooleanQuery.Builder bq = new BooleanQuery.Builder();
-        for (Term t : queryTerms) {
-          bq.add(new TermQuery(t), Occur.SHOULD);
-        }
-        final Query q1 = new ConstantScoreQuery(bq.build());
-        final Query q2 = new TermsQuery(queryTerms);
-        assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
-      }
-
-      reader.close();
-      dir.close();
-    }
-  }
-
-  private void assertSameMatches(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
-    final int maxDoc = searcher.getIndexReader().maxDoc();
-    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
-    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
-    assertEquals(td1.totalHits, td2.totalHits);
-    for (int i = 0; i < td1.scoreDocs.length; ++i) {
-      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
-      if (scores) {
-        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
-      }
-    }
-  }
-
-  private TermsQuery termsQuery(boolean singleField, Term...terms) {
-    return termsQuery(singleField, Arrays.asList(terms));
-  }
-
-  private TermsQuery termsQuery(boolean singleField, Collection<Term> termList) {
-    if (!singleField) {
-      return new TermsQuery(new ArrayList<>(termList));
-    }
-    final TermsQuery filter;
-    List<BytesRef> bytes = new ArrayList<>();
-    String field = null;
-    for (Term term : termList) {
-        bytes.add(term.bytes());
-        if (field != null) {
-          assertEquals(term.field(), field);
-        }
-        field = term.field();
-    }
-    assertNotNull(field);
-    filter = new TermsQuery(field, bytes);
-    return filter;
-  }
-
-  public void testHashCodeAndEquals() {
-    int num = atLeast(100);
-    final boolean singleField = random().nextBoolean();
-    List<Term> terms = new ArrayList<>();
-    Set<Term> uniqueTerms = new HashSet<>();
-    for (int i = 0; i < num; i++) {
-      String field = "field" + (singleField ? "1" : random().nextInt(100));
-      String string = TestUtil.randomRealisticUnicodeString(random());
-      terms.add(new Term(field, string));
-      uniqueTerms.add(new Term(field, string));
-      TermsQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
-      Collections.shuffle(terms, random());
-      TermsQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
-      assertEquals(right, left);
-      assertEquals(right.hashCode(), left.hashCode());
-      if (uniqueTerms.size() > 1) {
-        List<Term> asList = new ArrayList<>(uniqueTerms);
-        asList.remove(0);
-        TermsQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
-        assertFalse(left.equals(notEqual));
-        assertFalse(right.equals(notEqual));
-      }
-    }
-
-    TermsQuery tq1 = new TermsQuery(new Term("thing", "apple"));
-    TermsQuery tq2 = new TermsQuery(new Term("thing", "orange"));
-    assertFalse(tq1.hashCode() == tq2.hashCode());
-
-    // different fields with the same term should have differing hashcodes
-    tq1 = new TermsQuery(new Term("thing1", "apple"));
-    tq2 = new TermsQuery(new Term("thing2", "apple"));
-    assertFalse(tq1.hashCode() == tq2.hashCode());
-  }
-
-  public void testSingleFieldEquals() {
-    // Two terms with the same hash code
-    assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
-    TermsQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
-    TermsQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
-    assertFalse(left.equals(right));
-  }
-
-  public void testToString() {
-    TermsQuery termsQuery = new TermsQuery(new Term("field1", "a"),
-                                              new Term("field1", "b"),
-                                              new Term("field1", "c"));
-    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
-  }
-
-  public void testDedup() {
-    Query query1 = new TermsQuery(new Term("foo", "bar"));
-    Query query2 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-  }
-
-  public void testOrderDoesNotMatter() {
-    // order of terms if different
-    Query query1 = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    Query query2 = new TermsQuery(new Term("foo", "baz"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-
-    // order of fields is different
-    query1 = new TermsQuery(new Term("foo", "bar"), new Term("bar", "bar"));
-    query2 = new TermsQuery(new Term("bar", "bar"), new Term("foo", "bar"));
-    QueryUtils.checkEqual(query1, query2);
-  }
-
-  public void testRamBytesUsed() {
-    List<Term> terms = new ArrayList<>();
-    final int numTerms = 1000 + random().nextInt(1000);
-    for (int i = 0; i < numTerms; ++i) {
-      terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
-    }
-    TermsQuery query = new TermsQuery(terms);
-    final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
-    final long expectedRamBytesUsed = query.ramBytesUsed();
-    // error margin within 5%
-    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
-  }
-
-  private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
-
-    private final AtomicInteger counter;
-    
-    public TermsCountingDirectoryReaderWrapper(DirectoryReader in, AtomicInteger counter) throws IOException {
-      super(in, new TermsCountingSubReaderWrapper(counter));
-      this.counter = counter;
-    }
-
-    private static class TermsCountingSubReaderWrapper extends SubReaderWrapper {
-      private final AtomicInteger counter;
-
-      public TermsCountingSubReaderWrapper(AtomicInteger counter) {
-        this.counter = counter;
-      }
-
-      @Override
-      public LeafReader wrap(LeafReader reader) {
-        return new TermsCountingLeafReaderWrapper(reader, counter);
-      }
-    }
-
-    private static class TermsCountingLeafReaderWrapper extends FilterLeafReader {
-
-      private final AtomicInteger counter;
-
-      public TermsCountingLeafReaderWrapper(LeafReader in, AtomicInteger counter) {
-        super(in);
-        this.counter = counter;
-      }
-
-      @Override
-      public Fields fields() throws IOException {
-        return new FilterFields(in.fields()) {
-          @Override
-          public Terms terms(String field) throws IOException {
-            final Terms in = this.in.terms(field);
-            if (in == null) {
-              return null;
-            }
-            return new FilterTerms(in) {
-              @Override
-              public TermsEnum iterator() throws IOException {
-                counter.incrementAndGet();
-                return super.iterator();
-              }
-            };
-          }
-        };
-      }
-      
-    }
-
-    @Override
-    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
-      return new TermsCountingDirectoryReaderWrapper(in, counter);
-    }
-
-  }
-
-  public void testPullOneTermsEnumPerField() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
-    Document doc = new Document();
-    doc.add(new StringField("foo", "1", Store.NO));
-    doc.add(new StringField("bar", "2", Store.NO));
-    doc.add(new StringField("baz", "3", Store.NO));
-    w.addDocument(doc);
-    DirectoryReader reader = w.getReader();
-    w.close();
-    final AtomicInteger counter = new AtomicInteger();
-    DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
-
-    final List<Term> terms = new ArrayList<>();
-    final Set<String> fields = new HashSet<>();
-    // enough terms to avoid the rewrite
-    final int numTerms = TestUtil.nextInt(random(), TermsQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
-    for (int i = 0; i < numTerms; ++i) {
-      final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
-      final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
-      fields.add(field);
-      terms.add(new Term(field, term));
-    }
-
-    new IndexSearcher(wrapped).count(new TermsQuery(terms));
-    assertEquals(fields.size(), counter.get());
-    wrapped.close();
-    dir.close();
-  }
-  
-  public void testBinaryToString() {
-    TermsQuery query = new TermsQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
-    assertEquals("field:[ff fe]", query.toString());
-  }
-
-  public void testIsConsideredCostlyByQueryCache() throws IOException {
-    TermsQuery query = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
-    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
-    assertFalse(policy.shouldCache(query));
-    policy.onUse(query);
-    policy.onUse(query);
-    // cached after two uses
-    assertTrue(policy.shouldCache(query));
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
index 0273466..4e37f5d 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
@@ -19,10 +19,8 @@ package org.apache.lucene.spatial.prefix;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
@@ -31,10 +29,12 @@ import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * A basic implementation of {@link PrefixTreeStrategy} using a large
- * {@link TermsQuery} of all the cells from
+ * {@link TermInSetQuery} of all the cells from
  * {@link SpatialPrefixTree#getTreeCellIterator(org.locationtech.spatial4j.shape.Shape, int)}.
  * It only supports the search of indexed Point shapes.
  * <p>
@@ -105,7 +105,7 @@ public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
     }
     //unfortunately TermsQuery will needlessly sort & dedupe
     //TODO an automatonQuery might be faster?
-    return new TermsQuery(getFieldName(), terms);
+    return new TermInSetQuery(getFieldName(), terms);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
index 514c18e..bb26a2e 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
@@ -23,12 +23,11 @@ import java.util.Collections;
 import java.util.List;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.apache.lucene.spatial.prefix.NumberRangePrefixTreeStrategy.Facets;
 import org.apache.lucene.spatial.prefix.tree.Cell;
@@ -40,6 +39,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
 import org.junit.Before;
 import org.junit.Test;
+import org.locationtech.spatial4j.shape.Shape;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
@@ -132,7 +132,7 @@ public class NumberRangeFacetsTest extends StrategyTestCase {
             terms.add(new Term("id", acceptDocId.toString()));
           }
 
-          topAcceptDocs = searchForDocBits(new TermsQuery(terms));
+          topAcceptDocs = searchForDocBits(new TermInSetQuery(terms));
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index 366c4a9..a5cbee2 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -26,6 +26,15 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import com.carrotsearch.hppc.IntHashSet;
+import com.carrotsearch.hppc.IntObjectHashMap;
+import com.carrotsearch.hppc.LongHashSet;
+import com.carrotsearch.hppc.LongObjectHashMap;
+import com.carrotsearch.hppc.LongObjectMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.carrotsearch.hppc.cursors.LongObjectCursor;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
@@ -36,7 +45,6 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
@@ -46,6 +54,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
@@ -83,16 +92,6 @@ import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
-import com.carrotsearch.hppc.IntHashSet;
-import com.carrotsearch.hppc.IntObjectHashMap;
-import com.carrotsearch.hppc.LongHashSet;
-import com.carrotsearch.hppc.LongObjectHashMap;
-import com.carrotsearch.hppc.LongObjectMap;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.LongCursor;
-import com.carrotsearch.hppc.cursors.LongObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-
 /**
  * The ExpandComponent is designed to work with the CollapsingPostFilter.
  * The CollapsingPostFilter collapses a result set on a field.
@@ -720,7 +719,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       bytesRefs[++index] = term.toBytesRef();
     }
 
-    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermsQuery(fname, bytesRefs)));
+    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs)));
   }
 
   private Query getGroupQuery(String fname,
@@ -733,7 +732,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       IntObjectCursor<BytesRef> cursor = it.next();
       bytesRefs[++index] = cursor.value;
     }
-    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermsQuery(fname, bytesRefs)));
+    return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs)));
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index b67f88d..a5c898a 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -39,7 +39,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.legacy.LegacyNumericType;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -51,6 +50,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
 import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.BytesRef;
@@ -765,7 +765,7 @@ public abstract class FieldType extends FieldProperties {
       readableToIndexed(externalVal, br);
       lst.add( br.toBytesRef() );
     }
-    return new TermsQuery(field.getName() , lst);
+    return new TermInSetQuery(field.getName() , lst);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
index d53dcbf..3a60149 100644
--- a/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/TermsQParserPlugin.java
@@ -20,13 +20,13 @@ import java.util.Arrays;
 import java.util.regex.Pattern;
 
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DocValuesTermsQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -60,7 +60,7 @@ public class TermsQParserPlugin extends QParserPlugin {
     termsFilter {
       @Override
       Filter makeFilter(String fname, BytesRef[] bytesRefs) {
-        return new QueryWrapperFilter(new TermsQuery(fname, bytesRefs));
+        return new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs));
       }
     },
     booleanQuery {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
index 8cfcf79..3f762e3 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
@@ -25,7 +25,6 @@ import java.util.TreeSet;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
@@ -35,6 +34,7 @@ import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.util.BytesRef;
@@ -281,7 +281,7 @@ public class GraphQuery extends Query {
             collectorTerms.get(i, ref);
             termList.add(ref);
           }
-          q = new TermsQuery(fromField, termList);
+          q = new TermInSetQuery(fromField, termList);
         }
         
         // If there is a filter to be used while crawling the graph, add that.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22940f5c/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
index d3e6a7f..76b441b 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
@@ -19,12 +19,12 @@ package org.apache.solr.search;
 import java.util.Locale;
 import java.util.Random;
 
-import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.core.SolrInfoMBean;
@@ -224,13 +224,13 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req);
     qParser.setIsFilter(true); // this may change in the future
     q = qParser.getQuery();
-    assertEquals(26, ((TermsQuery)q).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)q).getTermData().size());
 
     // large numeric filter query should use TermsQuery (for trie fields)
     qParser = QParser.getParser("foo_i:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req);
     qParser.setIsFilter(true); // this may change in the future
     q = qParser.getQuery();
-    assertEquals(20, ((TermsQuery)q).getTermData().size());
+    assertEquals(20, ((TermInSetQuery)q).getTermData().size());
 
     // a filter() clause inside a relevancy query should be able to use a TermsQuery
     qParser = QParser.getParser("foo_s:aaa filter(foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z))", req);
@@ -245,7 +245,7 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
       qq = ((FilterQuery)qq).getQuery();
     }
 
-    assertEquals(26, ((TermsQuery)qq).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
 
     // test mixed boolean query, including quotes (which shouldn't matter)
     qParser = QParser.getParser("foo_s:(a +aaa b -bbb c d e f bar_s:(qqq www) g h i j k l m n o p q r s t u v w x y z)", req);
@@ -255,9 +255,9 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     qq = null;
     for (BooleanClause clause : ((BooleanQuery)q).clauses()) {
       qq = clause.getQuery();
-      if (qq instanceof TermsQuery) break;
+      if (qq instanceof TermInSetQuery) break;
     }
-    assertEquals(26, ((TermsQuery)qq).getTermData().size());
+    assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
 
     req.close();
   }


[03/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7610: Remove deprecated facet ValueSource methods

Posted by ho...@apache.org.
LUCENE-7610: Remove deprecated facet ValueSource methods


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

Branch: refs/heads/jira/solr-5944
Commit: ce8b678ba19a53c43033a235bdca54e5a68adcc8
Parents: e5f39f6
Author: Alan Woodward <ro...@apache.org>
Authored: Sat Jan 7 12:25:15 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sat Jan 7 13:07:13 2017 +0000

----------------------------------------------------------------------
 .../apache/lucene/facet/range/DoubleRange.java  |  9 ---
 .../facet/range/DoubleRangeFacetCounts.java     | 23 ------
 .../apache/lucene/facet/range/LongRange.java    | 10 ---
 .../facet/range/LongRangeFacetCounts.java       | 21 ------
 .../taxonomy/TaxonomyFacetSumValueSource.java   | 75 +-------------------
 .../facet/range/TestRangeFacetCounts.java       |  8 +--
 .../TestTaxonomyFacetSumValueSource.java        | 49 +++----------
 7 files changed, 13 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index ce377f5..50e771d 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -21,7 +21,6 @@ import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -178,14 +177,6 @@ public final class DoubleRange extends Range {
   }
 
   /**
-   * @deprecated Use {@link #getQuery(Query, DoubleValuesSource)}
-   */
-  @Deprecated
-  public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
-    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asDoubleValuesSource());
-  }
-
-  /**
    * Create a Query that matches documents in this range
    *
    * The query will check all documents that match the provided match query,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index 63fc935..5ed11a9 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -25,7 +25,6 @@ import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
@@ -61,15 +60,6 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
   }
 
   /**
-   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
-   *
-   * @deprecated Use {@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)}
-   * */
-  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
-    this(field, valueSource, hits, null, ranges);
-  }
-
-  /**
    * Create {@code RangeFacetCounts} using the provided {@link DoubleValuesSource}
    */
   public DoubleRangeFacetCounts(String field, DoubleValuesSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
@@ -78,19 +68,6 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
 
   /**
    * Create {@code RangeFacetCounts}, using the provided
-   * {@link ValueSource}, and using the provided Query as
-   * a fastmatch: only documents matching the query are
-   * checked for the matching ranges.
-   *
-   * @deprecated Use ({@link #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, Query, DoubleRange...)}
-   */
-  @Deprecated
-  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
-    this(field, valueSource.asDoubleValuesSource(), hits, fastMatchQuery, ranges);
-  }
-
-  /**
-   * Create {@code RangeFacetCounts}, using the provided
    * {@link DoubleValuesSource}, and using the provided Query as
    * a fastmatch: only documents matching the query are
    * checked for the matching ranges.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 20c408d..582d76f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -21,7 +21,6 @@ import java.util.Objects;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -169,15 +168,6 @@ public final class LongRange extends Range {
 
   }
 
-
-  /**
-   * @deprecated Use {@link #getQuery(Query, LongValuesSource)}
-   */
-  @Deprecated
-  public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
-    return new ValueSourceQuery(this, fastMatchQuery, valueSource.asLongValuesSource());
-  }
-
   /**
    * Create a Query that matches documents in this range
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
index a3cfc71..c9c42a3 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
@@ -51,33 +51,12 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
     this(field, LongValuesSource.fromLongField(field), hits, ranges);
   }
 
-  /**
-   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
-   *
-   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, LongRange...)}
-   */
-  @Deprecated
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
-    this(field, valueSource.asLongValuesSource(), hits, null, ranges);
-  }
-
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}. */
   public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
-  /**
-   * Create {@code RangeFacetCounts}, using the provided {@link ValueSource}.
-   *
-   * @deprecated Use {@link #LongRangeFacetCounts(String, LongValuesSource, FacetsCollector, Query, LongRange...)}
-   */
-  @Deprecated
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
-    this(field, valueSource.asLongValuesSource(), hits, fastMatchQuery, ranges);
-  }
-
-
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}, and using the provided Filter as
    *  a fastmatch: only documents passing the filter are

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 0a73ae5..6bc77c2 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -18,19 +18,14 @@ package org.apache.lucene.facet.taxonomy;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
-import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.IntsRef;
 
 /** Aggregates sum of values from {@link
@@ -42,21 +37,6 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
 
   /**
    * Aggreggates double facet values from the provided
-   *  {@link ValueSource}, pulling ordinals using {@link
-   *  DocValuesOrdinalsReader} against the default indexed
-   *  facet field {@link
-   *  FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
-   *
-   *  @deprecated {@link #TaxonomyFacetSumValueSource(TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
-   */
-  @Deprecated
-  public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
-                                     FacetsCollector fc, ValueSource valueSource) throws IOException {
-    this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
-  }
-
-  /**
-   * Aggreggates double facet values from the provided
    * {@link DoubleValuesSource}, pulling ordinals using {@link
    * DocValuesOrdinalsReader} against the default indexed
    * facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
@@ -64,29 +44,14 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
    public TaxonomyFacetSumValueSource(TaxonomyReader taxoReader, FacetsConfig config,
                                      FacetsCollector fc, DoubleValuesSource valueSource) throws IOException {
     this(new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME), taxoReader, config, fc, valueSource);
-  }
-
-  /**
-   * Aggreggates float facet values from the provided
-   *  {@link ValueSource}, and pulls ordinals from the
-   *  provided {@link OrdinalsReader}.
-   *
-   *  @deprecated use {@link #TaxonomyFacetSumValueSource(OrdinalsReader, TaxonomyReader, FacetsConfig, FacetsCollector, DoubleValuesSource)}
-   */
-  @Deprecated
-  public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
-                                     FacetsConfig config, FacetsCollector fc, ValueSource valueSource) throws IOException {
-    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
-    this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource.asDoubleValuesSource());
-  }
+   }
 
   /**
    * Aggreggates float facet values from the provided
    *  {@link DoubleValuesSource}, and pulls ordinals from the
    *  provided {@link OrdinalsReader}.
    */
-   public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
+  public TaxonomyFacetSumValueSource(OrdinalsReader ordinalsReader, TaxonomyReader taxoReader,
                                      FacetsConfig config, FacetsCollector fc, DoubleValuesSource vs) throws IOException {
     super(ordinalsReader.getIndexFieldName(), taxoReader, config);
     this.ordinalsReader = ordinalsReader;
@@ -134,41 +99,5 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
 
     rollup();
   }
-
-  /**
-   * {@link ValueSource} that returns the score for each
-   *  hit; use this to aggregate the sum of all hit scores
-   *  for each facet label.
-   *
-   * @deprecated Use {@link DoubleValuesSource#SCORES}
-   */
-  public static class ScoreValueSource extends ValueSource {
-
-    /** Sole constructor. */
-    public ScoreValueSource() {
-    }
-
-    @Override
-    public FunctionValues getValues(@SuppressWarnings("rawtypes") Map context, LeafReaderContext readerContext) throws IOException {
-      final Scorer scorer = (Scorer) context.get("scorer");
-      if (scorer == null) {
-        throw new IllegalStateException("scores are missing; be sure to pass keepScores=true to FacetsCollector");
-      }
-      return new DoubleDocValues(this) {
-        @Override
-        public double doubleVal(int document) {
-          try {
-            return scorer.score();
-          } catch (IOException exception) {
-            throw new RuntimeException(exception);
-          }
-        }
-      };
-    }
-    
-    @Override public boolean equals(Object o) { return o == this; }
-    @Override public int hashCode() { return System.identityHashCode(this); }
-    @Override public String description() { return "score()"; }
-  }
   
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index ff207d3..31f9e59 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -46,13 +46,11 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
-import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -437,7 +435,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
       } else {
         fastMatchQuery = null;
       }
-      ValueSource vs = new LongFieldSource("field");
+      LongValuesSource vs = LongValuesSource.fromLongField("field");
       Facets facets = new LongRangeFacetCounts("field", vs, sfc, fastMatchQuery, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
@@ -580,7 +578,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
       } else {
         fastMatchFilter = null;
       }
-      ValueSource vs = new DoubleFieldSource("field");
+      DoubleValuesSource vs = DoubleValuesSource.fromDoubleField("field");
       Facets facets = new DoubleRangeFacetCounts("field", vs, sfc, fastMatchFilter, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce8b678b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
index 31bf6e1..2af9925 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.facet.taxonomy;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -37,18 +36,12 @@ import org.apache.lucene.facet.FacetsConfig;
 import org.apache.lucene.facet.LabelAndValue;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.FunctionQuery;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
-import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
-import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
@@ -56,7 +49,6 @@ import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -121,7 +113,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     // Facets.search utility methods:
     searcher.search(new MatchAllDocsQuery(), c);
 
-    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, new IntFieldSource("num"));
+    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, DoubleValuesSource.fromIntField("num"));
 
     // Retrieve & verify results:
     assertEquals("dim=Author path=[] value=145.0 childCount=4\n  Lisa (50.0)\n  Frank (45.0)\n  Susan (40.0)\n  Bob (10.0)\n", facets.getTopChildren(10, "Author").toString());
@@ -181,7 +173,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector c = new FacetsCollector();
     searcher.search(new MatchAllDocsQuery(), c);    
 
-    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, new IntFieldSource("num"));
+    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, new FacetsConfig(), c, DoubleValuesSource.fromIntField("num"));
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
@@ -224,7 +216,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     FacetsCollector c = new FacetsCollector();
     searcher.search(new MatchAllDocsQuery(), c);    
 
-    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, config, c, new IntFieldSource("num"));
+    TaxonomyFacetSumValueSource facets = new TaxonomyFacetSumValueSource(taxoReader, config, c, DoubleValuesSource.fromIntField("num"));
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
@@ -295,7 +287,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     FacetsCollector sfc = new FacetsCollector();
     newSearcher(r).search(new MatchAllDocsQuery(), sfc);
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, new LongFieldSource("price"));
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, DoubleValuesSource.fromLongField("price"));
     assertEquals("dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n", facets.getTopChildren(10, "a").toString());
 
     iw.close();
@@ -319,34 +311,12 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     
     DirectoryReader r = DirectoryReader.open(iw);
     DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
-
-    ValueSource valueSource = new ValueSource() {
-      @Override
-      public FunctionValues getValues(@SuppressWarnings("rawtypes") Map context, LeafReaderContext readerContext) throws IOException {
-        final Scorer scorer = (Scorer) context.get("scorer");
-        assert scorer != null;
-        return new DoubleDocValues(this) {
-          @Override
-          public double doubleVal(int document) {
-            try {
-              return scorer.score();
-            } catch (IOException exception) {
-              throw new RuntimeException(exception);
-            }
-          }
-        };
-      }
-
-      @Override public boolean equals(Object o) { return o == this; }
-      @Override public int hashCode() { return System.identityHashCode(this); }
-      @Override public String description() { return "score()"; }
-    };
     
     FacetsCollector fc = new FacetsCollector(true);
     // score documents by their 'price' field - makes asserting the correct counts for the categories easier
     Query q = new FunctionQuery(new LongFieldSource("price"));
     FacetsCollector.search(newSearcher(r), q, 10, fc);
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, valueSource);
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, fc, DoubleValuesSource.SCORES);
     
     assertEquals("dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n", facets.getTopChildren(10, "a").toString());
 
@@ -374,10 +344,9 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     DirectoryReader r = DirectoryReader.open(iw);
     DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
 
-    ValueSource valueSource = new LongFieldSource("price");
     FacetsCollector sfc = new FacetsCollector();
     newSearcher(r).search(new MatchAllDocsQuery(), sfc);
-    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, valueSource);
+    Facets facets = new TaxonomyFacetSumValueSource(taxoReader, config, sfc, DoubleValuesSource.fromLongField("price"));
     
     assertEquals("dim=a path=[] value=10.0 childCount=2\n  1 (6.0)\n  0 (4.0)\n", facets.getTopChildren(10, "a").toString());
 
@@ -447,8 +416,6 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
     // NRT open
     TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
 
-    ValueSource values = new FloatFieldSource("value");
-
     int iters = atLeast(100);
     for(int iter=0;iter<iters;iter++) {
       String searchToken = tokens[random().nextInt(tokens.length)];
@@ -457,7 +424,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
       }
       FacetsCollector fc = new FacetsCollector();
       FacetsCollector.search(searcher, new TermQuery(new Term("content", searchToken)), 10, fc);
-      Facets facets = new TaxonomyFacetSumValueSource(tr, config, fc, values);
+      Facets facets = new TaxonomyFacetSumValueSource(tr, config, fc, DoubleValuesSource.fromFloatField("value"));
 
       // Slow, yet hopefully bug-free, faceting:
       @SuppressWarnings({"rawtypes","unchecked"}) Map<String,Float>[] expectedValues = new HashMap[numDims];
@@ -473,7 +440,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
               if (v == null) {
                 expectedValues[j].put(doc.dims[j], doc.value);
               } else {
-                expectedValues[j].put(doc.dims[j], v.floatValue() + doc.value);
+                expectedValues[j].put(doc.dims[j], v + doc.value);
               }
             }
           }


[29/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9954: Prevent against failure during failed snapshot cleanup from swallowing the actual cause for the snapshot to fail.

Posted by ho...@apache.org.
SOLR-9954: Prevent against failure during failed snapshot cleanup from swallowing the actual cause for the snapshot to fail.


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

Branch: refs/heads/jira/solr-5944
Commit: 118fc422d0cff8492db99edccb3d73068cf04b52
Parents: f99c967
Author: Timothy Potter <th...@gmail.com>
Authored: Tue Jan 10 18:35:19 2017 -0700
Committer: Timothy Potter <th...@gmail.com>
Committed: Tue Jan 10 18:35:19 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 3 +++
 solr/core/src/java/org/apache/solr/handler/SnapShooter.java | 6 +++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/118fc422/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0131b7b..0ee18ba 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -353,6 +353,9 @@ Bug Fixes
   and CloudMLTQParser included extra strings from the field definitions in the query.
   (Ere Maijala via Anshum Gupta)
 
+* SOLR-9954: Prevent against failure during failed snapshot cleanup from swallowing the actual cause
+  for the snapshot to fail. (thelabdude)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/118fc422/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
index bf02e4c..a6e8110 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
@@ -231,7 +231,11 @@ public class SnapShooter {
       return details;
     } finally {
       if (!success) {
-        backupRepo.deleteDirectory(snapshotDirPath);
+        try {
+          backupRepo.deleteDirectory(snapshotDirPath);
+        } catch (Exception excDuringDelete) {
+          LOG.warn("Failed to delete "+snapshotDirPath+" after snapshot creation failed due to: "+excDuringDelete);
+        }
       }
     }
   }


[46/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9893: For full Java 9 compatibility also update to latest Objenesis 2.5 (this allows mocking frameworks to instantiate objects without a ctor)

Posted by ho...@apache.org.
SOLR-9893: For full Java 9 compatibility also update to latest Objenesis 2.5 (this allows mocking frameworks to instantiate objects without a ctor)


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

Branch: refs/heads/jira/solr-5944
Commit: 27ec40d3f5117d22a14e179506bec8e545906077
Parents: 60d4a55
Author: Uwe Schindler <us...@apache.org>
Authored: Sun Jan 15 10:07:22 2017 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Sun Jan 15 10:07:22 2017 +0100

----------------------------------------------------------------------
 lucene/ivy-versions.properties       | 2 +-
 solr/licenses/objenesis-2.4.jar.sha1 | 1 -
 solr/licenses/objenesis-2.5.jar.sha1 | 1 +
 3 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/27ec40d3/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 770649c..6cb7f26 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -295,7 +295,7 @@ org.mortbay.jetty.version = 6.1.26
 /org.mortbay.jetty/jetty-util = ${org.mortbay.jetty.version}
 
 /org.noggit/noggit = 0.6
-/org.objenesis/objenesis = 2.4
+/org.objenesis/objenesis = 2.5
 
 org.ow2.asm.version = 5.1
 /org.ow2.asm/asm = ${org.ow2.asm.version}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/27ec40d3/solr/licenses/objenesis-2.4.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/objenesis-2.4.jar.sha1 b/solr/licenses/objenesis-2.4.jar.sha1
deleted file mode 100644
index 278f7dd..0000000
--- a/solr/licenses/objenesis-2.4.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-2916b6c96b50c5b3ec4452ed99401db745aabb27

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/27ec40d3/solr/licenses/objenesis-2.5.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/objenesis-2.5.jar.sha1 b/solr/licenses/objenesis-2.5.jar.sha1
new file mode 100644
index 0000000..f7f506d
--- /dev/null
+++ b/solr/licenses/objenesis-2.5.jar.sha1
@@ -0,0 +1 @@
+612ecb799912ccf77cba9b3ed8c813da086076e9


[43/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7626: I forgot to close the reader in this test

Posted by ho...@apache.org.
LUCENE-7626: I forgot to close the reader in this test


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

Branch: refs/heads/jira/solr-5944
Commit: e2c41af5017f67c279df239a1b99a00c4c4cf9b0
Parents: 9403372
Author: Mike McCandless <mi...@apache.org>
Authored: Sat Jan 14 06:21:01 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Jan 14 06:21:01 2017 -0500

----------------------------------------------------------------------
 .../src/test/org/apache/lucene/index/TestFixBrokenOffsets.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2c41af5/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
index bcd5a65..4ecbd13 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java
@@ -70,7 +70,7 @@ public class TestFixBrokenOffsets extends LuceneTestCase {
     assertNotNull("Broken offsets index not found", resource);
     Path path = createTempDir("brokenoffsets");
     TestUtil.unzip(resource, path);
-    Directory dir = FSDirectory.open(path);
+    Directory dir = newFSDirectory(path);
 
     // OK: index is 6.3.0 so offsets not checked:
     TestUtil.checkIndex(dir);
@@ -94,6 +94,7 @@ public class TestFixBrokenOffsets extends LuceneTestCase {
       codecReaders[i] = (CodecReader) leaves.get(i).reader();
     }
     w.addIndexes(codecReaders);
+    reader.close();
     w.close();
 
     // NOT OK: broken offsets were copied into a 7.0 segment:


[32/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9941: Clear deletes lists before log replay

Posted by ho...@apache.org.
SOLR-9941: Clear deletes lists before log replay


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

Branch: refs/heads/jira/solr-5944
Commit: 04f45aab768b053f32feece7343f994d25f0bb26
Parents: 2437204
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Wed Jan 11 21:51:04 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Wed Jan 11 21:51:04 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../java/org/apache/solr/update/UpdateLog.java  |   6 +
 .../org/apache/solr/search/TestRecovery.java    | 127 +++++++++++++++++++
 3 files changed, 136 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/04f45aab/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 204ea26..afcd10b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -74,6 +74,9 @@ Optimizations
 * SOLR-9584: Support Solr being proxied with another endpoint than default /solr, by using relative links
   in AdminUI javascripts (Yun Jie Zhou via janhoy)
 
+* SOLR-9941: Clear the deletes lists at UpdateLog before replaying from log. This prevents redundantly pre-applying
+  DBQs, during the log replay, to every update in the log as if the DBQs were out of order. (hossman, Ishan Chattopadhyaya)
+
 ==================  6.4.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/04f45aab/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index c40eafd..626dcd6 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -888,6 +888,12 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
     versionInfo.blockUpdates();
     try {
       state = State.REPLAYING;
+
+      // The deleteByQueries and oldDeletes lists
+      // would've been populated by items from the logs themselves (which we
+      // will replay now). So lets clear them out here before the replay.
+      deleteByQueries.clear();
+      oldDeletes.clear();
     } finally {
       versionInfo.unblockUpdates();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/04f45aab/solr/core/src/test/org/apache/solr/search/TestRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index f3e98dd..cc3ca47 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -25,6 +25,7 @@ import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.noggit.ObjectBuilder;
+import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.DirectUpdateHandler2;
@@ -198,6 +199,132 @@ public class TestRecovery extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testNewDBQAndDocMatchingOldDBQDuringLogReplay() throws Exception {
+    try {
+
+      DirectUpdateHandler2.commitOnClose = false;
+      final Semaphore logReplay = new Semaphore(0);
+      final Semaphore logReplayFinish = new Semaphore(0);
+
+      UpdateLog.testing_logReplayHook = () -> {
+        try {
+          assertTrue(logReplay.tryAcquire(timeout, TimeUnit.SECONDS));
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      };
+
+      UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
+
+      clearIndex();
+      assertU(commit());
+
+      // because we're sending updates during log replay, we can't emulate replica logic -- we need to use
+      // normal updates like a leader / single-node instance would get.
+      //
+      // (In SolrCloud mode, when a replica run recoverFromLog, replica in this time period will have state = DOWN,
+      // so It won't receive any updates.)
+      
+      updateJ(jsonAdd(sdoc("id","B0")),params());
+      updateJ(jsonAdd(sdoc("id","B1")),params()); // should be deleted by subsequent DBQ in tlog
+      updateJ(jsonAdd(sdoc("id","B2")),params()); // should be deleted by DBQ that arives during tlog replay
+      updateJ(jsonDelQ("id:B1 OR id:B3 OR id:B6"),params());
+      updateJ(jsonAdd(sdoc("id","B3")),params()); // should *NOT* be deleted by previous DBQ in tlog
+      updateJ(jsonAdd(sdoc("id","B4")),params()); // should be deleted by DBQ that arives during tlog replay
+      updateJ(jsonAdd(sdoc("id","B5")),params());
+      
+      // sanity check no updates have been applied yet (just in tlog)
+      assertJQ(req("q","*:*"),"/response/numFound==0");
+
+      h.close();
+      createCore(); // (Attempts to) kick off recovery (which is currently blocked by semaphore)
+
+      // verify that previous close didn't do a commit & that recovery should be blocked by our hook
+      assertJQ(req("q","*:*") ,"/response/numFound==0");
+
+      // begin recovery (first few items)
+      logReplay.release(TestUtil.nextInt(random(),1,6));
+      // ... but before recover is completely unblocked/finished, have a *new* DBQ arrive
+      // that should delete some items we either have just replayed, or are about to replay (or maybe both)...
+      updateJ(jsonDelQ("id:B2 OR id:B4"),params());
+      // ...and re-add a doc that would have matched a DBQ already in the tlog
+      // (which may/may-not have been replayed yet)
+      updateJ(jsonAdd(sdoc("id","B6")),params()); // should *NOT* be deleted by DBQ from tlog
+      assertU(commit());
+
+      // now completely unblock recovery
+      logReplay.release(1000);
+
+      // wait until recovery has finished
+      assertTrue(logReplayFinish.tryAcquire(timeout, TimeUnit.SECONDS));
+
+      // verify only the expected docs are found, even with out of order DBQ and DBQ that arived during recovery
+      assertJQ(req("q", "*:*", "fl", "id", "sort", "id asc")
+               , "/response/docs==[{'id':'B0'}, {'id':'B3'}, {'id':'B5'}, {'id':'B6'}]");
+      
+    } finally {
+      DirectUpdateHandler2.commitOnClose = true;
+      UpdateLog.testing_logReplayHook = null;
+      UpdateLog.testing_logReplayFinishHook = null;
+    }
+
+  }
+
+  @Test
+  public void testLogReplayWithReorderedDBQ() throws Exception {
+    try {
+
+      DirectUpdateHandler2.commitOnClose = false;
+      final Semaphore logReplay = new Semaphore(0);
+      final Semaphore logReplayFinish = new Semaphore(0);
+
+      UpdateLog.testing_logReplayHook = () -> {
+        try {
+          assertTrue(logReplay.tryAcquire(timeout, TimeUnit.SECONDS));
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      };
+
+      UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
+
+
+      clearIndex();
+      assertU(commit());
+
+      updateJ(jsonAdd(sdoc("id","B1", "_version_","1010")), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+      updateJ(jsonDelQ("id:B2"), params(DISTRIB_UPDATE_PARAM,FROM_LEADER, "_version_","-1017")); // This should've arrived after the 1015th update
+      updateJ(jsonAdd(sdoc("id","B2", "_version_","1015")), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+      updateJ(jsonAdd(sdoc("id","B3", "_version_","1020")), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+
+
+      assertJQ(req("q","*:*"),"/response/numFound==0");
+
+      h.close();
+      createCore();
+      // Solr should kick this off now
+      // h.getCore().getUpdateHandler().getUpdateLog().recoverFromLog();
+
+      // verify that previous close didn't do a commit
+      // recovery should be blocked by our hook
+      assertJQ(req("q","*:*") ,"/response/numFound==0");
+
+      // unblock recovery
+      logReplay.release(1000);
+
+      // wait until recovery has finished
+      assertTrue(logReplayFinish.tryAcquire(timeout, TimeUnit.SECONDS));
+
+      assertJQ(req("q","*:*") ,"/response/numFound==2");
+    } finally {
+      DirectUpdateHandler2.commitOnClose = true;
+      UpdateLog.testing_logReplayHook = null;
+      UpdateLog.testing_logReplayFinishHook = null;
+    }
+
+  }
+
+  @Test
   public void testBuffering() throws Exception {
 
     DirectUpdateHandler2.commitOnClose = false;


[49/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-5944: Merge branch 'master' into jira/solr-5944

Posted by ho...@apache.org.
SOLR-5944: Merge branch 'master' into jira/solr-5944

merged master into branch to pick up some bug fixes that we're waiting for...

* resolved conflicts / new-failures in TestRecovery
  - simple import collisions
  - divergent test changes resulted in a metrics assertion not expecting as many docs as it should (SOLR-9950)
* fixed (new) SkipExistingDocumentsProcessorFactory to use newer syntax for getInputDocumentFromTlog (SOLR-9918)
* SOLR-9941 allows TestRecovery.testLogReplayWithInPlaceUpdatesAndDeletes to start passing
* eliminated workaround for SOLR-9934 in TestInPlaceUpdatesStandalone now that the mainline fix is on this branch

Conflicts:
	solr/core/src/test/org/apache/solr/search/TestRecovery.java


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

Branch: refs/heads/jira/solr-5944
Commit: 027a92a400c60f23a81e9bf7ff9c32009583fb90
Parents: bb5dfb6 9aa78dc
Author: Chris Hostetter <ho...@apache.org>
Authored: Sun Jan 15 14:18:29 2017 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Sun Jan 15 14:18:29 2017 -0700

----------------------------------------------------------------------
 build.xml                                       |   42 +-
 .../dot.settings/org.eclipse.jdt.core.prefs     |    1 +
 dev-tools/idea/lucene/suggest/suggest.iml       |    2 -
 .../idea/solr/contrib/analytics/analytics.iml   |    1 +
 .../lucene/analysis/common/pom.xml.template     |   36 +-
 .../maven/lucene/analysis/icu/pom.xml.template  |   42 +-
 .../lucene/analysis/kuromoji/pom.xml.template   |   38 +-
 .../lucene/analysis/morfologik/pom.xml.template |   38 +-
 .../lucene/analysis/phonetic/pom.xml.template   |   38 +-
 .../maven/lucene/analysis/pom.xml.template      |   36 +-
 .../lucene/analysis/smartcn/pom.xml.template    |   36 +-
 .../lucene/analysis/stempel/pom.xml.template    |   36 +-
 .../maven/lucene/analysis/uima/pom.xml.template |   36 +-
 .../lucene/backward-codecs/pom.xml.template     |   36 +-
 .../maven/lucene/benchmark/pom.xml.template     |   36 +-
 .../lucene/classification/pom.xml.template      |   36 +-
 dev-tools/maven/lucene/codecs/pom.xml.template  |   36 +-
 .../lucene/codecs/src/java/pom.xml.template     |   36 +-
 .../lucene/codecs/src/test/pom.xml.template     |   36 +-
 dev-tools/maven/lucene/core/pom.xml.template    |   36 +-
 .../maven/lucene/core/src/java/pom.xml.template |   36 +-
 .../maven/lucene/core/src/test/pom.xml.template |   36 +-
 dev-tools/maven/lucene/demo/pom.xml.template    |   36 +-
 .../maven/lucene/expressions/pom.xml.template   |   36 +-
 dev-tools/maven/lucene/facet/pom.xml.template   |   36 +-
 .../maven/lucene/grouping/pom.xml.template      |   36 +-
 .../maven/lucene/highlighter/pom.xml.template   |   36 +-
 dev-tools/maven/lucene/join/pom.xml.template    |   36 +-
 dev-tools/maven/lucene/memory/pom.xml.template  |   36 +-
 dev-tools/maven/lucene/misc/pom.xml.template    |   36 +-
 dev-tools/maven/lucene/pom.xml.template         |   36 +-
 dev-tools/maven/lucene/queries/pom.xml.template |   36 +-
 .../maven/lucene/queryparser/pom.xml.template   |   36 +-
 .../maven/lucene/replicator/pom.xml.template    |   38 +-
 dev-tools/maven/lucene/sandbox/pom.xml.template |   36 +-
 .../lucene/spatial-extras/pom.xml.template      |   36 +-
 dev-tools/maven/lucene/spatial/pom.xml.template |   38 +-
 .../maven/lucene/spatial3d/pom.xml.template     |   36 +-
 dev-tools/maven/lucene/suggest/pom.xml.template |   36 +-
 .../lucene/test-framework/pom.xml.template      |   36 +-
 dev-tools/maven/pom.xml.template                |   36 +-
 .../contrib/analysis-extras/pom.xml.template    |   36 +-
 .../solr/contrib/analytics/pom.xml.template     |   36 +-
 .../solr/contrib/clustering/pom.xml.template    |   36 +-
 .../dataimporthandler-extras/pom.xml.template   |   36 +-
 .../contrib/dataimporthandler/pom.xml.template  |   36 +-
 .../solr/contrib/extraction/pom.xml.template    |   36 +-
 .../maven/solr/contrib/langid/pom.xml.template  |   36 +-
 .../maven/solr/contrib/ltr/pom.xml.template     |   36 +-
 .../solr/contrib/map-reduce/pom.xml.template    |   36 +-
 .../contrib/morphlines-cell/pom.xml.template    |   36 +-
 .../contrib/morphlines-core/pom.xml.template    |   36 +-
 dev-tools/maven/solr/contrib/pom.xml.template   |   36 +-
 .../maven/solr/contrib/uima/pom.xml.template    |   36 +-
 .../solr/contrib/velocity/pom.xml.template      |   36 +-
 dev-tools/maven/solr/core/pom.xml.template      |   36 +-
 .../maven/solr/core/src/java/pom.xml.template   |   36 +-
 .../maven/solr/core/src/test/pom.xml.template   |   36 +-
 dev-tools/maven/solr/pom.xml.template           |   36 +-
 dev-tools/maven/solr/solrj/pom.xml.template     |   36 +-
 .../maven/solr/solrj/src/java/pom.xml.template  |   36 +-
 .../maven/solr/solrj/src/test/pom.xml.template  |   36 +-
 .../maven/solr/test-framework/pom.xml.template  |   36 +-
 lucene/CHANGES.txt                              |  115 +-
 .../lucene/analysis/custom/CustomAnalyzer.java  |    2 +-
 .../miscellaneous/FixBrokenOffsetsFilter.java   |   78 +
 .../FixBrokenOffsetsFilterFactory.java          |   39 +
 .../analysis/synonym/FlattenGraphFilter.java    |  417 ++++
 .../synonym/FlattenGraphFilterFactory.java      |   44 +
 .../lucene/analysis/synonym/SynonymFilter.java  |    4 +
 .../analysis/synonym/SynonymFilterFactory.java  |    4 +
 .../analysis/synonym/SynonymGraphFilter.java    |  586 ++++++
 .../synonym/SynonymGraphFilterFactory.java      |  204 ++
 .../lucene/analysis/synonym/SynonymMap.java     |    7 +-
 .../lucene/analysis/util/CharTokenizer.java     |    6 +-
 ...ache.lucene.analysis.util.TokenFilterFactory |    3 +
 .../lucene/analysis/core/TestRandomChains.java  |  177 +-
 .../analysis/custom/TestCustomAnalyzer.java     |   21 +
 .../apache/lucene/analysis/custom/mapping1.txt  |    1 +
 .../apache/lucene/analysis/custom/mapping2.txt  |    1 +
 .../TestFixBrokenOffsetsFilter.java             |   50 +
 .../miscellaneous/TestWordDelimiterFilter.java  |   56 +-
 .../synonym/TestFlattenGraphFilter.java         |  284 +++
 .../synonym/TestSynonymGraphFilter.java         | 1956 ++++++++++++++++++
 .../apache/lucene/index/FixBrokenOffsets.java   |  125 ++
 .../java/org/apache/lucene/index/package.html   |   27 +
 .../lucene/index/TestFixBrokenOffsets.java      |  115 +
 .../lucene/index/index.630.brokenoffsets.zip    |  Bin 0 -> 3203 bytes
 lucene/common-build.xml                         |    2 +-
 .../lucene/analysis/TokenStreamToAutomaton.java |   11 +-
 .../PackedTokenAttributeImpl.java               |    9 +-
 .../apache/lucene/codecs/DocValuesConsumer.java |   10 +-
 .../org/apache/lucene/codecs/NormsConsumer.java |    2 +-
 .../lucene/codecs/StoredFieldsWriter.java       |    2 +-
 .../apache/lucene/codecs/TermVectorsWriter.java |    2 +-
 .../CompressingStoredFieldsWriter.java          |   68 +-
 .../lucene50/Lucene50StoredFieldsFormat.java    |    2 +-
 .../codecs/lucene60/Lucene60PointsWriter.java   |   11 +-
 .../lucene70/Lucene70DocValuesConsumer.java     |  163 +-
 .../lucene70/Lucene70DocValuesFormat.java       |    5 +-
 .../lucene70/Lucene70DocValuesProducer.java     |  220 +-
 .../org/apache/lucene/document/DoublePoint.java |   30 +-
 .../org/apache/lucene/document/FloatPoint.java  |   30 +-
 .../apache/lucene/index/AutomatonTermsEnum.java |    3 +
 .../lucene/index/BinaryDocValuesWriter.java     |   39 +-
 .../org/apache/lucene/index/CheckIndex.java     |   29 +-
 .../lucene/index/DefaultIndexingChain.java      |  136 +-
 .../org/apache/lucene/index/DocConsumer.java    |    2 +-
 .../org/apache/lucene/index/DocIDMerger.java    |  211 +-
 .../apache/lucene/index/DocValuesWriter.java    |    4 +-
 .../lucene/index/DocumentsWriterPerThread.java  |   31 +-
 .../lucene/index/FreqProxTermsWriter.java       |    8 +-
 .../org/apache/lucene/index/IndexWriter.java    |    8 +-
 .../lucene/index/MappingMultiPostingsEnum.java  |    2 +-
 .../org/apache/lucene/index/MergeState.java     |    8 +-
 .../apache/lucene/index/NormValuesWriter.java   |   17 +-
 .../lucene/index/NumericDocValuesWriter.java    |   51 +-
 .../apache/lucene/index/PointValuesWriter.java  |   96 +-
 .../apache/lucene/index/PrefixCodedTerms.java   |    4 +-
 .../org/apache/lucene/index/SegmentInfos.java   |    7 +-
 .../apache/lucene/index/SortedDocValues.java    |   22 +
 .../lucene/index/SortedDocValuesWriter.java     |   76 +-
 .../index/SortedNumericDocValuesWriter.java     |   64 +-
 .../apache/lucene/index/SortedSetDocValues.java |   22 +
 .../lucene/index/SortedSetDocValuesWriter.java  |   98 +-
 .../java/org/apache/lucene/index/Sorter.java    |   57 +-
 .../apache/lucene/index/SortingLeafReader.java  |   21 +-
 .../index/SortingStoredFieldsConsumer.java      |  206 ++
 .../index/SortingTermVectorsConsumer.java       |  181 ++
 .../lucene/index/StoredFieldsConsumer.java      |   85 +
 .../lucene/index/TermVectorsConsumer.java       |    7 +-
 .../java/org/apache/lucene/index/TermsHash.java |    4 +-
 .../TrackingTmpOutputDirectoryWrapper.java      |   53 +
 .../org/apache/lucene/search/DoubleValues.java  |   38 +
 .../lucene/search/DoubleValuesSource.java       |  340 +++
 .../org/apache/lucene/search/GraphQuery.java    |  136 ++
 .../lucene/search/LeafFieldComparator.java      |    2 +-
 .../org/apache/lucene/search/LongValues.java    |   38 +
 .../apache/lucene/search/LongValuesSource.java  |  244 +++
 .../lucene/search/MultiCollectorManager.java    |  107 +
 .../lucene/search/SimpleFieldComparator.java    |    2 +-
 .../apache/lucene/search/TermInSetQuery.java    |  369 ++++
 .../search/UsageTrackingQueryCachingPolicy.java |    5 +-
 .../org/apache/lucene/store/MMapDirectory.java  |  126 +-
 .../java/org/apache/lucene/util/LongValues.java |    9 +
 .../org/apache/lucene/util/QueryBuilder.java    |  113 +-
 .../apache/lucene/util/automaton/Automaton.java |    5 +-
 .../lucene/util/automaton/Operations.java       |    6 +-
 .../apache/lucene/util/automaton/StatePair.java |    4 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |   53 +-
 .../graph/GraphTokenStreamFiniteStrings.java    |  230 ++
 .../apache/lucene/util/graph/package-info.java  |   21 +
 .../apache/lucene/util/packed/DirectWriter.java |    8 +-
 .../lucene/analysis/TestGraphTokenizers.java    |  600 ++++++
 .../lucene70/TestLucene70DocValuesFormat.java   |  152 ++
 .../lucene/index/Test4GBStoredFields.java       |    2 +
 .../org/apache/lucene/index/TestCheckIndex.java |    5 -
 .../apache/lucene/index/TestDocIDMerger.java    |    4 +-
 .../apache/lucene/index/TestIndexSorting.java   |    2 -
 .../apache/lucene/index/TestIndexWriter.java    |    3 +-
 .../org/apache/lucene/index/TestTermsEnum.java  |    8 +
 .../lucene/search/TermInSetQueryTest.java       |  328 +++
 .../lucene/search/TestDoubleValuesSource.java   |  167 ++
 .../apache/lucene/search/TestGraphQuery.java    |   79 +
 .../apache/lucene/search/TestLRUQueryCache.java |    7 +
 .../lucene/search/TestLongValuesSource.java     |  149 ++
 .../apache/lucene/search/TestPointQueries.java  |   28 +
 .../org/apache/lucene/store/TestDirectory.java  |   13 +-
 .../apache/lucene/store/TestMmapDirectory.java  |    3 +-
 .../org/apache/lucene/store/TestMultiMMap.java  |    3 +-
 .../apache/lucene/util/TestQueryBuilder.java    |   15 +-
 .../org/apache/lucene/util/bkd/TestBKD.java     |   48 +-
 .../org/apache/lucene/util/fst/Test2BFST.java   |    2 +
 .../TestGraphTokenStreamFiniteStrings.java      |  217 ++
 .../demo/facet/DistanceFacetsExample.java       |   18 +-
 .../ExpressionAggregationFacetsExample.java     |    2 +-
 lucene/expressions/build.xml                    |   11 -
 .../org/apache/lucene/expressions/Bindings.java |   14 +-
 .../apache/lucene/expressions/Expression.java   |   21 +-
 .../expressions/ExpressionComparator.java       |  100 -
 .../expressions/ExpressionFunctionValues.java   |   35 +-
 .../lucene/expressions/ExpressionRescorer.java  |   33 +-
 .../lucene/expressions/ExpressionSortField.java |   77 -
 .../expressions/ExpressionValueSource.java      |   70 +-
 .../lucene/expressions/ScoreFunctionValues.java |   46 -
 .../lucene/expressions/ScoreValueSource.java    |   61 -
 .../lucene/expressions/SimpleBindings.java      |   30 +-
 .../expressions/js/JavascriptCompiler.java      |   15 +-
 .../apache/lucene/expressions/package-info.java |    5 +-
 .../lucene/expressions/TestDemoExpressions.java |   44 +-
 .../expressions/TestExpressionSortField.java    |    2 +-
 .../expressions/TestExpressionValueSource.java  |  111 +-
 .../expressions/js/TestCustomFunctions.java     |   20 +-
 .../expressions/js/TestJavascriptFunction.java  |    2 +-
 .../js/TestJavascriptOperations.java            |    2 +-
 .../org/apache/lucene/facet/DrillSideways.java  |  331 ++-
 .../org/apache/lucene/facet/FacetQuery.java     |   52 +
 .../lucene/facet/FacetsCollectorManager.java    |   55 +
 .../apache/lucene/facet/MultiFacetQuery.java    |   61 +
 .../org/apache/lucene/facet/package-info.java   |    2 +-
 .../apache/lucene/facet/range/DoubleRange.java  |   28 +-
 .../facet/range/DoubleRangeFacetCounts.java     |   63 +-
 .../apache/lucene/facet/range/LongRange.java    |   28 +-
 .../facet/range/LongRangeFacetCounts.java       |   26 +-
 .../org/apache/lucene/facet/range/Range.java    |   31 -
 .../lucene/facet/taxonomy/FakeScorer.java       |   53 -
 .../taxonomy/TaxonomyFacetSumValueSource.java   |  114 +-
 .../apache/lucene/facet/TestDrillSideways.java  |  315 +--
 .../org/apache/lucene/facet/TestFacetQuery.java |  103 +
 .../lucene/facet/TestParallelDrillSideways.java |   90 +
 .../facet/range/TestRangeFacetCounts.java       |   61 +-
 .../TestTaxonomyFacetSumValueSource.java        |   54 +-
 .../AbstractAllGroupHeadsCollector.java         |  176 --
 .../grouping/AbstractAllGroupsCollector.java    |   67 -
 .../AbstractDistinctValuesCollector.java        |   59 -
 .../AbstractFirstPassGroupingCollector.java     |  354 ----
 .../grouping/AbstractGroupFacetCollector.java   |  319 ---
 .../AbstractSecondPassGroupingCollector.java    |  162 --
 .../search/grouping/AllGroupHeadsCollector.java |  176 ++
 .../search/grouping/AllGroupsCollector.java     |   67 +
 .../search/grouping/CollectedSearchGroup.java   |    2 +-
 .../grouping/DistinctValuesCollector.java       |   59 +
 .../grouping/FirstPassGroupingCollector.java    |  363 ++++
 .../lucene/search/grouping/GroupDocs.java       |    8 +-
 .../search/grouping/GroupFacetCollector.java    |  324 +++
 .../apache/lucene/search/grouping/Grouper.java  |   56 +
 .../lucene/search/grouping/GroupingSearch.java  |  130 +-
 .../lucene/search/grouping/SearchGroup.java     |   23 +-
 .../grouping/SecondPassGroupingCollector.java   |  169 ++
 .../lucene/search/grouping/TopGroups.java       |   12 +-
 .../FunctionAllGroupHeadsCollector.java         |   32 +-
 .../function/FunctionAllGroupsCollector.java    |    4 +-
 .../FunctionDistinctValuesCollector.java        |   35 +-
 .../FunctionFirstPassGroupingCollector.java     |    6 +-
 .../grouping/function/FunctionGrouper.java      |   69 +
 .../FunctionSecondPassGroupingCollector.java    |    6 +-
 .../term/TermAllGroupHeadsCollector.java        |   26 +-
 .../grouping/term/TermAllGroupsCollector.java   |    8 +-
 .../term/TermDistinctValuesCollector.java       |   26 +-
 .../term/TermFirstPassGroupingCollector.java    |    6 +-
 .../grouping/term/TermGroupFacetCollector.java  |   10 +-
 .../search/grouping/term/TermGrouper.java       |   81 +
 .../term/TermSecondPassGroupingCollector.java   |    6 +-
 .../grouping/AllGroupHeadsCollectorTest.java    |   14 +-
 .../search/grouping/AllGroupsCollectorTest.java |    6 +-
 .../grouping/DistinctValuesCollectorTest.java   |   46 +-
 .../grouping/GroupFacetCollectorTest.java       |   10 +-
 .../lucene/search/grouping/TestGrouping.java    |   79 +-
 .../search/uhighlight/FieldHighlighter.java     |    4 +-
 .../uhighlight/LengthGoalBreakIterator.java     |  185 ++
 .../lucene/search/uhighlight/OffsetsEnum.java   |   45 +-
 .../lucene/search/uhighlight/Passage.java       |    1 +
 .../search/highlight/TokenSourcesTest.java      |    2 +-
 .../uhighlight/LengthGoalBreakIteratorTest.java |  104 +
 .../TestUnifiedHighlighterExtensibility.java    |   57 +-
 lucene/ivy-versions.properties                  |   10 +-
 .../apache/lucene/search/DocValuesStats.java    |  254 ++-
 .../search/TestDocValuesStatsCollector.java     |  333 ++-
 .../org/apache/lucene/queries/TermsQuery.java   |  381 ----
 .../lucene/queries/function/ValueSource.java    |  111 +
 .../apache/lucene/queries/TermsQueryTest.java   |  339 ---
 .../queryparser/classic/QueryParserBase.java    |   42 +-
 .../complexPhrase/ComplexPhraseQueryParser.java |    4 +-
 .../classic/TestMultiFieldQueryParser.java      |   11 +-
 .../queryparser/classic/TestQueryParser.java    |  131 +-
 .../complexPhrase/TestComplexPhraseQuery.java   |    6 +
 .../lucene/search/DocValuesNumbersQuery.java    |   26 +-
 .../lucene/search/DocValuesTermsQuery.java      |   49 +-
 .../org/apache/lucene/search/LongHashSet.java   |  156 ++
 .../lucene/document/TestHalfFloatPoint.java     |    2 +
 .../apache/lucene/search/LongHashSetTests.java  |  100 +
 .../lucene/search/TestDocValuesTermsQuery.java  |    1 +
 .../lucene/search/TestTermAutomatonQuery.java   |    3 +
 .../prefix/TermQueryPrefixTreeStrategy.java     |   10 +-
 .../spatial/prefix/NumberRangeFacetsTest.java   |    6 +-
 .../lucene/spatial3d/geom/GeoBBoxFactory.java   |   34 +-
 .../lucene/spatial3d/geom/GeoCircleFactory.java |    2 +-
 .../spatial3d/geom/GeoPolygonFactory.java       |    6 +-
 .../apache/lucene/spatial3d/geom/Vector.java    |    4 +
 lucene/suggest/build.xml                        |    8 +-
 .../suggest/DocumentValueSourceDictionary.java  |   58 +-
 .../analyzing/AnalyzingInfixSuggester.java      |  137 +-
 .../suggest/analyzing/FreeTextSuggester.java    |   45 -
 .../DocumentValueSourceDictionaryTest.java      |  287 ++-
 .../analysis/BaseTokenStreamTestCase.java       |   37 +-
 .../apache/lucene/analysis/MockTokenizer.java   |    3 +-
 .../asserting/AssertingLiveDocsFormat.java      |    9 +-
 .../index/BaseDocValuesFormatTestCase.java      |   32 +
 .../index/BaseStoredFieldsFormatTestCase.java   |    2 +
 .../index/BaseTermVectorsFormatTestCase.java    |   17 +-
 .../apache/lucene/index/BaseTestCheckIndex.java |   19 -
 .../org/apache/lucene/index/RandomCodec.java    |    2 +-
 .../org/apache/lucene/search/QueryUtils.java    |    4 -
 .../org/apache/lucene/util/LuceneTestCase.java  |  110 +-
 .../org/apache/lucene/util/RamUsageTester.java  |  102 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    4 +-
 .../lucene/analysis/TestGraphTokenizers.java    |  588 ------
 lucene/tools/javadoc/ecj.javadocs.prefs         |    2 +-
 lucene/tools/junit4/tests.policy                |    1 -
 solr/CHANGES.txt                                |  155 +-
 solr/bin/solr                                   |   26 +-
 solr/bin/solr.cmd                               |   36 +-
 .../plugin/AnalyticsStatisticsCollector.java    |    4 +-
 .../solr/collection1/conf/solrconfig.xml        |    5 -
 .../solr/handler/dataimport/JdbcDataSource.java |    4 +-
 .../handler/dataimport/SolrEntityProcessor.java |  192 +-
 .../dataimport/MockSolrEntityProcessor.java     |   18 +-
 .../handler/dataimport/TestJdbcDataSource.java  |   50 +-
 .../TestSolrEntityProcessorEndToEnd.java        |   27 +-
 .../dataimport/TestSolrEntityProcessorUnit.java |   70 +
 solr/contrib/ltr/README.md                      |  406 +---
 solr/contrib/ltr/example/README.md              |  132 ++
 solr/contrib/ltr/example/config.json            |   13 +-
 solr/contrib/ltr/example/exampleFeatures.json   |   26 +
 solr/contrib/ltr/example/libsvm_formatter.py    |   12 +-
 solr/contrib/ltr/example/solrconfig.xml         | 1722 ---------------
 .../ltr/example/techproducts-features.json      |   26 -
 .../contrib/ltr/example/techproducts-model.json |   18 -
 .../ltr/example/train_and_upload_demo_model.py  |  143 +-
 solr/contrib/ltr/example/user_queries.txt       |   12 +-
 .../org/apache/solr/ltr/CSVFeatureLogger.java   |   62 +
 .../java/org/apache/solr/ltr/FeatureLogger.java |  134 +-
 .../java/org/apache/solr/ltr/LTRRescorer.java   |    2 +-
 .../org/apache/solr/ltr/LTRScoringQuery.java    |    2 +-
 .../solr/ltr/SolrQueryRequestContextUtils.java  |    6 +-
 .../org/apache/solr/ltr/model/LinearModel.java  |   10 +
 .../ltr/model/MultipleAdditiveTreesModel.java   |    5 +
 .../LTRFeatureLoggerTransformerFactory.java     |  100 +-
 .../ltr/store/rest/ManagedFeatureStore.java     |    3 -
 .../solr/ltr/store/rest/ManagedModelStore.java  |    6 +-
 solr/contrib/ltr/src/java/overview.html         |    2 +-
 .../multipleadditivetreesmodel.json             |    2 +-
 .../solr/collection1/conf/solrconfig-ltr.xml    |    6 +-
 .../collection1/conf/solrconfig-ltr_Th10_10.xml |    5 +-
 .../collection1/conf/solrconfig-multiseg.xml    |   12 +-
 .../apache/solr/ltr/FeatureLoggerTestUtils.java |   44 +
 .../org/apache/solr/ltr/TestLTROnSolrCloud.java |   17 +-
 .../apache/solr/ltr/TestLTRQParserExplain.java  |    2 +-
 .../apache/solr/ltr/TestLTRQParserPlugin.java   |    5 +-
 .../org/apache/solr/ltr/TestLTRWithFacet.java   |    2 +-
 .../org/apache/solr/ltr/TestLTRWithSort.java    |    2 +-
 .../org/apache/solr/ltr/TestRerankBase.java     |   63 +-
 .../solr/ltr/TestSelectiveWeightCreation.java   |   13 +-
 .../ltr/feature/TestEdisMaxSolrFeature.java     |    2 +-
 .../solr/ltr/feature/TestExternalFeatures.java  |   39 +-
 .../ltr/feature/TestExternalValueFeatures.java  |   23 +-
 .../solr/ltr/feature/TestFeatureLogging.java    |  115 +-
 .../ltr/feature/TestFeatureLtrScoringModel.java |    2 +-
 .../solr/ltr/feature/TestFeatureStore.java      |    2 +-
 .../ltr/feature/TestFieldLengthFeature.java     |    2 +-
 .../solr/ltr/feature/TestFieldValueFeature.java |   15 +-
 .../solr/ltr/feature/TestFilterSolrFeature.java |    8 +-
 .../ltr/feature/TestNoMatchSolrFeature.java     |   94 +-
 .../ltr/feature/TestOriginalScoreFeature.java   |   11 +-
 .../solr/ltr/feature/TestRankingFeature.java    |    2 +-
 .../ltr/feature/TestUserTermScoreWithQ.java     |    2 +-
 .../ltr/feature/TestUserTermScorerQuery.java    |    2 +-
 .../ltr/feature/TestUserTermScorereQDF.java     |    2 +-
 .../solr/ltr/feature/TestValueFeature.java      |    2 +-
 .../apache/solr/ltr/model/TestLinearModel.java  |    2 +-
 .../model/TestMultipleAdditiveTreesModel.java   |   11 +-
 .../solr/ltr/store/rest/TestModelManager.java   |   49 +-
 .../store/rest/TestModelManagerPersistence.java |    6 +-
 solr/contrib/morphlines-core/ivy.xml            |    8 +-
 .../solr/solrcloud/conf/solrconfig.xml          |   11 -
 .../solr/SolrMorphlineZkAliasTest.java          |    7 -
 solr/core/ivy.xml                               |    5 +-
 .../org/apache/solr/cloud/CreateAliasCmd.java   |   21 +-
 .../java/org/apache/solr/cloud/Overseer.java    |    2 +-
 .../apache/solr/cloud/OverseerStatusCmd.java    |    4 +-
 .../apache/solr/cloud/rule/ReplicaAssigner.java |   31 +-
 .../java/org/apache/solr/cloud/rule/Rule.java   |    4 +-
 .../org/apache/solr/core/CoreContainer.java     |   84 +-
 .../org/apache/solr/core/DirectoryFactory.java  |   27 +
 .../apache/solr/core/HdfsDirectoryFactory.java  |   29 +
 .../org/apache/solr/core/JmxMonitoredMap.java   |   63 +-
 .../solr/core/MetricsDirectoryFactory.java      |  505 +++++
 .../java/org/apache/solr/core/NodeConfig.java   |   18 +-
 .../java/org/apache/solr/core/PluginInfo.java   |    4 +-
 .../java/org/apache/solr/core/SolrConfig.java   |    2 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  189 +-
 .../apache/solr/core/SolrDeletionPolicy.java    |    6 +
 .../org/apache/solr/core/SolrInfoMBean.java     |   11 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |   23 +-
 .../solr/core/StandardDirectoryFactory.java     |   46 +-
 .../apache/solr/handler/CdcrRequestHandler.java |    5 +
 .../org/apache/solr/handler/GraphHandler.java   |    4 +-
 .../org/apache/solr/handler/IndexFetcher.java   |   25 +-
 .../apache/solr/handler/PingRequestHandler.java |    5 +
 .../apache/solr/handler/ReplicationHandler.java |   21 +-
 .../apache/solr/handler/RequestHandlerBase.java |   57 +-
 .../org/apache/solr/handler/SchemaHandler.java  |    5 +
 .../org/apache/solr/handler/SnapShooter.java    |    6 +-
 .../apache/solr/handler/SolrConfigHandler.java  |    2 +-
 .../org/apache/solr/handler/StreamHandler.java  |   24 +-
 .../solr/handler/UpdateRequestHandler.java      |    5 +
 .../solr/handler/admin/CollectionsHandler.java  |    7 +-
 .../solr/handler/admin/ConfigSetsHandler.java   |    5 +
 .../solr/handler/admin/CoreAdminHandler.java    |   16 +-
 .../apache/solr/handler/admin/InfoHandler.java  |    5 +
 .../solr/handler/admin/LoggingHandler.java      |    6 +
 .../solr/handler/admin/LukeRequestHandler.java  |   23 +-
 .../solr/handler/admin/MetricsHandler.java      |  207 ++
 .../solr/handler/admin/PluginInfoHandler.java   |    5 +
 .../handler/admin/PropertiesRequestHandler.java |    5 +
 .../solr/handler/admin/SecurityConfHandler.java |    5 +
 .../admin/SegmentsInfoRequestHandler.java       |    6 +
 .../handler/admin/ShowFileRequestHandler.java   |    4 +
 .../handler/admin/SolrInfoMBeanHandler.java     |    5 +
 .../solr/handler/admin/SystemInfoHandler.java   |    7 +-
 .../solr/handler/admin/ThreadDumpHandler.java   |    5 +
 .../handler/admin/ZookeeperInfoHandler.java     |    5 +
 .../solr/handler/component/DebugComponent.java  |    5 +
 .../solr/handler/component/ExpandComponent.java |   30 +-
 .../solr/handler/component/FacetComponent.java  |    5 +
 .../handler/component/HighlightComponent.java   |    5 +
 .../handler/component/HttpShardHandler.java     |   48 +-
 .../component/HttpShardHandlerFactory.java      |  129 +-
 .../component/MoreLikeThisComponent.java        |    7 +-
 .../solr/handler/component/QueryComponent.java  |   23 +-
 .../component/QueryElevationComponent.java      |    4 +-
 .../handler/component/RealTimeGetComponent.java |    5 +
 .../solr/handler/component/ResponseBuilder.java |    1 -
 .../solr/handler/component/SearchHandler.java   |    2 +-
 .../solr/handler/component/ShardHandler.java    |    5 +-
 .../handler/component/SpellCheckComponent.java  |    5 +
 .../handler/component/TermVectorComponent.java  |    5 +
 .../solr/handler/component/TermsComponent.java  |    5 +
 .../solr/highlight/HighlightingPluginBase.java  |    2 +-
 .../solr/highlight/LuceneRegexFragmenter.java   |  217 ++
 .../apache/solr/highlight/RegexFragmenter.java  |  196 --
 .../solr/highlight/UnifiedSolrHighlighter.java  |  469 +++--
 .../solr/metrics/OperatingSystemMetricSet.java  |   92 +
 .../solr/metrics/SolrCoreMetricManager.java     |  148 ++
 .../org/apache/solr/metrics/SolrMetricInfo.java |  104 +
 .../apache/solr/metrics/SolrMetricManager.java  |  675 ++++++
 .../apache/solr/metrics/SolrMetricProducer.java |   32 +
 .../apache/solr/metrics/SolrMetricReporter.java |   83 +
 .../org/apache/solr/metrics/package-info.java   |   23 +
 .../metrics/reporters/JmxObjectNameFactory.java |  155 ++
 .../metrics/reporters/SolrGangliaReporter.java  |  144 ++
 .../metrics/reporters/SolrGraphiteReporter.java |  129 ++
 .../solr/metrics/reporters/SolrJmxReporter.java |  179 ++
 .../metrics/reporters/SolrSlf4jReporter.java    |  127 ++
 .../solr/metrics/reporters/package-info.java    |   22 +
 .../apache/solr/parser/SolrQueryParserBase.java |   16 +-
 .../request/PerSegmentSingleValuedFaceting.java |    9 +-
 .../org/apache/solr/request/SimpleFacets.java   |   41 +-
 .../solr/response/JSONResponseWriter.java       |   86 +-
 .../java/org/apache/solr/schema/FieldType.java  |    4 +-
 .../apache/solr/schema/PreAnalyzedField.java    |   11 +
 .../org/apache/solr/schema/SchemaManager.java   |   99 +-
 .../org/apache/solr/search/CacheConfig.java     |    2 +-
 .../solr/search/CollapsingQParserPlugin.java    |   14 +-
 .../solr/search/ComplexPhraseQParserPlugin.java |   70 +-
 .../org/apache/solr/search/FastLRUCache.java    |    2 +-
 .../java/org/apache/solr/search/Grouping.java   |   14 +-
 .../apache/solr/search/TermsQParserPlugin.java  |    4 +-
 .../apache/solr/search/facet/FacetModule.java   |    5 +
 .../org/apache/solr/search/facet/HLLAgg.java    |   13 +-
 .../apache/solr/search/facet/PercentileAgg.java |    7 +-
 .../org/apache/solr/search/facet/SlotAcc.java   |    5 +-
 .../apache/solr/search/facet/UniqueSlotAcc.java |    2 +-
 .../solr/search/grouping/CommandHandler.java    |    4 +-
 .../search/grouping/GroupingSpecification.java  |   18 +-
 .../command/SearchGroupsFieldCommand.java       |    8 +-
 .../command/TopGroupsFieldCommand.java          |    4 +-
 .../org/apache/solr/search/join/GraphQuery.java |    4 +-
 .../apache/solr/search/mlt/CloudMLTQParser.java |   49 +-
 .../solr/search/mlt/SimpleMLTQParser.java       |   30 +-
 .../security/AttributeOnlyServletContext.java   |  291 +++
 .../ConfigurableInternodeAuthHadoopPlugin.java  |   68 +
 .../security/DelegationTokenKerberosFilter.java |    6 +-
 .../solr/security/GenericHadoopAuthPlugin.java  |  266 +++
 .../apache/solr/security/HadoopAuthFilter.java  |  198 ++
 .../apache/solr/security/HadoopAuthPlugin.java  |  241 +++
 .../apache/solr/security/KerberosFilter.java    |    6 +-
 .../apache/solr/security/KerberosPlugin.java    |  314 +--
 .../solr/security/PKIAuthenticationPlugin.java  |    6 +
 .../solr/security/PermissionNameProvider.java   |    1 +
 ...tContinuesRecorderAuthenticationHandler.java |   71 +
 .../apache/solr/servlet/SolrDispatchFilter.java |   27 +
 .../solr/spelling/DirectSolrSpellChecker.java   |   18 +-
 .../DocumentExpressionDictionaryFactory.java    |    6 +-
 .../solr/update/DirectUpdateHandler2.java       |    5 -
 .../java/org/apache/solr/update/PeerSync.java   |   50 +-
 .../org/apache/solr/update/SolrIndexConfig.java |   16 +-
 .../org/apache/solr/update/SolrIndexWriter.java |  144 ++
 .../org/apache/solr/update/UpdateHandler.java   |    5 +
 .../java/org/apache/solr/update/UpdateLog.java  |   74 +-
 .../apache/solr/update/UpdateShardHandler.java  |   81 +-
 .../solr/update/UpdateShardHandlerConfig.java   |   14 +-
 .../AddSchemaFieldsUpdateProcessorFactory.java  |    3 +-
 .../SkipExistingDocumentsProcessorFactory.java  |  258 +++
 .../src/java/org/apache/solr/util/DOMUtil.java  |    5 +
 .../src/java/org/apache/solr/util/JmxUtil.java  |   78 +
 .../src/java/org/apache/solr/util/SolrCLI.java  |   65 +-
 .../src/java/org/apache/solr/util/TimeOut.java  |   13 +-
 .../stats/HttpClientMetricNameStrategy.java     |   28 +
 .../stats/InstrumentedHttpRequestExecutor.java  |  136 ++
 ...entedPoolingHttpClientConnectionManager.java |   72 +
 .../org/apache/solr/util/stats/MetricUtils.java |  203 ++
 .../org/apache/solr/util/stats/TimerUtils.java  |   58 -
 .../resources/EditableSolrConfigAttributes.json |   16 +-
 .../schema-minimal-with-another-uniqkey.xml     |   23 +
 ...dd-schema-fields-update-processor-chains.xml |    8 +-
 .../conf/solrconfig-cache-enable-disable.xml    |   80 +
 .../conf/solrconfig-indexmetrics.xml            |   57 +
 .../collection1/conf/solrconfig-schemaless.xml  |   45 +-
 .../solr/collection1/conf/solrconfig_perf.xml   |    1 -
 .../solr/security/hadoop_kerberos_config.json   |   16 +
 .../hadoop_simple_auth_with_delegation.json     |   29 +
 .../test-files/solr/solr-gangliareporter.xml    |   32 +
 .../test-files/solr/solr-graphitereporter.xml   |   31 +
 .../src/test-files/solr/solr-metricreporter.xml |   57 +
 .../src/test-files/solr/solr-slf4jreporter.xml  |   35 +
 .../TestReversedWildcardFilterFactory.java      |   56 +-
 .../apache/solr/cloud/AliasIntegrationTest.java |   50 +-
 .../solr/cloud/BasicDistributedZkTest.java      |    2 +-
 .../apache/solr/cloud/CdcrBootstrapTest.java    |   25 +-
 .../org/apache/solr/cloud/ClusterStateTest.java |    8 +
 .../solr/cloud/CollectionsAPISolrJTest.java     |  470 ++---
 .../DeleteLastCustomShardedReplicaTest.java     |  104 +-
 .../org/apache/solr/cloud/DeleteShardTest.java  |  211 +-
 .../solr/cloud/DocValuesNotIndexedTest.java     |   10 -
 .../cloud/LeaderFailoverAfterPartitionTest.java |    2 +-
 .../cloud/LeaderFailureAfterFreshStartTest.java |   68 +-
 ...verseerCollectionConfigSetProcessorTest.java |    4 +-
 .../cloud/OverseerModifyCollectionTest.java     |   92 +-
 .../apache/solr/cloud/OverseerRolesTest.java    |  165 +-
 .../apache/solr/cloud/OverseerStatusTest.java   |   55 +-
 .../solr/cloud/PeerSyncReplicationTest.java     |   70 +-
 .../apache/solr/cloud/RemoteQueryErrorTest.java |   53 +-
 .../apache/solr/cloud/TestCloudRecovery.java    |   26 +
 .../solr/cloud/TestDownShardTolerantSearch.java |   40 +-
 .../TestExclusionRuleCollectionAccess.java      |   38 +-
 .../TestSolrCloudWithSecureImpersonation.java   |    8 +-
 .../apache/solr/cloud/TestStressLiveNodes.java  |    2 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |    2 +-
 .../solr/core/BlobRepositoryMockingTest.java    |    9 +-
 .../org/apache/solr/core/CoreSorterTest.java    |    7 +
 .../solr/core/MockShardHandlerFactory.java      |    2 +-
 .../test/org/apache/solr/core/TestConfig.java   |   34 +
 .../solr/handler/TestReplicationHandler.java    |   20 +-
 .../admin/CoreMergeIndexesAdminHandlerTest.java |   10 +-
 .../solr/handler/admin/MBeansHandlerTest.java   |    4 +-
 .../solr/handler/admin/MetricsHandlerTest.java  |  149 ++
 .../admin/SegmentsInfoRequestHandlerTest.java   |   17 +-
 .../solr/handler/admin/StatsReloadRaceTest.java |   82 +-
 .../highlight/TestUnifiedSolrHighlighter.java   |   34 +-
 .../solr/index/hdfs/CheckHdfsIndexTest.java     |    5 -
 .../org/apache/solr/metrics/JvmMetricsTest.java |   68 +
 .../solr/metrics/SolrCoreMetricManagerTest.java |  171 ++
 .../solr/metrics/SolrMetricManagerTest.java     |  273 +++
 .../solr/metrics/SolrMetricReporterTest.java    |   69 +
 .../solr/metrics/SolrMetricTestUtils.java       |  103 +
 .../metrics/SolrMetricsIntegrationTest.java     |  143 ++
 .../metrics/reporters/MockMetricReporter.java   |   80 +
 .../reporters/SolrGangliaReporterTest.java      |   82 +
 .../reporters/SolrGraphiteReporterTest.java     |  116 ++
 .../metrics/reporters/SolrJmxReporterTest.java  |  156 ++
 .../reporters/SolrSlf4jReporterTest.java        |   77 +
 .../apache/solr/request/SimpleFacetsTest.java   |   12 +-
 .../apache/solr/response/JSONWriterTest.java    |   20 +-
 .../TestSubQueryTransformerDistrib.java         |   55 +-
 .../schema/TestSchemalessBufferedUpdates.java   |  160 ++
 .../ApacheLuceneSolrNearQueryBuilder.java       |   51 +
 .../TestComplexPhraseLeadingWildcard.java       |  113 +
 .../org/apache/solr/search/TestRTGBase.java     |   12 -
 .../org/apache/solr/search/TestRealTimeGet.java |    2 +-
 .../org/apache/solr/search/TestRecovery.java    |  183 +-
 .../apache/solr/search/TestRecoveryHdfs.java    |   11 -
 .../apache/solr/search/TestSolrCoreParser.java  |  110 +
 .../apache/solr/search/TestSolrQueryParser.java |   12 +-
 .../apache/solr/search/TestStressLucene.java    |    2 +-
 .../apache/solr/search/TestStressRecovery.java  |    2 +-
 .../apache/solr/search/TestStressReorder.java   |    5 +-
 .../solr/search/TestStressUserVersions.java     |    4 +-
 .../apache/solr/search/TestStressVersions.java  |    2 +-
 .../solr/search/facet/TestJsonFacets.java       |   71 +-
 .../solr/search/mlt/CloudMLTQParserTest.java    |   23 +-
 .../solr/search/mlt/SimpleMLTQParserTest.java   |   33 +-
 .../PKIAuthenticationIntegrationTest.java       |   40 +-
 .../security/TestPKIAuthenticationPlugin.java   |    8 +
 .../solr/security/hadoop/ImpersonationUtil.java |   73 +
 .../hadoop/ImpersonatorCollectionsHandler.java  |   60 +
 .../hadoop/TestDelegationWithHadoopAuth.java    |  400 ++++
 .../hadoop/TestImpersonationWithHadoopAuth.java |  215 ++
 .../TestSolrCloudWithHadoopAuthPlugin.java      |  138 ++
 .../solr/servlet/SolrRequestParserTest.java     |    3 +
 .../org/apache/solr/update/AutoCommitTest.java  |    2 +-
 .../apache/solr/update/CdcrUpdateLogTest.java   |   17 -
 .../apache/solr/update/SoftAutoCommitTest.java  |    2 +-
 .../apache/solr/update/SolrIndexConfigTest.java |    1 +
 .../solr/update/SolrIndexMetricsTest.java       |   94 +
 .../update/TestInPlaceUpdatesStandalone.java    |   14 +-
 ...ipExistingDocumentsProcessorFactoryTest.java |  335 +++
 .../apache/solr/util/stats/MetricUtilsTest.java |   56 +
 .../apache/solr/util/stats/TimerUtilsTest.java  |   58 -
 .../example-DIH/solr/db/conf/solrconfig.xml     |   11 -
 .../example-DIH/solr/mail/conf/solrconfig.xml   |   11 -
 .../example-DIH/solr/rss/conf/solrconfig.xml    |   11 -
 .../example-DIH/solr/solr/conf/solrconfig.xml   |   11 -
 .../example-DIH/solr/tika/conf/solrconfig.xml   |   11 -
 solr/example/files/conf/solrconfig.xml          |   16 +-
 solr/licenses/byte-buddy-1.6.2.jar.sha1         |    1 +
 solr/licenses/byte-buddy-LICENSE-ASL.txt        |  202 ++
 solr/licenses/byte-buddy-NOTICE.txt             |    4 +
 solr/licenses/gmetric4j-1.0.7.jar.sha1          |    1 +
 solr/licenses/gmetric4j-LICENSE-BSD.txt         |   31 +
 solr/licenses/gmetric4j-NOTICE.txt              |    0
 solr/licenses/metrics-ganglia-3.1.2.jar.sha1    |    1 +
 solr/licenses/metrics-ganglia-LICENSE-ASL.txt   |  203 ++
 solr/licenses/metrics-ganglia-NOTICE.txt        |   12 +
 solr/licenses/metrics-graphite-3.1.2.jar.sha1   |    1 +
 solr/licenses/metrics-graphite-LICENSE-ASL.txt  |  203 ++
 solr/licenses/metrics-graphite-NOTICE.txt       |   12 +
 solr/licenses/metrics-jetty-LICENSE-ASL.txt     |  203 ++
 solr/licenses/metrics-jetty-NOTICE.txt          |   12 +
 solr/licenses/metrics-jetty9-3.1.2.jar.sha1     |    1 +
 solr/licenses/metrics-json-LICENSE-ASL.txt      |  203 ++
 solr/licenses/metrics-json-NOTICE.txt           |   12 +
 solr/licenses/metrics-jvm-3.1.2.jar.sha1        |    1 +
 solr/licenses/metrics-jvm-LICENSE-ASL.txt       |  203 ++
 solr/licenses/metrics-jvm-NOTICE.txt            |   12 +
 solr/licenses/metrics-servlets-LICENSE-ASL.txt  |  203 ++
 solr/licenses/metrics-servlets-NOTICE.txt       |   12 +
 solr/licenses/mockito-core-1.9.5.jar.sha1       |    1 -
 solr/licenses/mockito-core-2.6.2.jar.sha1       |    1 +
 solr/licenses/objenesis-1.2.jar.sha1            |    1 -
 solr/licenses/objenesis-2.5.jar.sha1            |    1 +
 solr/server/build.xml                           |    4 +-
 solr/server/etc/jetty.xml                       |   20 +-
 solr/server/ivy.xml                             |   14 +-
 .../basic_configs/conf/solrconfig.xml           |   17 +-
 .../conf/solrconfig.xml                         |   16 +-
 .../conf/solrconfig.xml                         |   65 +-
 .../impl/DelegationTokenHttpSolrClient.java     |   34 +-
 .../solrj/impl/HttpClientBuilderFactory.java    |   41 +
 .../solr/client/solrj/impl/HttpClientUtil.java  |   35 +-
 .../solr/client/solrj/impl/HttpSolrClient.java  |   56 +-
 .../solrj/impl/Krb5HttpClientBuilder.java       |   10 +-
 .../solr/client/solrj/io/ops/AndOperation.java  |  101 +
 .../client/solrj/io/ops/BooleanOperation.java   |   26 +
 .../client/solrj/io/ops/EqualsOperation.java    |   70 +
 .../io/ops/GreaterThanEqualToOperation.java     |   70 +
 .../solrj/io/ops/GreaterThanOperation.java      |   70 +
 .../solr/client/solrj/io/ops/LeafOperation.java |   67 +
 .../solrj/io/ops/LessThanEqualToOperation.java  |   70 +
 .../client/solrj/io/ops/LessThanOperation.java  |   70 +
 .../solr/client/solrj/io/ops/NotOperation.java  |   87 +
 .../solr/client/solrj/io/ops/OrOperation.java   |   71 +
 .../client/solrj/io/stream/CloudSolrStream.java |    4 +
 .../client/solrj/io/stream/HavingStream.java    |  173 ++
 .../solr/client/solrj/io/stream/NullStream.java |  155 ++
 .../client/solrj/io/stream/PriorityStream.java  |  161 ++
 .../solrj/io/stream/expr/StreamFactory.java     |    2 +
 .../solrj/request/CollectionAdminRequest.java   |    2 +
 .../apache/solr/common/params/CommonParams.java |    4 +-
 .../solr/common/params/HighlightParams.java     |    4 +-
 .../solr/client/solrj/LargeVolumeTestBase.java  |    3 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |    1 -
 .../solrj/impl/BasicHttpSolrClientTest.java     |   31 +
 .../solrj/impl/CloudSolrClientCacheTest.java    |    7 +
 .../client/solrj/impl/CloudSolrClientTest.java  |    8 +-
 .../solrj/io/graph/GraphExpressionTest.java     |    3 +-
 .../solr/client/solrj/io/sql/JdbcTest.java      |    7 +-
 .../client/solrj/io/stream/JDBCStreamTest.java  |    9 +-
 .../solrj/io/stream/StreamExpressionTest.java   |  467 ++++-
 .../client/solrj/io/stream/StreamingTest.java   |   10 +-
 .../solr/common/params/CommonParamsTest.java    |    2 +
 .../java/org/apache/solr/SolrJettyTestBase.java |    2 -
 .../java/org/apache/solr/SolrTestCaseHS.java    |    4 +
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   24 +-
 .../solr/cloud/AbstractDistribZkTestBase.java   |   31 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java |    2 -
 .../apache/solr/cloud/MiniSolrCloudCluster.java |   29 +-
 .../apache/solr/cloud/SolrCloudTestCase.java    |   30 +-
 .../component/TrackingShardHandlerFactory.java  |    2 +-
 .../java/org/apache/solr/util/TestHarness.java  |   12 +-
 solr/webapp/web/js/angular/services.js          |   46 +-
 681 files changed, 31988 insertions(+), 11353 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/java/org/apache/solr/update/PeerSync.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 87833eb,7c2dae6..23a7670
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@@ -42,9 -40,10 +42,11 @@@ import java.util.concurrent.Synchronous
  import java.util.concurrent.ThreadPoolExecutor;
  import java.util.concurrent.TimeUnit;
  
+ import com.codahale.metrics.Gauge;
+ import com.codahale.metrics.Meter;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.lucene.util.BytesRef;
 +import org.apache.solr.common.SolrDocumentBase;
  import org.apache.solr.common.SolrException;
  import org.apache.solr.common.SolrException.ErrorCode;
  import org.apache.solr.common.SolrInputDocument;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
index 0000000,ec637a4..2c58410
mode 000000,100644..100644
--- a/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
@@@ -1,0 -1,255 +1,258 @@@
+ /*
+  * 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.update.processor;
+ 
+ import org.apache.lucene.util.BytesRef;
+ import org.apache.solr.common.SolrException;
+ import org.apache.solr.common.SolrInputDocument;
+ import org.apache.solr.common.util.NamedList;
+ import org.apache.solr.core.SolrCore;
+ import org.apache.solr.handler.component.RealTimeGetComponent;
+ import org.apache.solr.request.SolrQueryRequest;
+ import org.apache.solr.response.SolrQueryResponse;
+ import org.apache.solr.search.SolrIndexSearcher;
+ import org.apache.solr.update.AddUpdateCommand;
+ import org.apache.solr.update.UpdateCommand;
+ import org.apache.solr.util.RefCounted;
+ import org.apache.solr.util.plugin.SolrCoreAware;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import java.io.IOException;
+ import java.lang.invoke.MethodHandles;
++import java.util.Collections;
+ 
+ import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+ import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+ 
+ /**
+  * <p>
+  *     This Factory generates an UpdateProcessor that will (by default) skip inserting new documents
+  *     if there already exists a document with the same uniqueKey value in the index. It will also
+  *     skip Atomic Updates to a document if that document does not already exist. This behaviour is applied
+  *     to each document in turn, so adding a batch of documents can result in some being added and some
+  *     ignored, depending on what is already in the index. If all of the documents are skipped, no changes
+  *     to the index will occur.
+  * </p>
+  * These two forms of skipping can be switched on or off independently, by using init params:
+  * <ul>
+  *     <li><code>skipInsertIfExists</code> - This boolean parameter defaults to
+  *          <code>true</code>, but if set to <code>false</code> then inserts (i.e. not Atomic Updates)
+  *          will be passed through unchanged even if the document already exists.</li>
+  *     <li><code>skipUpdateIfMissing</code> - This boolean parameter defaults to
+  *         <code>true</code>, but if set to <code>false</code> then Atomic Updates
+  *          will be passed through unchanged regardless of whether the document exists.</li>
+  * </ul>
+  * <p>
+  *     These params can also be specified per-request, to override the configured behaviour
+  *     for specific updates e.g. <code>/update?skipUpdateIfMissing=true</code>
+  * </p>
+  * <p>
+  *     This implementation is a simpler alternative to {@link DocBasedVersionConstraintsProcessorFactory}
+  *     when you are not concerned with versioning, and just want to quietly ignore duplicate documents and/or
+  *     silently skip updates to non-existent documents (in the same way a database <code>UPDATE</code> would).
+  *
+  *     If your documents do have an explicit version field, and you want to ensure older versions are
+  *     skipped instead of replacing the indexed document, you should consider {@link DocBasedVersionConstraintsProcessorFactory}
+  *     instead.
+  * </p>
+  * <p>
+  *     An example chain configuration to use this for skipping duplicate inserts, but not skipping updates to
+  *     missing documents by default, is:
+  * </p>
+  * <pre class="prettyprint">
+  * &lt;updateRequestProcessorChain name="skipexisting"&gt;
+  *   &lt;processor class="solr.LogUpdateProcessorFactory" /&gt;
+  *   &lt;processor class="solr.SkipExistingDocumentsProcessorFactory"&gt;
+  *     &lt;bool name="skipInsertIfExists"&gt;true&lt;/bool&gt;
+  *     &lt;bool name="skipUpdateIfMissing"&gt;false&lt;/bool&gt; &lt;!-- Can override this per-request --&gt;
+  *   &lt;/processor&gt;
+  *   &lt;processor class="solr.DistributedUpdateProcessorFactory" /&gt;
+  *   &lt;processor class="solr.RunUpdateProcessorFactory" /&gt;
+  * &lt;/updateRequestProcessorChain&gt;
+  * </pre>
+  */
+ public class SkipExistingDocumentsProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
+   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+ 
+   private static final String PARAM_SKIP_INSERT_IF_EXISTS = "skipInsertIfExists";
+   private static final String PARAM_SKIP_UPDATE_IF_MISSING = "skipUpdateIfMissing";
+ 
+   private boolean skipInsertIfExists = true;
+   private boolean skipUpdateIfMissing = true;
+ 
+   @Override
+   public void init(NamedList args)  {
+     Object tmp = args.remove(PARAM_SKIP_INSERT_IF_EXISTS);
+     if (null != tmp) {
+       if (! (tmp instanceof Boolean) ) {
+         throw new SolrException(SERVER_ERROR, "'" + PARAM_SKIP_INSERT_IF_EXISTS + "' must be configured as a <bool>");
+       }
+       skipInsertIfExists = (Boolean)tmp;
+     }
+     tmp = args.remove(PARAM_SKIP_UPDATE_IF_MISSING);
+     if (null != tmp) {
+       if (! (tmp instanceof Boolean) ) {
+         throw new SolrException(SERVER_ERROR, "'" + PARAM_SKIP_UPDATE_IF_MISSING + "' must be configured as a <bool>");
+       }
+       skipUpdateIfMissing = (Boolean)tmp;
+     }
+ 
+     super.init(args);
+   }
+ 
+   @Override
+   public SkipExistingDocumentsUpdateProcessor getInstance(SolrQueryRequest req,
+                                                           SolrQueryResponse rsp,
+                                                           UpdateRequestProcessor next) {
+     // Ensure the parameters are forwarded to the leader
+     DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist(req, PARAM_SKIP_INSERT_IF_EXISTS, PARAM_SKIP_UPDATE_IF_MISSING);
+ 
+     // Allow the particular request to override the plugin's configured behaviour
+     boolean skipInsertForRequest = req.getOriginalParams().getBool(PARAM_SKIP_INSERT_IF_EXISTS, this.skipInsertIfExists);
+     boolean skipUpdateForRequest = req.getOriginalParams().getBool(PARAM_SKIP_UPDATE_IF_MISSING, this.skipUpdateIfMissing);
+ 
+     return new SkipExistingDocumentsUpdateProcessor(req, next, skipInsertForRequest, skipUpdateForRequest);
+   }
+ 
+   @Override
+   public void inform(SolrCore core) {
+ 
+     if (core.getUpdateHandler().getUpdateLog() == null) {
+       throw new SolrException(SERVER_ERROR, "updateLog must be enabled.");
+     }
+ 
+     if (core.getLatestSchema().getUniqueKeyField() == null) {
+       throw new SolrException(SERVER_ERROR, "schema must have uniqueKey defined.");
+     }
+   }
+ 
+   static class SkipExistingDocumentsUpdateProcessor extends UpdateRequestProcessor {
+ 
+     private final boolean skipInsertIfExists;
+     private final boolean skipUpdateIfMissing;
+     private final SolrCore core;
+ 
+     private DistributedUpdateProcessor distribProc;  // the distributed update processor following us
+     private DistributedUpdateProcessor.DistribPhase phase;
+ 
+     SkipExistingDocumentsUpdateProcessor(SolrQueryRequest req,
+                                          UpdateRequestProcessor next,
+                                          boolean skipInsertIfExists,
+                                          boolean skipUpdateIfMissing) {
+       super(next);
+       this.skipInsertIfExists = skipInsertIfExists;
+       this.skipUpdateIfMissing = skipUpdateIfMissing;
+       this.core = req.getCore();
+ 
+       for (UpdateRequestProcessor proc = next ;proc != null; proc = proc.next) {
+         if (proc instanceof DistributedUpdateProcessor) {
+           distribProc = (DistributedUpdateProcessor)proc;
+           break;
+         }
+       }
+ 
+       if (distribProc == null) {
+         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "DistributedUpdateProcessor must follow SkipExistingDocumentsUpdateProcessor");
+       }
+ 
+       phase = DistributedUpdateProcessor.DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
+     }
+ 
+     boolean isSkipInsertIfExists() {
+       return this.skipInsertIfExists;
+     }
+ 
+     boolean isSkipUpdateIfMissing() {
+       return this.skipUpdateIfMissing;
+     }
+ 
+     boolean doesDocumentExist(BytesRef indexedDocId) throws IOException {
+       assert null != indexedDocId;
+ 
 -      SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId);
++      // we don't need any fields populated, we just need to know if the doc is in the tlog...
++      SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId, null,
++                                                                               Collections.<String>emptySet(), false);
+       if (oldDoc == RealTimeGetComponent.DELETED) {
+         return false;
+       }
+       if (oldDoc != null) {
+         return true;
+       }
+ 
+       // need to look up in index now...
+       RefCounted<SolrIndexSearcher> newestSearcher = core.getRealtimeSearcher();
+       try {
+         SolrIndexSearcher searcher = newestSearcher.get();
+         return searcher.lookupId(indexedDocId) >= 0L;
+       } catch (IOException e) {
+         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document from index", e);
+       } finally {
+         if (newestSearcher != null) {
+           newestSearcher.decref();
+         }
+       }
+     }
+ 
+     boolean isLeader(UpdateCommand cmd) {
+       if ((cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0) {
+         return false;
+       }
+       if (phase == DistributedUpdateProcessor.DistribPhase.FROMLEADER) {
+         return false;
+       }
+       return distribProc.isLeader(cmd);
+     }
+ 
+     @Override
+     public void processAdd(AddUpdateCommand cmd) throws IOException {
+       BytesRef indexedDocId = cmd.getIndexedId();
+ 
+       boolean isUpdate = AtomicUpdateDocumentMerger.isAtomicUpdate(cmd);
+ 
+       // boolean existsByLookup = (RealTimeGetComponent.getInputDocument(core, indexedDocId) != null);
+       // if (docExists != existsByLookup) {
+       //   log.error("Found docExists {} but existsByLookup {} for doc {}", docExists, existsByLookup, indexedDocId.utf8ToString());
+       // }
+ 
+       if (log.isDebugEnabled()) {
+         log.debug("Document ID {} ... exists already? {} ... isAtomicUpdate? {} ... isLeader? {}",
+                   indexedDocId.utf8ToString(), doesDocumentExist(indexedDocId), isUpdate, isLeader(cmd));
+       }
+ 
+       if (skipInsertIfExists && !isUpdate && isLeader(cmd) && doesDocumentExist(indexedDocId)) {
+         if (log.isDebugEnabled()) {
+           log.debug("Skipping insert for pre-existing document ID {}", indexedDocId.utf8ToString());
+         }
+         return;
+       }
+ 
+       if (skipUpdateIfMissing && isUpdate && isLeader(cmd) && !doesDocumentExist(indexedDocId)) {
+         if (log.isDebugEnabled()) {
+           log.debug("Skipping update to non-existent document ID {}", indexedDocId.utf8ToString());
+         }
+         return;
+       }
+ 
+       if (log.isDebugEnabled()) {
+         log.debug("Passing on document ID {}", indexedDocId.utf8ToString());
+       }
+ 
+       super.processAdd(cmd);
+     }
+   }
+ }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/test/org/apache/solr/search/TestRecovery.java
----------------------------------------------------------------------
diff --cc solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 2782ef9,15aed5d..29efa52
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@@ -19,12 -19,15 +19,20 @@@ package org.apache.solr.search
  
  import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
  
+ import com.codahale.metrics.Gauge;
+ import com.codahale.metrics.Meter;
+ import com.codahale.metrics.Metric;
+ import com.codahale.metrics.MetricRegistry;
+ import org.apache.solr.metrics.SolrMetricManager;
  import org.noggit.ObjectBuilder;
++
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
++
+ import org.apache.lucene.util.TestUtil;
  import org.apache.solr.SolrTestCaseJ4;
  import org.apache.solr.request.SolrQueryRequest;
 +import org.apache.solr.schema.IndexSchema;
  import org.apache.solr.update.DirectUpdateHandler2;
  import org.apache.solr.update.UpdateLog;
  import org.apache.solr.update.UpdateHandler;
@@@ -83,22 -78,14 +91,15 @@@ public class TestRecovery extends SolrT
      }
    }
  
- 
-   // since we make up fake versions in these tests, we can get messed up by a DBQ with a real version
-   // since Solr can think following updates were reordered.
-   @Override
-   public void clearIndex() {
-     try {
-       deleteByQueryAndGetVersion("*:*", params("_version_", Long.toString(-Long.MAX_VALUE), DISTRIB_UPDATE_PARAM,FROM_LEADER));
-     } catch (Exception e) {
-       throw new RuntimeException(e);
-     }
+   private Map<String, Metric> getMetrics() {
+     SolrMetricManager manager = h.getCoreContainer().getMetricManager();
+     MetricRegistry registry = manager.registry(h.getCore().getCoreMetricManager().getRegistryName());
+     return registry.getMetrics();
    }
  
- 
    @Test
    public void testLogReplay() throws Exception {
 +    
      try {
  
        DirectUpdateHandler2.commitOnClose = false;
@@@ -155,6 -154,9 +170,9 @@@
  
        assertJQ(req("q","*:*") ,"/response/numFound==3");
  
 -      assertEquals(5L, replayDocs.getCount() - initialOps);
++      assertEquals(7L, replayDocs.getCount() - initialOps);
+       assertEquals(UpdateLog.State.ACTIVE.ordinal(), state.getValue().intValue());
+ 
        // make sure we can still access versions after recovery
        assertJQ(req("qt","/get", "getVersions",""+versions.size()) ,"/versions==" + versions);
  
@@@ -1119,133 -1258,6 +1276,133 @@@
      }
    }
  
 +  @Test
 +  public void testLogReplayWithInPlaceUpdatesAndDeletes() throws Exception {
 +
 +    try {
 +
 +      DirectUpdateHandler2.commitOnClose = false;
 +      final Semaphore logReplay = new Semaphore(0);
 +      final Semaphore logReplayFinish = new Semaphore(0);
 +
 +      UpdateLog.testing_logReplayHook = () -> {
 +        try {
 +          assertTrue(logReplay.tryAcquire(timeout, TimeUnit.SECONDS));
 +        } catch (Exception e) {
 +          throw new RuntimeException(e);
 +        }
 +      };
 +
 +      UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
 +
 +
 +      clearIndex();
 +      assertU(commit());
 +
 +      Deque<Long> versions = new ArrayDeque<>();
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A1"), null));
 +      
 +      // DBQ of updated document using id
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A2", "val_i_dvo", "1"), null));
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A2", "val_i_dvo", map("set", 2)), null)); // in-place update
 +      versions.addFirst(deleteByQueryAndGetVersion("id:A2", null));
 +
 +      // DBQ of updated document using updated value
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A3", "val_i_dvo", "101"), null));
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A3", "val_i_dvo", map("set", 102)), null)); // in-place update
 +      versions.addFirst(deleteByQueryAndGetVersion("val_i_dvo:102", null));
 +
 +      // DBQ using an intermediate update value (shouldn't delete anything)
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A4", "val_i_dvo", "200"), null));
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A4", "val_i_dvo", map("inc", "1")), null)); // in-place update
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A4", "val_i_dvo", map("inc", "1")), null)); // in-place update
 +      versions.addFirst(deleteByQueryAndGetVersion("val_i_dvo:201", null));
 +
 +      // DBI of updated document
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A5", "val_i_dvo", "300"), null));
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A5", "val_i_dvo", map("inc", "1")), null)); // in-place update
 +      versions.addFirst(addAndGetVersion(sdoc("id", "A5", "val_i_dvo", map("inc", "1")), null)); // in-place update
 +      versions.addFirst(deleteAndGetVersion("A5", null));
 +      
 +      assertJQ(req("q","*:*"),"/response/numFound==0");
 +      
 +
 +      assertJQ(req("qt","/get", "getVersions",""+versions.size()) ,"/versions==" + versions);
 +
 +      h.close();
 +      createCore();
 +
 +      // Solr should kick this off now
 +      // h.getCore().getUpdateHandler().getUpdateLog().recoverFromLog();
 +
 +      // verify that previous close didn't do a commit
 +      // recovery should be blocked by our hook
 +      assertJQ(req("q","*:*") ,"/response/numFound==0");
 +
 +      // make sure we can still access versions after a restart
 +      assertJQ(req("qt","/get", "getVersions",""+versions.size()),"/versions==" + versions);
 +
 +      // unblock recovery
 +      logReplay.release(1000);
 +
 +      // make sure we can still access versions during recovery
 +      assertJQ(req("qt","/get", "getVersions",""+versions.size()),"/versions==" + versions);
 +
 +      // wait until recovery has finished
 +      assertTrue(logReplayFinish.tryAcquire(timeout, TimeUnit.SECONDS));
 +      assertJQ(req("q","val_i_dvo:202") ,"/response/numFound==1"); // assert that in-place update is retained
 +
-       assertJQ(req("q","*:*") ,"/response/numFound==2");     // nocommit: Fails due to SOLR-9941
++      assertJQ(req("q","*:*") ,"/response/numFound==2");
 +      assertJQ(req("q","id:A2") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A3") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A4") ,"/response/numFound==1");
 +      assertJQ(req("q","id:A5") ,"/response/numFound==0");
 +
 +      // make sure we can still access versions after recovery
 +      assertJQ(req("qt","/get", "getVersions",""+versions.size()) ,"/versions==" + versions);
 +
 +      assertU(adoc("id","A10"));
 +
 +      h.close();
 +      createCore();
 +      // Solr should kick this off now
 +      // h.getCore().getUpdateHandler().getUpdateLog().recoverFromLog();
 +
 +      // wait until recovery has finished
 +      assertTrue(logReplayFinish.tryAcquire(timeout, TimeUnit.SECONDS));
 +      assertJQ(req("q","*:*") ,"/response/numFound==3");
 +      assertJQ(req("q","id:A2") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A3") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A4") ,"/response/numFound==1");
 +      assertJQ(req("q","id:A5") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A10"),"/response/numFound==1");
 +      
 +      // no updates, so insure that recovery does not run
 +      h.close();
 +      int permits = logReplay.availablePermits();
 +      createCore();
 +      // Solr should kick this off now
 +      // h.getCore().getUpdateHandler().getUpdateLog().recoverFromLog();
 +
 +      assertJQ(req("q","*:*") ,"/response/numFound==3");
 +      assertJQ(req("q","val_i_dvo:202") ,"/response/numFound==1"); // assert that in-place update is retained
 +      assertJQ(req("q","id:A2") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A3") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A4") ,"/response/numFound==1");
 +      assertJQ(req("q","id:A5") ,"/response/numFound==0");
 +      assertJQ(req("q","id:A10"),"/response/numFound==1");
 +      Thread.sleep(100);
 +      assertEquals(permits, logReplay.availablePermits()); // no updates, so insure that recovery didn't run
 +
 +      assertEquals(UpdateLog.State.ACTIVE, h.getCore().getUpdateHandler().getUpdateLog().getState());
 +
 +    } finally {
 +      DirectUpdateHandler2.commitOnClose = true;
 +      UpdateLog.testing_logReplayHook = null;
 +      UpdateLog.testing_logReplayFinishHook = null;
 +    }
 +
 +  }
  
    // NOTE: replacement must currently be same size
    private static void findReplace(byte[] from, byte[] to, byte[] data) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027a92a4/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
----------------------------------------------------------------------


[06/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9944: Map the nodes function name to the GatherNodesStream

Posted by ho...@apache.org.
SOLR-9944: Map the nodes function name to the GatherNodesStream


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

Branch: refs/heads/jira/solr-5944
Commit: aae4217abc09163837597bf761f21d8019091216
Parents: d817fd4
Author: Joel Bernstein <jb...@apache.org>
Authored: Sat Jan 7 22:17:17 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Sat Jan 7 22:17:44 2017 -0500

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/handler/GraphHandler.java      | 1 +
 solr/core/src/java/org/apache/solr/handler/StreamHandler.java     | 1 +
 .../apache/solr/client/solrj/io/graph/GraphExpressionTest.java    | 3 ++-
 3 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aae4217a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
index 5e5934f..3b52154 100644
--- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
@@ -117,6 +117,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
         .withFunctionName("topic", TopicStream.class)
         .withFunctionName("shortestPath", ShortestPathStream.class)
         .withFunctionName("gatherNodes", GatherNodesStream.class)
+        .withFunctionName("nodes", GatherNodesStream.class)
         .withFunctionName("sort", SortStream.class)
         .withFunctionName("scoreNodes", ScoreNodesStream.class)
         .withFunctionName("random", RandomStream.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aae4217a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 4b319f9..98486b8 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -142,6 +142,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("daemon", DaemonStream.class)
       .withFunctionName("shortestPath", ShortestPathStream.class)
       .withFunctionName("gatherNodes", GatherNodesStream.class)
+      .withFunctionName("nodes", GatherNodesStream.class)
       .withFunctionName("select", SelectStream.class)
       .withFunctionName("scoreNodes", ScoreNodesStream.class)
       .withFunctionName("model", ModelStream.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aae4217a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
index d6fc514..cf07058 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
@@ -270,6 +270,7 @@ public class GraphExpressionTest extends SolrCloudTestCase {
     StreamFactory factory = new StreamFactory()
         .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
         .withFunctionName("gatherNodes", GatherNodesStream.class)
+        .withFunctionName("nodes", GatherNodesStream.class)
         .withFunctionName("search", CloudSolrStream.class)
         .withFunctionName("count", CountMetric.class)
         .withFunctionName("avg", MeanMetric.class)
@@ -277,7 +278,7 @@ public class GraphExpressionTest extends SolrCloudTestCase {
         .withFunctionName("min", MinMetric.class)
         .withFunctionName("max", MaxMetric.class);
 
-    String expr = "gatherNodes(collection1, " +
+    String expr = "nodes(collection1, " +
         "walk=\"product1->product_s\"," +
         "gather=\"basket_s\")";
 


[19/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9932: add TestSolrCoreParser class

Posted by ho...@apache.org.
SOLR-9932: add TestSolrCoreParser class


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

Branch: refs/heads/jira/solr-5944
Commit: 5e9f9279cccb49050c04f75e4551f64151d1001c
Parents: d7beb0f
Author: Christine Poerschke <cp...@apache.org>
Authored: Mon Jan 9 13:05:43 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Jan 9 13:05:43 2017 +0000

----------------------------------------------------------------------
 .../ApacheLuceneSolrNearQueryBuilder.java       |  51 +++++++++
 .../apache/solr/search/TestSolrCoreParser.java  | 110 +++++++++++++++++++
 2 files changed, 161 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e9f9279/solr/core/src/test/org/apache/solr/search/ApacheLuceneSolrNearQueryBuilder.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/ApacheLuceneSolrNearQueryBuilder.java b/solr/core/src/test/org/apache/solr/search/ApacheLuceneSolrNearQueryBuilder.java
new file mode 100644
index 0000000..135ec45
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/ApacheLuceneSolrNearQueryBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queryparser.xml.DOMUtils;
+import org.apache.lucene.queryparser.xml.ParserException;
+import org.apache.lucene.queryparser.xml.QueryBuilder;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.solr.request.SolrQueryRequest;
+import org.w3c.dom.Element;
+
+public class ApacheLuceneSolrNearQueryBuilder extends SolrQueryBuilder {
+
+  public ApacheLuceneSolrNearQueryBuilder(String defaultField, Analyzer analyzer,
+      SolrQueryRequest req, QueryBuilder queryFactory) {
+    super(defaultField, analyzer, req, queryFactory);
+  }
+
+  @Override
+  public Query getQuery(Element e) throws ParserException {
+    final String fieldName = DOMUtils.getAttributeWithInheritanceOrFail(e, "fieldName");
+    final SpanQuery[] spanQueries = new SpanQuery[]{
+        new SpanTermQuery(new Term(fieldName, "Apache")),
+        new SpanTermQuery(new Term(fieldName, "Lucene")),
+        new SpanTermQuery(new Term(fieldName, "Solr"))
+    };
+    final int slop = 42;
+    final boolean inOrder = false;
+    return new SpanNearQuery(spanQueries, slop, inOrder);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5e9f9279/solr/core/src/test/org/apache/solr/search/TestSolrCoreParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrCoreParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrCoreParser.java
new file mode 100644
index 0000000..3ef96c3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrCoreParser.java
@@ -0,0 +1,110 @@
+/*
+ * 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.search;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.queryparser.xml.CoreParser;
+import org.apache.lucene.queryparser.xml.ParserException;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MatchNoDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+
+public class TestSolrCoreParser extends LuceneTestCase {
+
+  private SolrCoreParser solrCoreParser;
+
+  private CoreParser solrCoreParser() {
+    if (solrCoreParser == null) {
+      final String defaultField = "contents";
+      final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET);
+      final SolrQueryRequest req = null;
+      solrCoreParser = new SolrCoreParser(defaultField, analyzer, req);
+      {
+        final NamedList<String> args = new NamedList<>();
+        args.add("HelloQuery", HelloQueryBuilder.class.getCanonicalName());
+        args.add("GoodbyeQuery", GoodbyeQueryBuilder.class.getCanonicalName());
+        args.add("HandyQuery", HandyQueryBuilder.class.getCanonicalName());
+        args.add("ApacheLuceneSolr", ApacheLuceneSolrNearQueryBuilder.class.getCanonicalName());
+        solrCoreParser.init(args);
+      }
+    }
+    return solrCoreParser;
+  }
+
+  private Query parseXmlString(String xml) throws IOException, ParserException {
+    final byte[] xmlBytes = xml.getBytes(StandardCharsets.UTF_8);
+    final InputStream xmlStream = new ByteArrayInputStream(xmlBytes);
+    return solrCoreParser().parse(xmlStream);
+  }
+
+  private Query parseHandyQuery(String lhsXml, String rhsXml) throws IOException, ParserException {
+    final String xml = "<HandyQuery>"
+        + "<Left>" + lhsXml + "</Left>"
+        + "<Right>" + rhsXml + "</Right>"
+        + "</HandyQuery>";
+    return parseXmlString(xml);
+  }
+
+  public void testHello() throws IOException, ParserException {
+    final Query query = parseXmlString("<HelloQuery/>");
+    assertTrue(query instanceof MatchAllDocsQuery);
+  }
+
+  public void testGoodbye() throws IOException, ParserException {
+    final Query query = parseXmlString("<GoodbyeQuery/>");
+    assertTrue(query instanceof MatchNoDocsQuery);
+  }
+
+  public void testApacheLuceneSolr() throws IOException, ParserException {
+    final String fieldName = "contents";
+    final Query query = parseXmlString("<ApacheLuceneSolr fieldName='"+fieldName+"'/>");
+    assertTrue(query instanceof SpanNearQuery);
+    final SpanNearQuery snq = (SpanNearQuery)query;
+    assertEquals(fieldName, snq.getField());
+    assertEquals(42, snq.getSlop());
+    assertFalse(snq.isInOrder());
+    assertEquals(3, snq.getClauses().length);
+    assertTrue(snq.getClauses()[0] instanceof SpanTermQuery);
+    assertTrue(snq.getClauses()[1] instanceof SpanTermQuery);
+    assertTrue(snq.getClauses()[2] instanceof SpanTermQuery);
+  }
+
+  public void testHandyQuery() throws IOException, ParserException {
+    final String lhsXml = "<HelloQuery/>";
+    final String rhsXml = "<GoodbyeQuery/>";
+    final Query query = parseHandyQuery(lhsXml, rhsXml);
+    assertTrue(query instanceof BooleanQuery);
+    final BooleanQuery bq = (BooleanQuery)query;
+    assertEquals(2, bq.clauses().size());
+    assertTrue(bq.clauses().get(0).getQuery() instanceof MatchAllDocsQuery);
+    assertTrue(bq.clauses().get(1).getQuery() instanceof MatchNoDocsQuery);
+  }
+
+}


[35/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9947 Clean up metrics and SolrInfoMBean categories. Add a hierarhical view of SolrInfoMBeans in JMX.

Posted by ho...@apache.org.
SOLR-9947 Clean up metrics and SolrInfoMBean categories. Add a hierarhical view of
SolrInfoMBeans in JMX.


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

Branch: refs/heads/jira/solr-5944
Commit: 6c6c077435bcc5bd3f4520a70a4c678d4b3f7661
Parents: d2664b1
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Jan 12 13:14:12 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Jan 12 13:14:12 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../org/apache/solr/core/CoreContainer.java     |  10 +-
 .../org/apache/solr/core/JmxMonitoredMap.java   |  63 ++++++--
 .../org/apache/solr/core/SolrInfoMBean.java     |   5 +-
 .../apache/solr/handler/CdcrRequestHandler.java |   5 +
 .../apache/solr/handler/PingRequestHandler.java |   5 +
 .../apache/solr/handler/ReplicationHandler.java |   5 +
 .../apache/solr/handler/RequestHandlerBase.java |   2 +-
 .../org/apache/solr/handler/SchemaHandler.java  |   5 +
 .../apache/solr/handler/SolrConfigHandler.java  |   2 +-
 .../solr/handler/UpdateRequestHandler.java      |   5 +
 .../solr/handler/admin/CollectionsHandler.java  |   5 +
 .../solr/handler/admin/ConfigSetsHandler.java   |   5 +
 .../solr/handler/admin/CoreAdminHandler.java    |   5 +
 .../apache/solr/handler/admin/InfoHandler.java  |   5 +
 .../solr/handler/admin/LoggingHandler.java      |   6 +
 .../solr/handler/admin/LukeRequestHandler.java  |   5 +
 .../solr/handler/admin/MetricsHandler.java      |   5 +
 .../solr/handler/admin/PluginInfoHandler.java   |   5 +
 .../handler/admin/PropertiesRequestHandler.java |   5 +
 .../solr/handler/admin/SecurityConfHandler.java |   5 +
 .../admin/SegmentsInfoRequestHandler.java       |   5 +
 .../handler/admin/ShowFileRequestHandler.java   |   4 +
 .../handler/admin/SolrInfoMBeanHandler.java     |   5 +
 .../solr/handler/admin/SystemInfoHandler.java   |   7 +-
 .../solr/handler/admin/ThreadDumpHandler.java   |   5 +
 .../handler/admin/ZookeeperInfoHandler.java     |   5 +
 .../solr/handler/component/DebugComponent.java  |   5 +
 .../solr/handler/component/ExpandComponent.java |   5 +
 .../solr/handler/component/FacetComponent.java  |   5 +
 .../handler/component/HighlightComponent.java   |   5 +
 .../component/HttpShardHandlerFactory.java      |   2 +-
 .../component/MoreLikeThisComponent.java        |   7 +-
 .../solr/handler/component/QueryComponent.java  |   5 +
 .../handler/component/RealTimeGetComponent.java |   5 +
 .../handler/component/SpellCheckComponent.java  |   5 +
 .../handler/component/TermVectorComponent.java  |   5 +
 .../solr/handler/component/TermsComponent.java  |   5 +
 .../solr/highlight/HighlightingPluginBase.java  |   2 +-
 .../solr/metrics/SolrCoreMetricManager.java     |   3 +-
 .../metrics/reporters/JmxObjectNameFactory.java | 155 +++++++++++++++++++
 .../solr/metrics/reporters/SolrJmxReporter.java | 105 -------------
 .../apache/solr/search/facet/FacetModule.java   |   5 +
 .../solr/security/PKIAuthenticationPlugin.java  |   6 +
 .../solr/update/DirectUpdateHandler2.java       |   5 -
 .../org/apache/solr/update/UpdateHandler.java   |   5 +
 .../java/org/apache/solr/update/UpdateLog.java  |  28 ++--
 .../apache/solr/update/UpdateShardHandler.java  |   2 +-
 .../test-files/solr/solr-gangliareporter.xml    |   2 +-
 .../test-files/solr/solr-graphitereporter.xml   |   2 +-
 .../src/test-files/solr/solr-slf4jreporter.xml  |   4 +-
 .../solr/cloud/BasicDistributedZkTest.java      |   2 +-
 .../solr/handler/admin/MBeansHandlerTest.java   |   4 +-
 .../solr/handler/admin/MetricsHandlerTest.java  |  18 +--
 .../metrics/SolrMetricsIntegrationTest.java     |   2 +-
 .../reporters/SolrGangliaReporterTest.java      |   2 +-
 .../reporters/SolrGraphiteReporterTest.java     |   2 +-
 .../org/apache/solr/search/TestRecovery.java    |   2 +-
 .../client/solrj/impl/CloudSolrClientTest.java  |   6 +-
 59 files changed, 438 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0cf50d4..b6055fe 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -242,6 +242,9 @@ New Features
 
 * SOLR-9886: Add a 'enable' flag to caches to enable/disable them (Pushkar Raste, noble)
 
+* SOLR-9947: Clean up some SolrInfoMBean categories. Add an alternative hierarchical view in JMX
+  for SolrInfoMBeans, which uses similar conventions to SolrJmxReporter. (ab)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 8a72617..3c4ed56 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -466,7 +466,7 @@ public class CoreContainer {
     coreContainerWorkExecutor = MetricUtils.instrumentedExecutorService(
         coreContainerWorkExecutor,
         metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node)),
-        SolrMetricManager.mkName("coreContainerWorkExecutor", "threadPool"));
+        SolrMetricManager.mkName("coreContainerWorkExecutor", SolrInfoMBean.Category.CONTAINER.toString(), "threadPool"));
 
     shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader);
     if (shardHandlerFactory instanceof SolrMetricProducer) {
@@ -518,11 +518,11 @@ public class CoreContainer {
     Gauge<Integer> unloadedCores = () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size();
 
     metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
-        loadedCores, true, "loaded", "cores");
+        loadedCores, true, "loaded", SolrInfoMBean.Category.CONTAINER.toString(), "cores");
     metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
-        lazyCores, true, "lazy", "cores");
+        lazyCores, true, "lazy",SolrInfoMBean.Category.CONTAINER.toString(), "cores");
     metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
-        unloadedCores, true, "unloaded", "cores");
+        unloadedCores, true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores");
 
     // setup executor to load cores in parallel
     ExecutorService coreLoadExecutor = MetricUtils.instrumentedExecutorService(
@@ -530,7 +530,7 @@ public class CoreContainer {
             cfg.getCoreLoadThreadCount(isZooKeeperAware()),
             new DefaultSolrThreadFactory("coreLoadExecutor")),
         metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node)),
-        SolrMetricManager.mkName("coreLoadExecutor", "threadPool"));
+        SolrMetricManager.mkName("coreLoadExecutor",SolrInfoMBean.Category.CONTAINER.toString(), "threadPool"));
     final List<Future<SolrCore>> futures = new ArrayList<>();
     try {
       List<CoreDescriptor> cds = coresLocator.discover(this);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
index a5a27dc..b2a5c79 100644
--- a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
+++ b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
@@ -53,6 +53,8 @@ import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrConfig.JmxConfiguration;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.apache.solr.metrics.reporters.JmxObjectNameFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,15 +77,21 @@ public class JmxMonitoredMap<K, V> extends
         ConcurrentHashMap<String, SolrInfoMBean> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private static final String REPORTER_NAME = "_jmx_";
+
   // set to true to use cached statistics NamedLists between getMBeanInfo calls to work
   // around over calling getStatistics on MBeanInfos when iterating over all attributes (SOLR-6586)
-  private boolean useCachedStatsBetweenGetMBeanInfoCalls = Boolean.getBoolean("useCachedStatsBetweenGetMBeanInfoCalls");
+  private final boolean useCachedStatsBetweenGetMBeanInfoCalls = Boolean.getBoolean("useCachedStatsBetweenGetMBeanInfoCalls");
   
-  private MBeanServer server = null;
+  private final MBeanServer server;
+
+  private final String jmxRootName;
 
-  private String jmxRootName;
+  private final String coreHashCode;
 
-  private String coreHashCode;
+  private final JmxObjectNameFactory nameFactory;
+
+  private final String registryName;
 
   public JmxMonitoredMap(String coreName, String coreHashCode,
                          final JmxConfiguration jmxConfig) {
@@ -108,6 +116,9 @@ public class JmxMonitoredMap<K, V> extends
       }
 
       if (servers == null || servers.isEmpty()) {
+        server = null;
+        registryName = null;
+        nameFactory = null;
         log.debug("No JMX servers found, not exposing Solr information with JMX.");
         return;
       }
@@ -115,20 +126,23 @@ public class JmxMonitoredMap<K, V> extends
       log.info("JMX monitoring is enabled. Adding Solr mbeans to JMX Server: "
                + server);
     } else {
+      MBeanServer newServer = null;
       try {
         // Create a new MBeanServer with the given serviceUrl
-        server = MBeanServerFactory.newMBeanServer();
+        newServer = MBeanServerFactory.newMBeanServer();
         JMXConnectorServer connector = JMXConnectorServerFactory
                 .newJMXConnectorServer(new JMXServiceURL(jmxConfig.serviceUrl),
-                        null, server);
+                        null, newServer);
         connector.start();
         log.info("JMX monitoring is enabled at " + jmxConfig.serviceUrl);
       } catch (Exception e) {
         // Release the reference
-        server = null;
         throw new RuntimeException("Could not start JMX monitoring ", e);
       }
+      server = newServer;
     }
+    registryName = SolrCoreMetricManager.createRegistryName(null, coreName);
+    nameFactory = new JmxObjectNameFactory(REPORTER_NAME + coreHashCode, registryName);
   }
 
   /**
@@ -138,12 +152,12 @@ public class JmxMonitoredMap<K, V> extends
   @Override
   public void clear() {
     if (server != null) {
-      QueryExp exp = Query.eq(Query.attr("coreHashCode"), Query.value(coreHashCode));
+      QueryExp exp = Query.or(Query.eq(Query.attr("coreHashCode"), Query.value(coreHashCode)),
+                            Query.eq(Query.attr("reporter"), Query.value(REPORTER_NAME + coreHashCode)));
       
       Set<ObjectName> objectNames = null;
       try {
-        ObjectName instance = new ObjectName(jmxRootName + ":*");
-        objectNames = server.queryNames(instance, exp);
+        objectNames = server.queryNames(null, exp);
       } catch (Exception e) {
         log.warn("Exception querying for mbeans", e);
       }
@@ -175,19 +189,39 @@ public class JmxMonitoredMap<K, V> extends
   public SolrInfoMBean put(String key, SolrInfoMBean infoBean) {
     if (server != null && infoBean != null) {
       try {
+        // back-compat name
         ObjectName name = getObjectName(key, infoBean);
         if (server.isRegistered(name))
           server.unregisterMBean(name);
         SolrDynamicMBean mbean = new SolrDynamicMBean(coreHashCode, infoBean, useCachedStatsBetweenGetMBeanInfoCalls);
         server.registerMBean(mbean, name);
+        // now register it also under new name
+        String beanName = createBeanName(infoBean, key);
+        name = nameFactory.createName(null, registryName, beanName);
+        if (server.isRegistered(name))
+          server.unregisterMBean(name);
+        server.registerMBean(mbean, name);
       } catch (Exception e) {
-        log.warn( "Failed to register info bean: " + key, e);
+        log.warn( "Failed to register info bean: key=" + key + ", infoBean=" + infoBean, e);
       }
     }
 
     return super.put(key, infoBean);
   }
 
+  private String createBeanName(SolrInfoMBean infoBean, String key) {
+    if (infoBean.getCategory() == null) {
+      throw new IllegalArgumentException("SolrInfoMBean.category must never be null: " + infoBean);
+    }
+    StringBuilder sb = new StringBuilder();
+    sb.append(infoBean.getCategory().toString());
+    sb.append('.');
+    sb.append(key);
+    sb.append('.');
+    sb.append(infoBean.getName());
+    return sb.toString();
+  }
+
   /**
    * Removes the SolrInfoMBean object at the given key and unregisters it from
    * MBeanServer
@@ -212,10 +246,17 @@ public class JmxMonitoredMap<K, V> extends
       return;
 
     try {
+      // remove legacy name
       ObjectName name = getObjectName(key, infoBean);
       if (server.isRegistered(name) && coreHashCode.equals(server.getAttribute(name, "coreHashCode"))) {
         server.unregisterMBean(name);
       }
+      // remove new name
+      String beanName = createBeanName(infoBean, key);
+      name = nameFactory.createName(null, registryName, beanName);
+      if (server.isRegistered(name)) {
+        server.unregisterMBean(name);
+      }
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "Failed to unregister info bean: " + key, e);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
index 04c8395..bf77db4 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -30,9 +30,10 @@ import org.apache.solr.common.util.NamedList;
 public interface SolrInfoMBean {
 
   /**
-   * Category of {@link SolrCore} component.
+   * Category of Solr component.
    */
-  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, OTHER }
+  enum Category { CONTAINER, ADMIN, CORE, QUERY, UPDATE, CACHE, HIGHLIGHTER, QUERYPARSER, SPELLCHECKER,
+    SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, OTHER }
 
   /**
    * Top-level group of beans for a subsystem.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
index f706637..fcc4bbe 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
@@ -806,6 +806,11 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
     return "Manage Cross Data Center Replication";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.REPLICATION;
+  }
+
   /**
    * A thread subclass for executing a single
    * {@link org.apache.solr.handler.CdcrParams.CdcrAction#SHARDCHECKPOINT} action.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
index 4b72e0f..04b930a 100644
--- a/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
@@ -328,4 +328,9 @@ public class PingRequestHandler extends RequestHandlerBase implements SolrCoreAw
   public String getDescription() {
     return "Reports application health to a load-balancer";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index edf5e94..b875144 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -814,6 +814,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   }
 
   @Override
+  public Category getCategory() {
+    return Category.REPLICATION;
+  }
+
+  @Override
   public String getDescription() {
     return "ReplicationHandler provides replication of index and configuration files from Master to Slaves";
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 1fd6c30..b70c096 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -227,7 +227,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   
   @Override
   public Category getCategory() {
-    return Category.QUERYHANDLER;
+    return Category.QUERY;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
index 35e463b..9c2d45c 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -252,6 +252,11 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
   }
 
   @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
+  @Override
   public void inform(SolrCore core) {
     isImmutableConfigSet = SolrConfigHandler.getImmutable(core);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index b6cb596..1c584b1 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -708,7 +708,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
 
   @Override
   public Category getCategory() {
-    return Category.OTHER;
+    return Category.ADMIN;
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
index 7c97331..6628368 100644
--- a/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java
@@ -170,6 +170,11 @@ public class UpdateRequestHandler extends ContentStreamHandlerBase implements Pe
     return "Add documents using XML (with XSLT), CSV, JSON, or javabin";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.UPDATE;
+  }
+
   public static final String DOC_PATH = "/update/json/docs";
   public static final String JSON_PATH = "/update/json";
   public static final String CSV_PATH = "/update/csv";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index e683e96..148d73c 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -335,6 +335,11 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     return "Manage SolrCloud Collections";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   public static final String SYSTEM_COLL = ".system";
 
   private static void createSysConfigSet(CoreContainer coreContainer) throws KeeperException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
index ab2b172..f3a8dd2 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
@@ -161,6 +161,11 @@ public class ConfigSetsHandler extends RequestHandlerBase {
     return "Manage SolrCloud ConfigSets";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   enum ConfigSetOperation {
     CREATE_OP(CREATE) {
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index 05a1671..a415d8a 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -275,6 +275,11 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   }
 
   @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
+  @Override
   public Name getPermissionName(AuthorizationContext ctx) {
     String action = ctx.getParams().get(CoreAdminParams.ACTION);
     if (action == null) return CORE_READ_PERM;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
index 9c437ab..8fdac21 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/InfoHandler.java
@@ -103,6 +103,11 @@ public class InfoHandler extends RequestHandlerBase {
     return "System Information";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   protected PropertiesRequestHandler getPropertiesHandler() {
     return propertiesHandler;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/LoggingHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/LoggingHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/LoggingHandler.java
index a561594..b10aed1 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/LoggingHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/LoggingHandler.java
@@ -156,4 +156,10 @@ public class LoggingHandler extends RequestHandlerBase implements SolrCoreAware
   public String getDescription() {
     return "Logging Handler";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
index d7dedf1..50f46ef 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
@@ -703,6 +703,11 @@ public class LukeRequestHandler extends RequestHandlerBase
   }
 
   @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
+  @Override
   public URL[] getDocs() {
     try {
       return new URL[] { new URL("http://wiki.apache.org/solr/LukeRequestHandler") };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
index 428a72b..0c87875 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -155,6 +155,11 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
     return "A handler to return all the metrics gathered by Solr";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   enum MetricType {
     histogram(Histogram.class),
     meter(Meter.class),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/PluginInfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/PluginInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/PluginInfoHandler.java
index 6bc34ed..a096e79 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/PluginInfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/PluginInfoHandler.java
@@ -90,4 +90,9 @@ public class PluginInfoHandler extends RequestHandlerBase
   public String getDescription() {
     return "Registry";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
index bae61a7..c16cded 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/PropertiesRequestHandler.java
@@ -55,4 +55,9 @@ public class PropertiesRequestHandler extends RequestHandlerBase
   public String getDescription() {
     return "Get System Properties";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
index 88e4b01..1b81722 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
@@ -163,6 +163,11 @@ public abstract class SecurityConfHandler extends RequestHandlerBase implements
     return "Edit or read security configuration";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   /**
    * Gets security.json from source
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
index df1bbd8..1baf25a 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SegmentsInfoRequestHandler.java
@@ -117,4 +117,9 @@ public class SegmentsInfoRequestHandler extends RequestHandlerBase {
   public String getDescription() {
     return "Lucene segments info.";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
index ea08438..805a690 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
@@ -364,4 +364,8 @@ public class ShowFileRequestHandler extends RequestHandlerBase
   public String getDescription() {
     return "Admin Config File -- view or update config files directly";
   }
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/SolrInfoMBeanHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SolrInfoMBeanHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SolrInfoMBeanHandler.java
index 4d3c2b5..f5f28c5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SolrInfoMBeanHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SolrInfoMBeanHandler.java
@@ -298,4 +298,9 @@ public class SolrInfoMBeanHandler extends RequestHandlerBase {
   public String getDescription() {
     return "Get Info (and statistics) for registered SolrInfoMBeans";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
index a873c09..d031d69 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
@@ -408,7 +408,12 @@ public class SystemInfoHandler extends RequestHandlerBase
   public String getDescription() {
     return "Get System Info";
   }
-  
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   private static final long ONE_KB = 1024;
   private static final long ONE_MB = ONE_KB * ONE_KB;
   private static final long ONE_GB = ONE_KB * ONE_MB;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java
index 3ddc9db..f0e3970 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java
@@ -130,4 +130,9 @@ public class ThreadDumpHandler extends RequestHandlerBase
   public String getDescription() {
     return "Thread Dump";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
index e2ce63d..0616ac8 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
@@ -97,6 +97,11 @@ public final class ZookeeperInfoHandler extends RequestHandlerBase {
     return "Fetch Zookeeper contents";
   }
 
+  @Override
+  public Category getCategory() {
+    return Category.ADMIN;
+  }
+
   /**
    * Enumeration of ways to filter collections on the graph panel.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java b/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java
index 42d21e0..be21733 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/DebugComponent.java
@@ -389,6 +389,11 @@ public class DebugComponent extends SearchComponent
   }
 
   @Override
+  public Category getCategory() {
+    return Category.OTHER;
+  }
+
+  @Override
   public URL[] getDocs() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index a5cbee2..c06aab0 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -746,6 +746,11 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
   }
 
   @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
+
+  @Override
   public URL[] getDocs() {
     try {
       return new URL[]{

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
index 90608c0..1cc05ab 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
@@ -1213,6 +1213,11 @@ public class FacetComponent extends SearchComponent {
   }
 
   @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
+
+  @Override
   public URL[] getDocs() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java b/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
index 00ed4ef..d147be2 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
@@ -273,6 +273,11 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
   public String getDescription() {
     return "Highlighting";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.HIGHLIGHTER;
+  }
   
   @Override
   public URL[] getDocs() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index be6e763..87d5c3d 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -377,7 +377,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
 
   @Override
   public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    String expandedScope = SolrMetricManager.mkName(scope, SolrInfoMBean.Category.HTTP.name());
+    String expandedScope = SolrMetricManager.mkName(scope, SolrInfoMBean.Category.QUERY.name());
     clientConnectionManager.initializeMetrics(manager, registry, expandedScope);
     httpRequestExecutor.initializeMetrics(manager, registry, expandedScope);
     commExecutor = MetricUtils.instrumentedExecutorService(commExecutor,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java b/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
index 7cf6d39..893cdf3 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
@@ -409,7 +409,12 @@ public class MoreLikeThisComponent extends SearchComponent {
   public String getDescription() {
     return "More Like This";
   }
-  
+
+  @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
+
   @Override
   public URL[] getDocs() {
     return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
index 9bd5efb..88ff731 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
@@ -1370,6 +1370,11 @@ public class QueryComponent extends SearchComponent
   }
 
   @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
+
+  @Override
   public URL[] getDocs() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index 39e5b8a..8ce7301 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -599,6 +599,11 @@ public class RealTimeGetComponent extends SearchComponent
   }
 
   @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
+
+  @Override
   public URL[] getDocs() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java b/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
index 4966237..a229a85 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
@@ -862,4 +862,9 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
   public String getDescription() {
     return "A Spell Checker component";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.SPELLCHECKER;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java b/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
index 60ceca0..c887277 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
@@ -481,6 +481,11 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
   public String getDescription() {
     return "A Component for working with Term Vectors";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
 }
 
 class FieldOptions {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
index 918efa0..b76537a 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
@@ -584,4 +584,9 @@ public class TermsComponent extends SearchComponent {
   public String getDescription() {
     return "A Component for working with Term Enumerators";
   }
+
+  @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/highlight/HighlightingPluginBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/HighlightingPluginBase.java b/solr/core/src/java/org/apache/solr/highlight/HighlightingPluginBase.java
index c03a961..f60ada8 100644
--- a/solr/core/src/java/org/apache/solr/highlight/HighlightingPluginBase.java
+++ b/solr/core/src/java/org/apache/solr/highlight/HighlightingPluginBase.java
@@ -61,7 +61,7 @@ public abstract class HighlightingPluginBase implements SolrInfoMBean
   @Override
   public Category getCategory()
   {
-    return Category.HIGHLIGHTING;
+    return Category.HIGHLIGHTER;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
index e7ca9c4..3bebcd3 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -127,8 +127,7 @@ public class SolrCoreMetricManager implements Closeable {
     return registryName;
   }
 
-  /* package visibility for tests. */
-  String createRegistryName(String collectionName, String coreName) {
+  public static String createRegistryName(String collectionName, String coreName) {
     if (collectionName == null || (collectionName != null && !coreName.startsWith(collectionName + "_"))) {
       // single core, or unknown naming scheme
       return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, coreName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java b/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
new file mode 100644
index 0000000..4df5257
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
@@ -0,0 +1,155 @@
+/*
+ * 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.metrics.reporters;
+
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import java.util.Arrays;
+
+import com.codahale.metrics.ObjectNameFactory;
+import org.apache.solr.metrics.SolrMetricInfo;
+
+/**
+ * Factory to create MBean names for a given metric.
+ */
+public class JmxObjectNameFactory implements ObjectNameFactory {
+
+  private final String domain;
+  private final String[] subdomains;
+  private final String reporterName;
+  private final String[] props;
+
+  /**
+   * Create ObjectName factory.
+   * @param reporterName name of the reporter
+   * @param domain JMX domain name
+   * @param additionalProperties additional properties as key, value pairs.
+   */
+  public JmxObjectNameFactory(String reporterName, String domain, String... additionalProperties) {
+    this.reporterName = reporterName;
+    this.domain = domain;
+    this.subdomains = domain.split("\\.");
+    if (additionalProperties != null && (additionalProperties.length % 2) != 0) {
+      throw new IllegalArgumentException("additionalProperties length must be even: " + Arrays.toString(additionalProperties));
+    }
+    this.props = additionalProperties;
+  }
+
+  /**
+   * Create a hierarchical name.
+   *
+   * @param type    metric class, eg. "counters", may be null for non-metric MBeans
+   * @param currentDomain  JMX domain
+   * @param name    object name
+   */
+  @Override
+  public ObjectName createName(String type, String currentDomain, String name) {
+    SolrMetricInfo metricInfo = SolrMetricInfo.of(name);
+
+    // It turns out that ObjectName(String) mostly preserves key ordering
+    // as specified in the constructor (except for the 'type' key that ends
+    // up at top level) - unlike ObjectName(String, Map) constructor
+    // that seems to have a mind of its own...
+    StringBuilder sb = new StringBuilder();
+    if (domain.equals(currentDomain)) {
+      if (subdomains != null && subdomains.length > 1) {
+        // use only first segment as domain
+        sb.append(subdomains[0]);
+        sb.append(':');
+        // use remaining segments as properties
+        for (int i = 1; i < subdomains.length; i++) {
+          if (i > 1) {
+            sb.append(',');
+          }
+          sb.append("dom");
+          sb.append(String.valueOf(i));
+          sb.append('=');
+          sb.append(subdomains[i]);
+        }
+        sb.append(','); // separate from other properties
+      } else {
+        sb.append(currentDomain);
+        sb.append(':');
+      }
+    } else {
+      sb.append(currentDomain);
+      sb.append(':');
+    }
+    sb.append("reporter=");
+    sb.append(reporterName);
+    sb.append(',');
+    if (metricInfo != null) {
+      sb.append("category=");
+      sb.append(metricInfo.category.toString());
+      sb.append(",scope=");
+      sb.append(metricInfo.scope);
+      // we could also split by type, but don't call it 'type' :)
+      // if (type != null) {
+      //   sb.append(",class=");
+      //   sb.append(type);
+      // }
+      sb.append(",name=");
+      sb.append(metricInfo.name);
+    } else {
+      // make dotted names into hierarchies
+      String[] path = name.split("\\.");
+      for (int i = 0; i < path.length - 1; i++) {
+        if (i > 0) {
+          sb.append(',');
+        }
+        sb.append("name"); sb.append(String.valueOf(i));
+        sb.append('=');
+        sb.append(path[i]);
+      }
+      if (path.length > 1) {
+        sb.append(',');
+      }
+      // split by type
+      // if (type != null) {
+      //   sb.append("class=");
+      //   sb.append(type);
+      // }
+      sb.append("name=");
+      sb.append(path[path.length - 1]);
+    }
+    if (props != null && props.length > 0) {
+      for (int i = 0; i < props.length; i += 2) {
+        if (props[i] == null || props[i].isEmpty()) {
+          continue;
+        }
+        if (props[i + 1] == null || props[i + 1].isEmpty()) {
+          continue;
+        }
+        sb.append(',');
+        sb.append(props[i]);
+        sb.append('=');
+        sb.append(props[i + 1]);
+      }
+    }
+
+    ObjectName objectName;
+
+    try {
+      objectName = new ObjectName(sb.toString());
+    } catch (MalformedObjectNameException e) {
+      throw new RuntimeException(sb.toString(), e);
+    }
+
+    return objectName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
index 47fbf11..0e78eee 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
@@ -17,8 +17,6 @@
 package org.apache.solr.metrics.reporters;
 
 import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
@@ -26,9 +24,7 @@ import java.lang.management.ManagementFactory;
 import java.util.Locale;
 
 import com.codahale.metrics.JmxReporter;
-import com.codahale.metrics.ObjectNameFactory;
 import org.apache.solr.core.PluginInfo;
-import org.apache.solr.metrics.SolrMetricInfo;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
 import org.apache.solr.util.JmxUtil;
@@ -180,105 +176,4 @@ public class SolrJmxReporter extends SolrMetricReporter {
         getClass().getName(), Integer.toHexString(hashCode()), domain, serviceUrl, agentId);
   }
 
-  /**
-   * Factory to create MBean names for a given metric.
-   */
-  private static class JmxObjectNameFactory implements ObjectNameFactory {
-
-    private final String domain;
-    private final String[] subdomains;
-    private final String reporterName;
-
-    JmxObjectNameFactory(String reporterName, String domain) {
-      this.reporterName = reporterName;
-      this.domain = domain;
-      this.subdomains = domain.split("\\.");
-    }
-
-    /**
-     * Create a hierarchical name of a metric.
-     *
-     * @param type    metric class, eg. "counters"
-     * @param currentDomain  JMX domain
-     * @param name    metric name
-     */
-    @Override
-    public ObjectName createName(String type, String currentDomain, String name) {
-      SolrMetricInfo metricInfo = SolrMetricInfo.of(name);
-
-      // It turns out that ObjectName(String) mostly preserves key ordering
-      // as specified in the constructor (except for the 'type' key that ends
-      // up at top level) - unlike ObjectName(String, Map) constructor
-      // that seems to have a mind of its own...
-      StringBuilder sb = new StringBuilder();
-      if (domain.equals(currentDomain)) {
-        if (subdomains != null && subdomains.length > 1) {
-          // use only first segment as domain
-          sb.append(subdomains[0]);
-          sb.append(':');
-          // use remaining segments as properties
-          for (int i = 1; i < subdomains.length; i++) {
-            if (i > 1) {
-              sb.append(',');
-            }
-            sb.append("dom");
-            sb.append(String.valueOf(i));
-            sb.append('=');
-            sb.append(subdomains[i]);
-          }
-          sb.append(','); // separate from other properties
-        } else {
-          sb.append(currentDomain);
-          sb.append(':');
-        }
-      } else {
-        sb.append(currentDomain);
-        sb.append(':');
-      }
-      sb.append("reporter=");
-      sb.append(reporterName);
-      sb.append(',');
-      if (metricInfo != null) {
-        sb.append("category=");
-        sb.append(metricInfo.category.toString());
-        sb.append(",scope=");
-        sb.append(metricInfo.scope);
-        // we could also split by type, but don't call it 'type' :)
-        // sb.append(",class=");
-        //sb.append(type);
-        sb.append(",name=");
-        sb.append(metricInfo.name);
-      } else {
-        // make dotted names into hierarchies
-        String[] path = name.split("\\.");
-        for (int i = 0; i < path.length - 1; i++) {
-          if (i > 0) {
-            sb.append(',');
-          }
-          sb.append("name"); sb.append(String.valueOf(i));
-          sb.append('=');
-          sb.append(path[i]);
-        }
-        if (path.length > 1) {
-          sb.append(',');
-        }
-        // split by type
-        // sb.append("class=");
-        // sb.append(type);
-        sb.append("name=");
-        sb.append(path[path.length - 1]);
-      }
-
-      ObjectName objectName;
-
-      try {
-        objectName = new ObjectName(sb.toString());
-      } catch (MalformedObjectNameException e) {
-        throw new RuntimeException(sb.toString(), e);
-      }
-
-      return objectName;
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
index f8d677a..87aaa8f 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
@@ -292,6 +292,11 @@ public class FacetModule extends SearchComponent {
   }
 
   @Override
+  public Category getCategory() {
+    return Category.QUERY;
+  }
+
+  @Override
   public String getSource() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index 9de9e42..fdd4408 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -235,6 +235,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       public String getDescription() {
         return "Return the public key of this server";
       }
+
+      @Override
+      public Category getCategory() {
+        return Category.ADMIN;
+      }
+
     };
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index 73731ad..035ae8d 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -850,11 +850,6 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   }
 
   @Override
-  public Category getCategory() {
-    return Category.UPDATEHANDLER;
-  }
-
-  @Override
   public String getSource() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
index f52e353..1cf8a3f 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
@@ -200,4 +200,9 @@ public abstract class UpdateHandler implements SolrInfoMBean {
   }
 
   public abstract void split(SplitIndexCommand cmd) throws IOException;
+
+  @Override
+  public Category getCategory() {
+    return Category.UPDATE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 626dcd6..7c2dae6 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -103,7 +103,20 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
       }
     }
   }
-  public enum State { REPLAYING, BUFFERING, APPLYING_BUFFERED, ACTIVE }
+
+  // NOTE: when adding new states make sure to keep existing numbers, because external metrics
+  // monitoring may depend on these values being stable.
+  public enum State { REPLAYING(0), BUFFERING(1), APPLYING_BUFFERED(2), ACTIVE(3);
+    private final int value;
+
+    State(final int value) {
+      this.value = value;
+    }
+
+    public int getValue() {
+      return value;
+    }
+  }
 
   public static final int ADD = 0x01;
   public static final int DELETE = 0x02;
@@ -365,19 +378,14 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
       }
     };
     replayLogsCountGauge = () -> logs.size();
-    replayBytesGauge = () -> {
-      if (state == State.REPLAYING) {
-        return getTotalLogsSize();
-      } else {
-        return 0L;
-      }
-    };
+    replayBytesGauge = () -> getTotalLogsSize();
+
     manager.register(registry, bufferedOpsGauge, true, "ops", scope, "buffered");
     manager.register(registry, replayLogsCountGauge, true, "logs", scope, "replay", "remaining");
     manager.register(registry, replayBytesGauge, true, "bytes", scope, "replay", "remaining");
-    applyingBufferedOpsMeter = manager.meter(registry, "ops", scope, "applying_buffered");
+    applyingBufferedOpsMeter = manager.meter(registry, "ops", scope, "applyingBuffered");
     replayOpsMeter = manager.meter(registry, "ops", scope, "replay");
-    stateGauge = () -> state.ordinal();
+    stateGauge = () -> state.getValue();
     manager.register(registry, stateGauge, true, "state", scope);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index 67447a3..5affae6 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -128,7 +128,7 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
 
   @Override
   public Category getCategory() {
-    return Category.HTTP;
+    return Category.UPDATE;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test-files/solr/solr-gangliareporter.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-gangliareporter.xml b/solr/core/src/test-files/solr/solr-gangliareporter.xml
index 9e7233c..4327388 100644
--- a/solr/core/src/test-files/solr/solr-gangliareporter.xml
+++ b/solr/core/src/test-files/solr/solr-gangliareporter.xml
@@ -24,7 +24,7 @@
     <!-- for unit tests this is set to 1 second - DO NOT USE THIS VALUE IN PRODUCTION! -->
     <int name="period">1</int>
     <str name="prefix">test</str>
-    <str name="filter">cores</str>
+    <str name="filter">CONTAINER.cores</str>
     <!-- this is only used when unit testing - DO NOT USE otherwise! -->
     <bool name="testing">true</bool>
   </reporter>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test-files/solr/solr-graphitereporter.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-graphitereporter.xml b/solr/core/src/test-files/solr/solr-graphitereporter.xml
index a0557df..7c7f5ca 100644
--- a/solr/core/src/test-files/solr/solr-graphitereporter.xml
+++ b/solr/core/src/test-files/solr/solr-graphitereporter.xml
@@ -24,7 +24,7 @@
     <!-- for unit tests this is set to 1 second - DO NOT USE THIS VALUE IN PRODUCTION! -->
     <int name="period">1</int>
     <str name="prefix">test</str>
-    <str name="filter">cores</str>
+    <str name="filter">CONTAINER.cores</str>
     <bool name="pickled">false</bool>
   </reporter>
  </metrics>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test-files/solr/solr-slf4jreporter.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-slf4jreporter.xml b/solr/core/src/test-files/solr/solr-slf4jreporter.xml
index 1a08416..f3144ca 100644
--- a/solr/core/src/test-files/solr/solr-slf4jreporter.xml
+++ b/solr/core/src/test-files/solr/solr-slf4jreporter.xml
@@ -22,13 +22,13 @@
     <!-- for unit tests this is set to 1 second - DO NOT USE THIS VALUE IN PRODUCTION! -->
     <int name="period">1</int>
     <str name="prefix">test</str>
-    <str name="filter">cores</str>
+    <str name="filter">CONTAINER.cores</str>
   </reporter>
   <reporter name="test2" group="node" class="org.apache.solr.metrics.reporters.SolrSlf4jReporter">
     <!-- for unit tests this is set to 1 second - DO NOT USE THIS VALUE IN PRODUCTION! -->
     <int name="period">1</int>
     <str name="prefix">test</str>
-    <str name="filter">cores</str>
+    <str name="filter">CONTAINER.cores</str>
     <str name="logger">foobar</str>
   </reporter>
  </metrics>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
index 906116e..25c483b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
@@ -754,7 +754,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
       QueryRequest req = new QueryRequest(params);
       NamedList<Object> resp = client.request(req);
       NamedList mbeans = (NamedList) resp.get("solr-mbeans");
-      NamedList uhandlerCat = (NamedList) mbeans.get("UPDATEHANDLER");
+      NamedList uhandlerCat = (NamedList) mbeans.get("UPDATE");
       NamedList uhandler = (NamedList) uhandlerCat.get("updateHandler");
       NamedList stats = (NamedList) uhandler.get("stats");
       return (Long) stats.get("commits");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/handler/admin/MBeansHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MBeansHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MBeansHandlerTest.java
index 7169888..21634b7 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MBeansHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MBeansHandlerTest.java
@@ -56,7 +56,7 @@ public class MBeansHandlerTest extends SolrTestCaseJ4 {
     NamedList<NamedList<NamedList<Object>>> diff = SolrInfoMBeanHandler.fromXML(xml);
 
     // The stats bean for SolrInfoMBeanHandler
-    NamedList stats = (NamedList)diff.get("QUERYHANDLER").get("/admin/mbeans").get("stats");
+    NamedList stats = (NamedList)diff.get("ADMIN").get("/admin/mbeans").get("stats");
     
     //System.out.println("stats:"+stats);
     assertEquals("Was: 1, Now: 2, Delta: 1", stats.get("requests"));
@@ -67,7 +67,7 @@ public class MBeansHandlerTest extends SolrTestCaseJ4 {
         "key","org.apache.solr.handler.admin.CollectionsHandler"
     ));
     NamedList<NamedList<NamedList<Object>>> nl = SolrInfoMBeanHandler.fromXML(xml);
-    assertNotNull( nl.get("QUERYHANDLER").get("org.apache.solr.handler.admin.CollectionsHandler"));
+    assertNotNull( nl.get("ADMIN").get("org.apache.solr.handler.admin.CollectionsHandler"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
index a3e724a..edab3ce 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -51,10 +51,10 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(((NamedList) nl.get("SEARCHER.new.errors")).get("count"));
     assertEquals(0L, ((NamedList) nl.get("SEARCHER.new.errors")).get("count"));
     nl = (NamedList) values.get("solr.node");
-    assertNotNull(nl.get("cores.loaded")); // int gauge
-    assertEquals(1, ((NamedList) nl.get("cores.loaded")).get("value"));
-    assertNotNull(nl.get("QUERYHANDLER./admin/authorization.clientErrors")); // timer type
-    assertEquals(5, ((NamedList) nl.get("QUERYHANDLER./admin/authorization.clientErrors")).size());
+    assertNotNull(nl.get("CONTAINER.cores.loaded")); // int gauge
+    assertEquals(1, ((NamedList) nl.get("CONTAINER.cores.loaded")).get("value"));
+    assertNotNull(nl.get("ADMIN./admin/authorization.clientErrors")); // timer type
+    assertEquals(5, ((NamedList) nl.get("ADMIN./admin/authorization.clientErrors")).size());
 
     resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
@@ -91,10 +91,10 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertEquals(1, values.size());
     values = (NamedList) values.get("solr.node");
     assertNotNull(values);
-    assertNull(values.get("QUERYHANDLER./admin/authorization.errors")); // this is a timer node
+    assertNull(values.get("ADMIN./admin/authorization.errors")); // this is a timer node
 
     resp = new SolrQueryResponse();
-    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "prefix", "cores"), resp);
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "prefix", "CONTAINER.cores"), resp);
     values = resp.getValues();
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
@@ -105,10 +105,10 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertEquals(3, ((NamedList)values.get("solr.node")).size());
     assertNotNull(values.get("solr.node"));
     values = (NamedList) values.get("solr.node");
-    assertNotNull(values.get("cores.lazy")); // this is a gauge node
+    assertNotNull(values.get("CONTAINER.cores.lazy")); // this is a gauge node
 
     resp = new SolrQueryResponse();
-    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "prefix", "cores"), resp);
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "prefix", "CONTAINER.cores"), resp);
     values = resp.getValues();
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");
@@ -117,7 +117,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNull(values.get("solr.node"));
 
     resp = new SolrQueryResponse();
-    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "timer", "prefix", "cores"), resp);
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "timer", "prefix", "CONTAINER.cores"), resp);
     values = resp.getValues();
     assertNotNull(values.get("metrics"));
     values = (NamedList) values.get("metrics");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
index c6449ac..27c038b 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -51,7 +51,7 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
   private static final String MULTIREGISTRY = "multiregistry";
   private static final String[] INITIAL_REPORTERS = {REPORTER_NAMES[0], REPORTER_NAMES[1], UNIVERSAL, SPECIFIC, MULTIGROUP, MULTIREGISTRY};
   private static final String[] RENAMED_REPORTERS = {REPORTER_NAMES[0], REPORTER_NAMES[1], UNIVERSAL, MULTIGROUP};
-  private static final SolrInfoMBean.Category HANDLER_CATEGORY = SolrInfoMBean.Category.QUERYHANDLER;
+  private static final SolrInfoMBean.Category HANDLER_CATEGORY = SolrInfoMBean.Category.QUERY;
 
   private CoreContainer cc;
   private SolrMetricManager metricManager;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
index d9675ef..c50ff3c 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
@@ -75,7 +75,7 @@ public class SolrGangliaReporterTest extends SolrTestCaseJ4 {
     assertTrue(names.size() >= 3);
     String[] frozenNames = (String[])names.toArray(new String[names.size()]);
     for (String name : frozenNames) {
-      assertTrue(name, name.startsWith("test.solr.node.cores."));
+      assertTrue(name, name.startsWith("test.solr.node.CONTAINER.cores."));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
index 3d1c482..e58c9dd 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
@@ -71,7 +71,7 @@ public class SolrGraphiteReporterTest extends SolrTestCaseJ4 {
       assertTrue(mock.lines.size() >= 3);
       String[] frozenLines = (String[])mock.lines.toArray(new String[mock.lines.size()]);
       for (String line : frozenLines) {
-        assertTrue(line, line.startsWith("test.solr.node.cores."));
+        assertTrue(line, line.startsWith("test.solr.node.CONTAINER.cores."));
       }
     } finally {
       mock.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/core/src/test/org/apache/solr/search/TestRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index cc3ca47..15aed5d 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -366,7 +366,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
       assertEquals(UpdateLog.State.BUFFERING.ordinal(), state.getValue().intValue());
       Gauge<Integer> bufferedOps = (Gauge<Integer>)metrics.get("TLOG.buffered.ops");
       int initialOps = bufferedOps.getValue();
-      Meter applyingBuffered = (Meter)metrics.get("TLOG.applying_buffered.ops");
+      Meter applyingBuffered = (Meter)metrics.get("TLOG.applyingBuffered.ops");
       long initialApplyingOps = applyingBuffered.getCount();
 
       // simulate updates from a leader

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6c6c0774/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index 541ffed..cbaaa1b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@ -391,7 +391,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
 
   private Long getNumRequests(String baseUrl, String collectionName) throws
       SolrServerException, IOException {
-    return getNumRequests(baseUrl, collectionName, "QUERYHANDLER", "standard", false);
+    return getNumRequests(baseUrl, collectionName, "QUERY", "standard", false);
   }
 
   private Long getNumRequests(String baseUrl, String collectionName, String category, String key, boolean returnNumErrors) throws
@@ -431,7 +431,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
         for (String adminPath : adminPathToMbean.keySet()) {
           long errorsBefore = 0;
           for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
-            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "QUERYHANDLER", adminPathToMbean.get(adminPath), true);
+            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "QUERY", adminPathToMbean.get(adminPath), true);
             errorsBefore += numRequests;
             log.info("Found {} requests to {} on {}", numRequests, adminPath, runner.getBaseUrl());
           }
@@ -448,7 +448,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
           }
           long errorsAfter = 0;
           for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
-            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "QUERYHANDLER", adminPathToMbean.get(adminPath), true);
+            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "QUERY", adminPathToMbean.get(adminPath), true);
             errorsAfter += numRequests;
             log.info("Found {} requests to {} on {}", numRequests, adminPath, runner.getBaseUrl());
           }


[09/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter

Posted by ho...@apache.org.
SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter


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

Branch: refs/heads/jira/solr-5944
Commit: 570880d3acb45c925e8dc77172e0725ab8ba07b8
Parents: ea49989
Author: David Smiley <ds...@apache.org>
Authored: Sat Jan 7 23:32:37 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sat Jan 7 23:32:37 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  5 +++-
 .../solr/highlight/UnifiedSolrHighlighter.java  | 12 +++++++++-
 .../highlight/TestUnifiedSolrHighlighter.java   | 24 ++++++++++++++++----
 .../solr/common/params/HighlightParams.java     |  2 +-
 4 files changed, 35 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 899dcd3..c18381e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -109,7 +109,7 @@ Upgrade Notes
 
 * SOLR-9708: You are encouraged to try out the UnifiedHighlighter by setting hl.method=unified and report feedback. It
   might become the default in 7.0. It's more efficient/faster than the other highlighters, especially compared to the
-  original Highlighter. That said, some options aren't supported yet, notably hl.fragsize.
+  original Highlighter. That said, some options aren't supported yet.
   It will get more features in time, especially with your input. See HighlightParams.java
   for a listing of highlight parameters annotated with which highlighters use them.
   hl.useFastVectorHighlighter is now considered deprecated in lieu of hl.method=fastVector.
@@ -225,6 +225,9 @@ New Features
 * SOLR-7466: Enable leading wildcard in complexphrase query parser, optimize it with  ReversedWildcardFilterFactory 
   when it's provided (Mikhail Khludnev)
 
+* SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter to avoid snippets/Passages that are too small.
+  Defaults to 70. (David Smiley)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
index 910fa2b..5b59b85 100644
--- a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
@@ -30,6 +30,7 @@ import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
 import org.apache.lucene.search.uhighlight.DefaultPassageFormatter;
+import org.apache.lucene.search.uhighlight.LengthGoalBreakIterator;
 import org.apache.lucene.search.uhighlight.PassageFormatter;
 import org.apache.lucene.search.uhighlight.PassageScorer;
 import org.apache.lucene.search.uhighlight.UnifiedHighlighter;
@@ -299,7 +300,16 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
       String variant = params.getFieldParam(field, HighlightParams.BS_VARIANT);
       Locale locale = parseLocale(language, country, variant);
       String type = params.getFieldParam(field, HighlightParams.BS_TYPE);
-      return parseBreakIterator(type, locale);
+      BreakIterator baseBI = parseBreakIterator(type, locale);
+
+      // Use a default fragsize the same as the regex Fragmenter (original Highlighter) since we're
+      //  both likely shooting for sentence-like patterns.
+      int fragsize = params.getFieldInt(field, HighlightParams.FRAGSIZE, LuceneRegexFragmenter.DEFAULT_FRAGMENT_SIZE);
+      if (fragsize <= 1 || baseBI instanceof WholeBreakIterator) { // no real minimum size
+        return baseBI;
+      }
+      return LengthGoalBreakIterator.createMinLength(baseBI, fragsize);
+      // TODO option for using createClosestToLength()
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
index e2511be..2eb4ba3 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -78,7 +78,8 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         "text2", "document one", "text3", "crappy document", "id", "101"));
     assertU(commit());
     assertQ("multiple snippets test",
-        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.snippets", "2", "hl.bs.type", "SENTENCE"),
+        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.snippets", "2", "hl.bs.type", "SENTENCE",
+            "hl.fragsize", "0"),
         "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/*)=2",
         "//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>Document</em> snippet one. '",
         "//lst[@name='highlighting']/lst[@name='101']/arr/str[2]='<em>Document</em> snippet two.'");
@@ -202,21 +203,34 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text3']/str='crappier [document]'");
   }
   
-  public void testBreakIterator() {
+  public void testBreakIteratorWord() {
     assertQ("different breakiterator", 
-        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WORD"),
+        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WORD", "hl.fragsize", "-1"),
         "count(//lst[@name='highlighting']/*)=2",
         "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/str='<em>document</em>'",
         "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/str='<em>document</em>'");
   }
   
-  public void testBreakIterator2() {
+  public void testBreakIteratorWhole() {
     assertU(adoc("text", "Document one has a first sentence. Document two has a second sentence.", "id", "103"));
     assertU(commit());
     assertQ("different breakiterator", 
-        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WHOLE"),
+        req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WHOLE", "hl.fragsize", "-1"),
         "//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='<em>Document</em> one has a first sentence. <em>Document</em> two has a second sentence.'");
   }
+
+  public void testFragsize() {
+    // test default is 70... so make a sentence that is a little less (closer to 70 than end of text)
+    clearIndex();
+    assertU(adoc("id", "10", "text", "This is a sentence just under seventy chars in length blah blah. Next sentence is here."));
+    assertU(commit());
+    assertQ("default fragsize",
+        req("q", "text:seventy", "hl", "true"),
+        "//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. Next sentence is here.'");
+    assertQ("smaller fragsize",
+        req("q", "text:seventy", "hl", "true", "hl.fragsize", "60"), // a bit smaller
+        "//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. '");
+  }
   
   public void testEncoder() {
     assertU(adoc("text", "Document one has a first <i>sentence</i>.", "id", "103"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/570880d3/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
index 917e9f5..997fc7e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
@@ -49,7 +49,7 @@ public interface HighlightParams {
   public static final String HIGHLIGHT_ALTERNATE = HIGHLIGHT+".highlightAlternate"; // OH, FVH
 
   // sizing
-  public static final String FRAGSIZE    = HIGHLIGHT+".fragsize"; // OH, FVH
+  public static final String FRAGSIZE    = HIGHLIGHT+".fragsize"; // OH, FVH, UH
   public static final String FRAGMENTER  = HIGHLIGHT+".fragmenter"; // OH
   public static final String INCREMENT   = HIGHLIGHT+".increment"; // OH
   public static final String REGEX       = "regex"; // OH


[15/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7611: Remove queries javadoc link from suggester help page

Posted by ho...@apache.org.
LUCENE-7611: Remove queries javadoc link from suggester help page


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

Branch: refs/heads/jira/solr-5944
Commit: 322ad889604688db9d22ba7dfa1e389a01e34857
Parents: 8bc151d
Author: Alan Woodward <ro...@apache.org>
Authored: Sun Jan 8 20:07:30 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sun Jan 8 20:07:30 2017 +0000

----------------------------------------------------------------------
 lucene/suggest/build.xml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/322ad889/lucene/suggest/build.xml
----------------------------------------------------------------------
diff --git a/lucene/suggest/build.xml b/lucene/suggest/build.xml
index bc4bed5..47d4a63 100644
--- a/lucene/suggest/build.xml
+++ b/lucene/suggest/build.xml
@@ -38,7 +38,6 @@
     <invoke-module-javadoc>
       <links>
         <link href="../analyzers-common"/>
-        <link href="../queries"/>
       </links>
     </invoke-module-javadoc>
   </target>


[07/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9944: Update CHANGES.txt

Posted by ho...@apache.org.
SOLR-9944: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-5944
Commit: ac14fc32e045d45b5129dc237f7e5472fc86e4a0
Parents: aae4217
Author: Joel Bernstein <jb...@apache.org>
Authored: Sat Jan 7 22:19:46 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Sat Jan 7 22:19:46 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ac14fc32/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0d61730..899dcd3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -401,6 +401,8 @@ Other Changes
 
 * SOLR-3990: Moves getIndexSize() from ReplicationHandler to SolrCore (Shawn Heisey)
 
+* SOLR-9944: Map the nodes function name to the GatherNodesStream (Joel Bernstein)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.


[25/50] [abbrv] lucene-solr:jira/solr-5944: * SOLR-9886: Add a 'enable' flag to caches to enable/disable them

Posted by ho...@apache.org.
* SOLR-9886: Add a 'enable' flag to caches to enable/disable them


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

Branch: refs/heads/jira/solr-5944
Commit: 2048b82443db548f76d584f9a95b5628c407edde
Parents: 2b4e3dd
Author: Noble Paul <no...@apache.org>
Authored: Tue Jan 10 21:05:38 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jan 10 21:05:38 2017 +1030

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../org/apache/solr/search/CacheConfig.java     |  2 +-
 .../org/apache/solr/search/FastLRUCache.java    |  2 +-
 .../src/java/org/apache/solr/util/DOMUtil.java  |  5 ++
 .../resources/EditableSolrConfigAttributes.json | 16 +++-
 .../conf/solrconfig-cache-enable-disable.xml    | 80 ++++++++++++++++++++
 .../test/org/apache/solr/core/TestConfig.java   | 32 ++++++++
 7 files changed, 136 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2b79f04..2a5d5bb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -230,6 +230,8 @@ New Features
 
 * SOLR-9856: Collect metrics for shard replication and tlog replay on replicas (ab).
 
+* SOLR-9886: Add a 'enable' flag to caches to enable/disable them (Pushkar Raste, noble)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/core/src/java/org/apache/solr/search/CacheConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CacheConfig.java b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
index d3565a6..648fe38 100644
--- a/solr/core/src/java/org/apache/solr/search/CacheConfig.java
+++ b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
@@ -89,7 +89,7 @@ public class CacheConfig implements MapSerializable{
 
   public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
     Node node = solrConfig.getNode(xpath, false);
-    if(node == null) {
+    if(node == null || !"true".equals(DOMUtil.getAttrOrDefault(node, "enabled", "true"))) {
       Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
       if(m==null) return null;
       List<String> parts = StrUtils.splitSmart(xpath, '/');

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
index 6c2e4d5..9c4b892 100644
--- a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
@@ -69,7 +69,7 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>
     } else {
       minLimit = Integer.parseInt(str);
     }
-    if (minLimit==0) minLimit=1;
+    if (minLimit <= 0) minLimit = 1;
     if (limit <= minLimit) limit=minLimit+1;
 
     int acceptableLimit;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/core/src/java/org/apache/solr/util/DOMUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/DOMUtil.java b/solr/core/src/java/org/apache/solr/util/DOMUtil.java
index 773d08c..29dab93 100644
--- a/solr/core/src/java/org/apache/solr/util/DOMUtil.java
+++ b/solr/core/src/java/org/apache/solr/util/DOMUtil.java
@@ -79,6 +79,11 @@ public class DOMUtil {
     return getAttr(nd.getAttributes(), name);
   }
 
+  public static String getAttrOrDefault(Node nd, String name, String def) {
+    String attr = getAttr(nd.getAttributes(), name);
+    return attr == null ? def : attr;
+  }
+
   public static String getAttr(NamedNodeMap attrs, String name, String missing_err) {
     Node attr = attrs==null? null : attrs.getNamedItem(name);
     if (attr==null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/core/src/resources/EditableSolrConfigAttributes.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/EditableSolrConfigAttributes.json b/solr/core/src/resources/EditableSolrConfigAttributes.json
index b0d6c2f..ce9d1ad 100644
--- a/solr/core/src/resources/EditableSolrConfigAttributes.json
+++ b/solr/core/src/resources/EditableSolrConfigAttributes.json
@@ -1,4 +1,14 @@
 {
+//-------legend----------
+// 0  = string attribute
+// 1  = string node
+// 10 = boolean attribute
+// 11 = boolean node
+// 20 = int attrubute
+// 21 = int node
+// 30 = float attribute
+// 31 = float node
+//------------------------
   "updateHandler":{
     "autoCommit":{
       "maxDocs":20,
@@ -12,6 +22,7 @@
   "query":{
     "filterCache":{
       "class":0,
+      "enabled":10,
       "size":0,
       "initialSize":20,
       "autowarmCount":20,
@@ -19,6 +30,7 @@
       "regenerator":0},
     "queryResultCache":{
       "class":0,
+      "enabled":10,
       "size":20,
       "initialSize":20,
       "autowarmCount":20,
@@ -26,12 +38,14 @@
       "regenerator":0},
     "documentCache":{
       "class":0,
+      "enabled":10,
       "size":20,
       "initialSize":20,
       "autowarmCount":20,
       "regenerator":0},
     "fieldValueCache":{
       "class":0,
+      "enabled":10,
       "size":20,
       "initialSize":20,
       "autowarmCount":20,
@@ -56,4 +70,4 @@
   "peerSync":{
     "useRangeVersions":11
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
new file mode 100644
index 0000000..4053ebe
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-cache-enable-disable.xml
@@ -0,0 +1,80 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  <dataDir>${solr.data.dir:}</dataDir>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+  <requestHandler name="standard" class="solr.StandardRequestHandler" />
+  
+  <query>
+    <!-- Maximum number of clauses in a boolean query... can affect
+        range or wildcard queries that expand to big boolean
+        queries.  An exception is thrown if exceeded.
+    -->
+    <maxBooleanClauses>1024</maxBooleanClauses>
+
+    <!-- Cache specification for Filters or DocSets - unordered set of *all* documents
+         that match a particular query.
+      -->
+    <filterCache
+      enabled="${filterCache.enabled}"
+      class="solr.search.FastLRUCache"
+      size="512"
+      initialSize="512"
+      autowarmCount="2"/>
+
+    <queryResultCache
+      enabled="${queryResultCache.enabled}"
+      class="solr.search.LRUCache"
+      size="512"
+      initialSize="512"
+      autowarmCount="2"/>
+
+    <documentCache
+      enabled="${documentCache.enabled}"
+      class="solr.search.LRUCache"
+      size="512"
+      initialSize="512"
+      autowarmCount="0"/>
+
+
+    <!-- If true, stored fields that are not requested will be loaded lazily.
+    -->
+    <enableLazyFieldLoading>true</enableLazyFieldLoading>
+
+    <queryResultWindowSize>10</queryResultWindowSize>
+
+    <!-- set maxSize artificially low to exercise both types of sets -->
+    <HashDocSet maxSize="3" loadFactor="0.75"/>
+
+    <!-- boolToFilterOptimizer converts boolean clauses with zero boost
+         into cached filters if the number of docs selected by the clause exceeds
+         the threshold (represented as a fraction of the total index)
+    -->
+    <boolTofilterOptimizer enabled="false" cacheSize="32" threshold=".05"/>
+
+  </query>
+  
+</config>
+
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2048b824/solr/core/src/test/org/apache/solr/core/TestConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfig.java b/solr/core/src/test/org/apache/solr/core/TestConfig.java
index 8244b32..094f013 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfig.java
@@ -105,6 +105,38 @@ public class TestConfig extends SolrTestCaseJ4 {
     assertTrue("file handler should have been automatically registered", handler != null);
 
   }
+  
+ @Test
+ public void testCacheEnablingDisabling() throws Exception {
+   // ensure if cache is not defined in the config then cache is disabled 
+   SolrConfig sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-defaults.xml", null);
+   assertNull(sc.filterCacheConfig);
+   assertNull(sc.queryResultCacheConfig);
+   assertNull(sc.documentCacheConfig);
+   
+   // enable all the caches via system properties and verify 
+   System.setProperty("filterCache.enabled", "true");
+   System.setProperty("queryResultCache.enabled", "true");
+   System.setProperty("documentCache.enabled", "true");
+   sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-cache-enable-disable.xml", null);
+   assertNotNull(sc.filterCacheConfig);
+   assertNotNull(sc.queryResultCacheConfig);
+   assertNotNull(sc.documentCacheConfig);
+   
+   // disable all the caches via system properties and verify
+   System.setProperty("filterCache.enabled", "false");
+   System.setProperty("queryResultCache.enabled", "false");
+   System.setProperty("documentCache.enabled", "false");
+   sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-cache-enable-disable.xml", null);
+   assertNull(sc.filterCacheConfig);
+   assertNull(sc.queryResultCacheConfig);
+   assertNull(sc.documentCacheConfig);
+   
+   System.clearProperty("filterCache.enabled");
+   System.clearProperty("queryResultCache.enabled");
+   System.clearProperty("documentCache.enabled");
+ }
+  
 
   // If defaults change, add test methods to cover each version
   @Test


[27/50] [abbrv] lucene-solr:jira/solr-5944: use relative URL path instead of absolute path starting from /solr

Posted by ho...@apache.org.
use relative URL path instead of absolute path starting from /solr


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

Branch: refs/heads/jira/solr-5944
Commit: e0b4caccd3312b011cdfbb3951ea43812486ca98
Parents: 98422e0
Author: Yun Jie Zhou <yu...@cn.ibm.com>
Authored: Fri Sep 30 11:18:48 2016 -0400
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Tue Jan 10 14:18:31 2017 +0100

----------------------------------------------------------------------
 solr/webapp/web/js/angular/services.js | 46 ++++++++++++++---------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e0b4cacc/solr/webapp/web/js/angular/services.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/services.js b/solr/webapp/web/js/angular/services.js
index 146939b..07043c3 100644
--- a/solr/webapp/web/js/angular/services.js
+++ b/solr/webapp/web/js/angular/services.js
@@ -19,11 +19,11 @@ var solrAdminServices = angular.module('solrAdminServices', ['ngResource']);
 
 solrAdminServices.factory('System',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/info/system', {"wt":"json", "_":Date.now()});
+    return $resource('admin/info/system', {"wt":"json", "_":Date.now()});
   }])
 .factory('Collections',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/collections',
+    return $resource('admin/collections',
     {'wt':'json', '_':Date.now()}, {
     "list": {params:{action: "LIST"}},
     "status": {params:{action: "CLUSTERSTATUS"}},
@@ -40,7 +40,7 @@ solrAdminServices.factory('System',
   }])
 .factory('Cores',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/cores',
+    return $resource('admin/cores',
     {'wt':'json', '_':Date.now()}, {
     "query": {},
     "list": {params:{indexInfo: false}},
@@ -54,7 +54,7 @@ solrAdminServices.factory('System',
   }])
 .factory('Logging',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/info/logging', {'wt':'json', '_':Date.now()}, {
+    return $resource('admin/info/logging', {'wt':'json', '_':Date.now()}, {
       "events": {params: {since:'0'}},
       "levels": {},
       "setLevel": {}
@@ -62,7 +62,7 @@ solrAdminServices.factory('System',
   }])
 .factory('Zookeeper',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/zookeeper', {wt:'json', _:Date.now()}, {
+    return $resource('admin/zookeeper', {wt:'json', _:Date.now()}, {
       "simple": {},
       "dump": {params: {dump: "true"}},
       "liveNodes": {params: {path: '/live_nodes'}},
@@ -81,30 +81,30 @@ solrAdminServices.factory('System',
   }])
 .factory('Properties',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/info/properties', {'wt':'json', '_':Date.now()});
+    return $resource('admin/info/properties', {'wt':'json', '_':Date.now()});
   }])
 .factory('Threads',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/info/threads', {'wt':'json', '_':Date.now()});
+    return $resource('admin/info/threads', {'wt':'json', '_':Date.now()});
   }])
 .factory('Properties',
   ['$resource', function($resource) {
-    return $resource('/solr/admin/info/properties', {'wt':'json', '_':Date.now()});
+    return $resource('admin/info/properties', {'wt':'json', '_':Date.now()});
   }])
 .factory('Replication',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/replication', {'wt':'json', core: "@core", '_':Date.now()}, {
+    return $resource(':core/replication', {'wt':'json', core: "@core", '_':Date.now()}, {
       "details": {params: {command: "details"}},
       "command": {params: {}}
     });
   }])
 .factory('CoreSystem',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/admin/system', {wt:'json', core: "@core", _:Date.now()});
+    return $resource(':core/admin/system', {wt:'json', core: "@core", _:Date.now()});
   }])
 .factory('Update',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/:handler', {core: '@core', wt:'json', _:Date.now(), handler:'update'}, {
+    return $resource(':core/:handler', {core: '@core', wt:'json', _:Date.now(), handler:'update'}, {
       "optimize": {params: { optimize: "true"}},
       "commit": {params: {commit: "true"}},
       "post": {headers: {'Content-type': 'application/json'}, method: "POST", params: {handler: '@handler'}},
@@ -115,7 +115,7 @@ solrAdminServices.factory('System',
   }])
 .service('FileUpload', function ($http) {
     this.upload = function(params, file, success, error){
-        var url = "/solr/" + params.core + "/" + params.handler + "?";
+        var url = "" + params.core + "/" + params.handler + "?";
         raw = params.raw;
         delete params.core;
         delete params.handler;
@@ -135,7 +135,7 @@ solrAdminServices.factory('System',
 })
 .factory('Luke',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/admin/luke', {core: '@core', wt:'json', _:Date.now()}, {
+    return $resource(':core/admin/luke', {core: '@core', wt:'json', _:Date.now()}, {
       "index":  {params: {numTerms: 0, show: 'index'}},
       "raw": {params: {numTerms: 0}},
       "schema": {params: {show:'schema'}},
@@ -156,13 +156,13 @@ solrAdminServices.factory('System',
   }])
 .factory('Analysis',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/analysis/field', {core: '@core', wt:'json', _:Date.now()}, {
+    return $resource(':core/analysis/field', {core: '@core', wt:'json', _:Date.now()}, {
       "field": {params: {"analysis.showmatch": true}}
     });
   }])
 .factory('DataImport',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/:name', {core: '@core', name: '@name', indent:'on', wt:'json', _:Date.now()}, {
+    return $resource(':core/:name', {core: '@core', name: '@name', indent:'on', wt:'json', _:Date.now()}, {
       "config": {params: {command: "show-config"}, headers: {doNotIntercept: "true"},
                  transformResponse: function(data) {
                     return {config: data};
@@ -177,14 +177,14 @@ solrAdminServices.factory('System',
   }])
 .factory('Ping',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/admin/ping', {wt:'json', core: '@core', ts:Date.now(), _:Date.now()}, {
+    return $resource(':core/admin/ping', {wt:'json', core: '@core', ts:Date.now(), _:Date.now()}, {
      "ping": {},
      "status": {params:{action:"status"}, headers: {doNotIntercept: "true"}
     }});
   }])
 .factory('Mbeans',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/admin/mbeans', {'wt':'json', core: '@core', '_':Date.now()}, {
+    return $resource(':core/admin/mbeans', {'wt':'json', core: '@core', '_':Date.now()}, {
         stats: {params: {stats: true}},
         info: {},
         reference: {
@@ -203,7 +203,7 @@ solrAdminServices.factory('System',
   }])
 .factory('Files',
   ['$resource', function($resource) {
-    return $resource('/solr/:core/admin/file', {'wt':'json', core: '@core', '_':Date.now()}, {
+    return $resource(':core/admin/file', {'wt':'json', core: '@core', '_':Date.now()}, {
       "list": {},
       "get": {method: "GET", interceptor: {
           response: function(config) {return config;}
@@ -214,7 +214,7 @@ solrAdminServices.factory('System',
   }])
 .factory('Query',
     ['$resource', function($resource) {
-       var resource = $resource('/solr/:core/:handler', {core: '@core', handler: '@handler', '_':Date.now()}, {
+       var resource = $resource(':core/:handler', {core: '@core', handler: '@handler', '_':Date.now()}, {
            "query": {
              method: "GET",
              transformResponse: function (data) {
@@ -232,19 +232,19 @@ solrAdminServices.factory('System',
                    }
                }
            }
-           return "/solr/" + params.core + "/" + params.handler + "?" + qs.sort().join("&");
+           return "" + params.core + "/" + params.handler + "?" + qs.sort().join("&");
        }
        return resource;
 }])
 .factory('Segments',
    ['$resource', function($resource) {
-       return $resource('/solr/:core/admin/segments', {'wt':'json', core: '@core', _:Date.now()}, {
+       return $resource(':core/admin/segments', {'wt':'json', core: '@core', _:Date.now()}, {
            get: {}
        });
 }])
 .factory('Schema',
    ['$resource', function($resource) {
-     return $resource('/solr/:core/schema', {wt: 'json', core: '@core', _:Date.now()}, {
+     return $resource(':core/schema', {wt: 'json', core: '@core', _:Date.now()}, {
        get: {method: "GET"},
        check: {method: "GET", headers: {doNotIntercept: "true"}},
        post: {method: "POST"}
@@ -252,7 +252,7 @@ solrAdminServices.factory('System',
 }])
 .factory('Config',
    ['$resource', function($resource) {
-     return $resource('/solr/:core/config', {wt: 'json', core: '@core', _:Date.now()}, {
+     return $resource(':core/config', {wt: 'json', core: '@core', _:Date.now()}, {
        get: {method: "GET"}
      })
 }]);


[37/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-8542: changed Solr Reference Guide section references from "Result Reranking" to "Learning To Rank"

Posted by ho...@apache.org.
SOLR-8542: changed Solr Reference Guide section references from "Result Reranking" to "Learning To Rank"

(Also added bullet points for the solr/contrib/ltr/README links.)


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

Branch: refs/heads/jira/solr-5944
Commit: 987e2650b54af5547ca44abc213cce3a73105f3f
Parents: dee7709
Author: Christine Poerschke <cp...@apache.org>
Authored: Thu Jan 12 17:53:35 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Thu Jan 12 17:56:18 2017 +0000

----------------------------------------------------------------------
 solr/contrib/ltr/README.md                                | 10 +++++-----
 solr/contrib/ltr/example/README.md                        |  2 +-
 .../sample_techproducts_configs/conf/solrconfig.xml       |  6 +++---
 3 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/987e2650/solr/contrib/ltr/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/README.md b/solr/contrib/ltr/README.md
index a997d16..cc28dcc 100644
--- a/solr/contrib/ltr/README.md
+++ b/solr/contrib/ltr/README.md
@@ -8,17 +8,17 @@ deploy that model to Solr and use it to rerank your top X search results.
 # Getting Started With Solr Learning To Rank
 
 For information on how to get started with solr ltr please see:
-[Solr Reference Guide's section on Result Reranking](https://cwiki.apache.org/confluence/display/solr/Result+Reranking)
+ * [Solr Reference Guide's section on Learning To Rank](https://cwiki.apache.org/confluence/display/solr/Learning+To+Rank)
 
 # Getting Started With Solr
 
 For information on how to get started with solr please see:
-[solr/README.txt](../../README.txt)
-[Solr Quick Start](http://lucene.apache.org/solr/quickstart.html)
+ * [solr/README.txt](../../README.txt)
+ * [Solr Quick Start](http://lucene.apache.org/solr/quickstart.html)
 
 # How To Contribute
 
 For information on how to contribute see:
-http://wiki.apache.org/lucene-java/HowToContribute
-http://wiki.apache.org/solr/HowToContribute
+ * http://wiki.apache.org/lucene-java/HowToContribute
+ * http://wiki.apache.org/solr/HowToContribute
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/987e2650/solr/contrib/ltr/example/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/README.md b/solr/contrib/ltr/example/README.md
index 7842494..054c3ab 100644
--- a/solr/contrib/ltr/example/README.md
+++ b/solr/contrib/ltr/example/README.md
@@ -1,6 +1,6 @@
 This README file is only about this example directory's content.
 
-Please refer to the Solr Reference Guide's section on [Result Reranking](https://cwiki.apache.org/confluence/display/solr/Result+Reranking) section for broader information on Learning to Rank (LTR) with Apache Solr.
+Please refer to the Solr Reference Guide's section on [Learning To Rank](https://cwiki.apache.org/confluence/display/solr/Learning+To+Rank) section for broader information on Learning to Rank (LTR) with Apache Solr.
 
 # Start Solr with the LTR plugin enabled
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/987e2650/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index 990a23f..69a1519 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -515,7 +515,7 @@
          when running solr to run with ltr enabled:
            -Dsolr.ltr.enabled=true
 
-         https://cwiki.apache.org/confluence/display/solr/Result+Reranking
+         https://cwiki.apache.org/confluence/display/solr/Learning+To+Rank
       -->
     <cache enable="${solr.ltr.enabled:false}" name="QUERY_DOC_FV"
            class="solr.search.LRUCache"
@@ -1613,7 +1613,7 @@
         when running solr to run with ltr enabled:
           -Dsolr.ltr.enabled=true
 
-        https://cwiki.apache.org/confluence/display/solr/Result+Reranking
+        https://cwiki.apache.org/confluence/display/solr/Learning+To+Rank
 
         Query parser is used to rerank top docs with a provided model
     -->
@@ -1654,7 +1654,7 @@
       when running solr to run with ltr enabled:
         -Dsolr.ltr.enabled=true
 
-      https://cwiki.apache.org/confluence/display/solr/Result+Reranking
+      https://cwiki.apache.org/confluence/display/solr/Learning+To+Rank
       -->
     <transformer enable="${solr.ltr.enabled:false}" name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
       <str name="fvCacheName">QUERY_DOC_FV</str>


[47/50] [abbrv] lucene-solr:jira/solr-5944: Revert "Add getMatchingChildren() method to Scorer"

Posted by ho...@apache.org.
Revert "Add getMatchingChildren() method to Scorer"

This reverts commit 9403372fbc36aced848bf8aa498bc71b7b94567b.


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

Branch: refs/heads/jira/solr-5944
Commit: 9aa78dcca350b11c752dddbc3cfa78b84ecfcf68
Parents: 27ec40d
Author: Alan Woodward <ro...@apache.org>
Authored: Sun Jan 15 10:18:20 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Sun Jan 15 10:18:20 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ---
 .../apache/lucene/search/DisjunctionScorer.java |  8 -----
 .../lucene/search/MinShouldMatchSumScorer.java  |  9 ------
 .../java/org/apache/lucene/search/Scorer.java   |  8 -----
 .../search/TestBooleanQueryVisitSubscorers.java | 33 --------------------
 5 files changed, 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa78dcc/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 540188e..58201d6 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -130,10 +130,6 @@ New features
   SortedSetDocValues to allow filtering their TermsEnums with a
   CompiledAutomaton (Alan Woodward, Mike McCandless)
 
-* LUCENE-7628: Scorer now has a getMatchingChildren() method that will 
-  return all child scorers positioned on the current document.  (Alan
-  Woodward)
-
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa78dcc/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
index a76999e..c53942a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
@@ -202,12 +202,4 @@ abstract class DisjunctionScorer extends Scorer {
     return children;
   }
 
-  @Override
-  public Collection<ChildScorer> getMatchingChildren() throws IOException {
-    List<ChildScorer> children = new ArrayList<>();
-    for (DisiWrapper w = getSubMatches(); w != null; w = w.next) {
-      children.add(new ChildScorer(w.scorer, "SHOULD"));
-    }
-    return children;
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa78dcc/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index b977400..032b5fe 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -133,15 +133,6 @@ final class MinShouldMatchSumScorer extends Scorer {
   }
 
   @Override
-  public Collection<ChildScorer> getMatchingChildren() throws IOException {
-    List<ChildScorer> children = new ArrayList<>();
-    for (DisiWrapper s = lead; s != null; s = s.next) {
-      children.add(new ChildScorer(s.scorer, "SHOULD"));
-    }
-    return children;
-  }
-
-  @Override
   public DocIdSetIterator iterator() {
     return new DocIdSetIterator() {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa78dcc/lucene/core/src/java/org/apache/lucene/search/Scorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Scorer.java b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
index 2e35e91..f434327 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Scorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Scorer.java
@@ -82,14 +82,6 @@ public abstract class Scorer {
   public Collection<ChildScorer> getChildren() {
     return Collections.emptyList();
   }
-
-  /**
-   * Returns child sub-scorers that match the current document
-   * @lucene.experimental
-   */
-  public Collection<ChildScorer> getMatchingChildren() throws IOException {
-    return getChildren();
-  }
   
   /** A child Scorer and its relationship to its parent.
    * the meaning of the relationship depends upon the parent query. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa78dcc/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index 0909d5d..60ba528 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -108,39 +108,6 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
     assertEquals(2, tfs.get(1).intValue()); // f2:search + f2:lucene
     assertEquals(2, tfs.get(2).intValue()); // f2:search + f2:lucene
   }
-
-  public void testDisjunctionMatches() throws IOException {
-    BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
-    bq1.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
-    bq1.add(new PhraseQuery(F2, "search", "engine"), Occur.SHOULD);
-
-    Weight w1 = scorerSearcher.createNormalizedWeight(bq1.build(), true);
-    Scorer s1 = w1.scorer(reader.leaves().get(0));
-    assertEquals(0, s1.iterator().nextDoc());
-    assertEquals(2, s1.getMatchingChildren().size());
-
-    BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
-    bq2.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
-    bq2.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
-
-    Weight w2 = scorerSearcher.createNormalizedWeight(bq2.build(), true);
-    Scorer s2 = w2.scorer(reader.leaves().get(0));
-    assertEquals(0, s2.iterator().nextDoc());
-    assertEquals(1, s2.getMatchingChildren().size());
-  }
-
-  public void testMinShouldMatchMatches() throws IOException {
-    BooleanQuery.Builder bq = new BooleanQuery.Builder();
-    bq.add(new TermQuery(new Term(F1, "lucene")), Occur.SHOULD);
-    bq.add(new TermQuery(new Term(F2, "lucene")), Occur.SHOULD);
-    bq.add(new PhraseQuery(F2, "search", "library"), Occur.SHOULD);
-    bq.setMinimumNumberShouldMatch(2);
-
-    Weight w = scorerSearcher.createNormalizedWeight(bq.build(), true);
-    Scorer s = w.scorer(reader.leaves().get(0));
-    assertEquals(0, s.iterator().nextDoc());
-    assertEquals(2, s.getMatchingChildren().size());
-  }
   
   public void testConjunctions() throws IOException {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();


[08/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7620: UnifiedHighlighter: new LengthGoalBreakIterator wrapper

Posted by ho...@apache.org.
LUCENE-7620: UnifiedHighlighter: new LengthGoalBreakIterator wrapper


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

Branch: refs/heads/jira/solr-5944
Commit: ea49989524e96563f2b9bdd4256012239907882f
Parents: ac14fc3
Author: David Smiley <ds...@apache.org>
Authored: Sat Jan 7 23:10:48 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sat Jan 7 23:10:48 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +-
 .../uhighlight/LengthGoalBreakIterator.java     | 185 +++++++++++++++++++
 .../lucene/search/uhighlight/Passage.java       |   1 +
 .../uhighlight/LengthGoalBreakIteratorTest.java | 104 +++++++++++
 4 files changed, 295 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 30c9ab0..4bbf9ee 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -216,7 +216,11 @@ Improvements
   ensure all dimensions are indexed. (Adrien Grand)
 
 * LUCENE-7614: Complex Phrase Query parser ignores double quotes around single token 
-  prefix, wildcard, range queries (Mikhail Khludnev) 
+  prefix, wildcard, range queries (Mikhail Khludnev)
+
+* LUCENE-7620: Added LengthGoalBreakIterator, a wrapper around another B.I. to skip breaks
+  that would create Passages that are too short.  Only for use with the UnifiedHighlighter
+  (and probably PostingsHighlighter).  (David Smiley)
 
 Optimizations
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java
new file mode 100644
index 0000000..3134013
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/LengthGoalBreakIterator.java
@@ -0,0 +1,185 @@
+/*
+ * 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.lucene.search.uhighlight;
+
+import java.text.BreakIterator;
+import java.text.CharacterIterator;
+
+/**
+ * Wraps another {@link BreakIterator} to skip past breaks that would result in passages that are too
+ * short.  It's still possible to get a short passage but only at the very end of the input text.
+ * <p>
+ * Important: This is not a general purpose {@link BreakIterator}; it's only designed to work in a way
+ * compatible with the {@link UnifiedHighlighter}.  Some assumptions are checked with Java assertions.
+ *
+ * @lucene.experimental
+ */
+public class LengthGoalBreakIterator extends BreakIterator {
+
+  private final BreakIterator baseIter;
+  private final int lengthGoal;
+  private final boolean isMinimumLength; // if false then is "closest to" length
+
+  /** Breaks will be at least {@code minLength} apart (to the extent possible). */
+  public static LengthGoalBreakIterator createMinLength(BreakIterator baseIter, int minLength) {
+    return new LengthGoalBreakIterator(baseIter, minLength, true);
+  }
+
+  /** Breaks will be on average {@code targetLength} apart; the closest break to this target (before or after)
+   * is chosen. */
+  public static LengthGoalBreakIterator createClosestToLength(BreakIterator baseIter, int targetLength) {
+    return new LengthGoalBreakIterator(baseIter, targetLength, false);
+  }
+
+  private LengthGoalBreakIterator(BreakIterator baseIter, int lengthGoal, boolean isMinimumLength) {
+    this.baseIter = baseIter;
+    this.lengthGoal = lengthGoal;
+    this.isMinimumLength = isMinimumLength;
+  }
+
+  // note: the only methods that will get called are setText(txt), getText(),
+  // getSummaryPassagesNoHighlight: current(), first(), next()
+  // highlightOffsetsEnums: preceding(int), and following(int)
+  //   Nonetheless we make some attempt to implement the rest; mostly delegating.
+
+  @Override
+  public String toString() {
+    String goalDesc = isMinimumLength ? "minLen" : "targetLen";
+    return getClass().getSimpleName() + "{" + goalDesc + "=" + lengthGoal + ", baseIter=" + baseIter + "}";
+  }
+
+  @Override
+  public Object clone() {
+    return new LengthGoalBreakIterator((BreakIterator) baseIter.clone(), lengthGoal, isMinimumLength);
+  }
+
+  @Override
+  public CharacterIterator getText() {
+    return baseIter.getText();
+  }
+
+  @Override
+  public void setText(String newText) {
+    baseIter.setText(newText);
+  }
+
+  @Override
+  public void setText(CharacterIterator newText) {
+    baseIter.setText(newText);
+  }
+
+  @Override
+  public int current() {
+    return baseIter.current();
+  }
+
+  @Override
+  public int first() {
+    return baseIter.first();
+  }
+
+  @Override
+  public int last() {
+    return baseIter.last();
+  }
+
+  @Override
+  public int next(int n) {
+    assert false : "Not supported";
+    return baseIter.next(n); // probably wrong
+  }
+
+  // called by getSummaryPassagesNoHighlight to generate default summary.
+  @Override
+  public int next() {
+    return following(current());
+  }
+
+  @Override
+  public int previous() {
+    assert false : "Not supported";
+    return baseIter.previous();
+  }
+
+  // called while the current position is the start of a new passage; find end of passage
+  @Override
+  public int following(int followingIdx) {
+    final int startIdx = current();
+    if (followingIdx < startIdx) {
+      assert false : "Not supported";
+      return baseIter.following(followingIdx);
+    }
+    final int targetIdx = startIdx + lengthGoal;
+    // When followingIdx >= targetIdx, we can simply delegate since it will be >= the target
+    if (followingIdx >= targetIdx - 1) {
+      return baseIter.following(followingIdx);
+    }
+    // If target exceeds the text length, return the last index.
+    if (targetIdx >= getText().getEndIndex()) {
+      return baseIter.last();
+    }
+
+    // Find closest break >= the target
+    final int afterIdx = baseIter.following(targetIdx - 1);
+    if (afterIdx == DONE) { // we're at the end; can this happen?
+      return current();
+    }
+    if (afterIdx == targetIdx) { // right on the money
+      return afterIdx;
+    }
+    if (isMinimumLength) { // thus never undershoot
+      return afterIdx;
+    }
+
+    // note: it is a shame that we invoke preceding() *in addition to* following(); BI's are sometimes expensive.
+
+    // Find closest break < target
+    final int beforeIdx = baseIter.preceding(targetIdx); // or could do baseIter.previous() but we hope the BI implements preceding()
+    if (beforeIdx <= followingIdx) { // too far back
+      return moveToBreak(afterIdx);
+    }
+
+    if (targetIdx - beforeIdx <= afterIdx - targetIdx) {
+      return beforeIdx;
+    }
+    return moveToBreak(afterIdx);
+  }
+
+  private int moveToBreak(int idx) { // precondition: idx is a known break
+    // bi.isBoundary(idx) has side-effect of moving the position.  Not obvious!
+    //boolean moved = baseIter.isBoundary(idx); // probably not particularly expensive
+    //assert moved && current() == idx;
+
+    // TODO fix: Would prefer to do "- 1" instead of "- 2" but CustomSeparatorBreakIterator has a bug.
+    int current = baseIter.following(idx - 2);
+    assert current == idx : "following() didn't move us to the expected index.";
+    return idx;
+  }
+
+  // called at start of new Passage given first word start offset
+  @Override
+  public int preceding(int offset) {
+    return baseIter.preceding(offset); // no change needed
+  }
+
+  @Override
+  public boolean isBoundary(int offset) {
+    assert false : "Not supported";
+    return baseIter.isBoundary(offset);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
index d64b96e..3efb694 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
@@ -171,6 +171,7 @@ public class Passage {
 
   /** @lucene.internal */
   public void setEndOffset(int endOffset) {
+    assert startOffset <= endOffset;
     this.endOffset = endOffset;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea499895/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java
new file mode 100644
index 0000000..42d2bf6
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.lucene.search.uhighlight;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.postingshighlight.CustomSeparatorBreakIterator;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.QueryBuilder;
+
+public class LengthGoalBreakIteratorTest extends LuceneTestCase {
+  private static final String FIELD = "body";
+
+  // We test LengthGoalBreakIterator as it is used by the UnifiedHighlighter instead of directly, because it is
+  //  not a general purpose BreakIterator.  A unit test of it directly wouldn't give as much confidence.
+
+  private final Analyzer analyzer =
+      new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
+
+  // We do a '.' BreakIterator and test varying the length goal.
+  //                      0         1
+  //                      01234567890123456789
+  final String content = "Aa bb. Cc dd. Ee ff";
+
+  public void testTargetLen() throws IOException {
+    // "goal" means target length goal to find closest break
+
+    // at first word:
+    Query query = query("aa");
+    assertEquals("almost two sent",
+        "<b>Aa</b> bb.", highlightClosestToLen(content, query, 9));
+    assertEquals( "barely two sent",
+        "<b>Aa</b> bb. Cc dd.", highlightClosestToLen(content, query, 10));
+    assertEquals("long goal",
+        "<b>Aa</b> bb. Cc dd. Ee ff", highlightClosestToLen(content, query, 17 + random().nextInt(20)));
+
+    // at some word not at start of passage
+    query = query("dd");
+    assertEquals("short goal",
+        " Cc <b>dd</b>.", highlightClosestToLen(content, query, random().nextInt(5)));
+    assertEquals("almost two sent",
+        " Cc <b>dd</b>.", highlightClosestToLen(content, query, 10));
+    assertEquals("barely two sent",
+        " Cc <b>dd</b>. Ee ff", highlightClosestToLen(content, query, 11));
+    assertEquals("long goal",
+        " Cc <b>dd</b>. Ee ff", highlightClosestToLen(content, query, 12 + random().nextInt(20)));
+  }
+
+  public void testMinLen() throws IOException {
+    // minLen mode is simpler than targetLen... just test a few cases
+
+    Query query = query("dd");
+    assertEquals("almost two sent",
+        " Cc <b>dd</b>.", highlightMinLen(content, query, 6));
+    assertEquals("barely two sent",
+        " Cc <b>dd</b>. Ee ff", highlightMinLen(content, query, 7));
+  }
+
+  public void testDefaultSummaryTargetLen() throws IOException {
+    Query query = query("zz");
+    assertEquals("Aa bb.",
+        highlightClosestToLen(content, query, random().nextInt(10))); // < 10
+    assertEquals("Aa bb. Cc dd.",
+        highlightClosestToLen(content, query, 10 + 6)); // cusp of adding 3rd sentence
+    assertEquals("Aa bb. Cc dd. Ee ff",
+        highlightClosestToLen(content, query, 17 + random().nextInt(20))); // >= 14
+  }
+
+  private Query query(String qStr) {
+    return new QueryBuilder(analyzer).createBooleanQuery(FIELD, qStr);
+  }
+
+  private String highlightClosestToLen(String content, Query query, int lengthGoal) throws IOException {
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer);
+    highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createClosestToLength(new CustomSeparatorBreakIterator('.'), lengthGoal));
+    return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString();
+  }
+
+  private String highlightMinLen(String content, Query query, int lengthGoal) throws IOException {
+    // differs from above only by "createMinLength"
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer);
+    highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createMinLength(new CustomSeparatorBreakIterator('.'), lengthGoal));
+    return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString();
+  }
+}
\ No newline at end of file


[17/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9854 Relax test assertions.

Posted by ho...@apache.org.
SOLR-9854 Relax test assertions.


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

Branch: refs/heads/jira/solr-5944
Commit: b9827bcba9ee4e1b0dea8a18c6efe4322a400b09
Parents: b017731
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jan 9 11:16:40 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 9 11:16:40 2017 +0100

----------------------------------------------------------------------
 .../src/test/org/apache/solr/update/SolrIndexMetricsTest.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b9827bcb/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
index e17b1bd..4f5ea69 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
@@ -67,13 +67,13 @@ public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
 
     // check basic index meters
     Timer timer = (Timer)metrics.get("INDEX.merge.minor");
-    assertEquals("minorMerge: " + timer.getCount(), 4, timer.getCount());
+    assertTrue("minorMerge: " + timer.getCount(), timer.getCount() >= 3);
     timer = (Timer)metrics.get("INDEX.merge.major");
     assertEquals("majorMerge: " + timer.getCount(), 0, timer.getCount());
     Meter meter = (Meter)metrics.get("INDEX.merge.major.docs");
     assertEquals("majorMergeDocs: " + meter.getCount(), 0, meter.getCount());
     meter = (Meter)metrics.get("INDEX.flush");
-    assertEquals("flush: " + meter.getCount(), 19, meter.getCount());
+    assertTrue("flush: " + meter.getCount(), meter.getCount() > 10);
 
     // check basic directory meters
     meter = (Meter)metrics.get("DIRECTORY.total.reads");


[14/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9902: Fix move impl.

Posted by ho...@apache.org.
SOLR-9902: Fix move impl.


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

Branch: refs/heads/jira/solr-5944
Commit: 8bc151d1c61932dda26c682cf2281535f0c36058
Parents: 25290ab
Author: markrmiller <ma...@apache.org>
Authored: Sun Jan 8 10:22:42 2017 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Sun Jan 8 10:22:55 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                                  | 2 +-
 .../src/java/org/apache/solr/core/StandardDirectoryFactory.java   | 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bc151d1/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c18381e..11151d7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -248,7 +248,7 @@ Optimizations
   resulting in less produced garbage and 5-7% better performance.
   (yonik)
 
-* SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation. (Mark Miller)
+* SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation. (Mark Miller, Mike Drob)
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bc151d1/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index 37c15ed..1bc4914 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -131,13 +131,14 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
     if (baseFromDir instanceof FSDirectory && baseToDir instanceof FSDirectory) {
   
       Path path1 = ((FSDirectory) baseFromDir).getDirectory().toAbsolutePath();
-      Path path2 = ((FSDirectory) baseFromDir).getDirectory().toAbsolutePath();
+      Path path2 = ((FSDirectory) baseToDir).getDirectory().toAbsolutePath();
       
       try {
         Files.move(path1.resolve(fileName), path2.resolve(fileName), StandardCopyOption.ATOMIC_MOVE);
       } catch (AtomicMoveNotSupportedException e) {
         Files.move(path1.resolve(fileName), path2.resolve(fileName));
       }
+      return;
     }
 
     super.move(fromDir, toDir, fileName, ioContext);


[12/50] [abbrv] lucene-solr:jira/solr-5944: promote this test case to core

Posted by ho...@apache.org.
promote this test case to core


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

Branch: refs/heads/jira/solr-5944
Commit: f985fcaa23cb9ef96ed823e5bf7957049e0d9461
Parents: e64111c
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 8 06:38:37 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 8 06:38:37 2017 -0500

----------------------------------------------------------------------
 .../lucene/analysis/TestGraphTokenizers.java    | 600 +++++++++++++++++++
 .../lucene/analysis/TestGraphTokenizers.java    | 600 -------------------
 2 files changed, 600 insertions(+), 600 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f985fcaa/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
new file mode 100644
index 0000000..8899dd1
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
@@ -0,0 +1,600 @@
+/*
+ * 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.lucene.analysis;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
+public class TestGraphTokenizers extends BaseTokenStreamTestCase {
+
+  // Makes a graph TokenStream from the string; separate
+  // positions with single space, multiple tokens at the same
+  // position with /, and add optional position length with
+  // :.  EG "a b c" is a simple chain, "a/x b c" adds 'x'
+  // over 'a' at position 0 with posLen=1, "a/x:3 b c" adds
+  // 'x' over a with posLen=3.  Tokens are in normal-form!
+  // So, offsets are computed based on the first token at a
+  // given position.  NOTE: each token must be a single
+  // character!  We assume this when computing offsets...
+  
+  // NOTE: all input tokens must be length 1!!!  This means
+  // you cannot turn on MockCharFilter when random
+  // testing...
+
+  private static class GraphTokenizer extends Tokenizer {
+    private List<Token> tokens;
+    private int upto;
+    private int inputLength;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+    private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      tokens = null;
+      upto = 0;
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (tokens == null) {
+        fillTokens();
+      }
+      //System.out.println("graphTokenizer: incr upto=" + upto + " vs " + tokens.size());
+      if (upto == tokens.size()) {
+        //System.out.println("  END @ " + tokens.size());
+        return false;
+      } 
+      final Token t = tokens.get(upto++);
+      //System.out.println("  return token=" + t);
+      clearAttributes();
+      termAtt.append(t.toString());
+      offsetAtt.setOffset(t.startOffset(), t.endOffset());
+      posIncrAtt.setPositionIncrement(t.getPositionIncrement());
+      posLengthAtt.setPositionLength(t.getPositionLength());
+      return true;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      // NOTE: somewhat... hackish, but we need this to
+      // satisfy BTSTC:
+      final int lastOffset;
+      if (tokens != null && !tokens.isEmpty()) {
+        lastOffset = tokens.get(tokens.size()-1).endOffset();
+      } else {
+        lastOffset = 0;
+      }
+      offsetAtt.setOffset(correctOffset(lastOffset),
+                          correctOffset(inputLength));
+    }
+
+    private void fillTokens() throws IOException {
+      final StringBuilder sb = new StringBuilder();
+      final char[] buffer = new char[256];
+      while (true) {
+        final int count = input.read(buffer);
+        if (count == -1) {
+          break;
+        }
+        sb.append(buffer, 0, count);
+        //System.out.println("got count=" + count);
+      }
+      //System.out.println("fillTokens: " + sb);
+
+      inputLength = sb.length();
+
+      final String[] parts = sb.toString().split(" ");
+
+      tokens = new ArrayList<>();
+      int pos = 0;
+      int maxPos = -1;
+      int offset = 0;
+      //System.out.println("again");
+      for(String part : parts) {
+        final String[] overlapped = part.split("/");
+        boolean firstAtPos = true;
+        int minPosLength = Integer.MAX_VALUE;
+        for(String part2 : overlapped) {
+          final int colonIndex = part2.indexOf(':');
+          final String token;
+          final int posLength;
+          if (colonIndex != -1) {
+            token = part2.substring(0, colonIndex);
+            posLength = Integer.parseInt(part2.substring(1+colonIndex));
+          } else {
+            token = part2;
+            posLength = 1;
+          }
+          maxPos = Math.max(maxPos, pos + posLength);
+          minPosLength = Math.min(minPosLength, posLength);
+          final Token t = new Token(token, offset, offset + 2*posLength - 1);
+          t.setPositionLength(posLength);
+          t.setPositionIncrement(firstAtPos ? 1:0);
+          firstAtPos = false;
+          //System.out.println("  add token=" + t + " startOff=" + t.startOffset() + " endOff=" + t.endOffset());
+          tokens.add(t);
+        }
+        pos += minPosLength;
+        offset = 2 * pos;
+      }
+      assert maxPos <= pos: "input string mal-formed: posLength>1 tokens hang over the end";
+    }
+  }
+
+  public void testMockGraphTokenFilterBasic() throws Exception {
+
+    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkAnalysisConsistency(random(), a, false, "a b c d e f g h i j k");
+    }
+  }
+
+  public void testMockGraphTokenFilterOnGraphInput() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new GraphTokenizer();
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      checkAnalysisConsistency(random(), a, false, "a/x:3 c/y:2 d e f/z:4 g h i j k");
+    }
+  }
+
+  // Just deletes (leaving hole) token 'a':
+  private final static class RemoveATokens extends TokenFilter {
+    private int pendingPosInc;
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+
+    public RemoveATokens(TokenStream in) {
+      super(in);
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      pendingPosInc = 0;
+    }
+
+    @Override
+    public void end() throws IOException {
+      super.end();
+      posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      while (true) {
+        final boolean gotOne = input.incrementToken();
+        if (!gotOne) {
+          return false;
+        } else if (termAtt.toString().equals("a")) {
+          pendingPosInc += posIncAtt.getPositionIncrement();
+        } else {
+          posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
+          pendingPosInc = 0;
+          return true;
+        }
+      }
+    }
+  }
+
+  public void testMockGraphTokenFilterBeforeHoles() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            final TokenStream t3 = new RemoveATokens(t2);
+            return new TokenStreamComponents(t, t3);
+          }
+        };
+
+      Random random = random();
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
+    }
+  }
+
+  public void testMockGraphTokenFilterAfterHoles() throws Exception {
+    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new RemoveATokens(t);
+            final TokenStream t3 = new MockGraphTokenFilter(random(), t2);
+            return new TokenStreamComponents(t, t3);
+          }
+        };
+
+      Random random = random();
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
+      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
+    }
+  }
+
+  public void testMockGraphTokenFilterRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  // Two MockGraphTokenFilters
+  public void testDoubleMockGraphTokenFilterRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  public void testMockGraphTokenFilterBeforeHolesRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
+            final TokenStream t2 = new MockHoleInjectingTokenFilter(random(), t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  public void testMockGraphTokenFilterAfterHolesRandom() throws Exception {
+    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter);
+      }
+
+      // Make new analyzer each time, because MGTF has fixed
+      // seed:
+      final Analyzer a = new Analyzer() {
+          @Override
+          protected TokenStreamComponents createComponents(String fieldName) {
+            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
+            final TokenStream t1 = new MockHoleInjectingTokenFilter(random(), t);
+            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
+            return new TokenStreamComponents(t, t2);
+          }
+        };
+      
+      Random random = random();
+      checkRandomData(random, a, 5, atLeast(100));
+    }
+  }
+
+  private static Token token(String term, int posInc, int posLength) {
+    final Token t = new Token(term, 0, 0);
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  private static Token token(String term, int posInc, int posLength, int startOffset, int endOffset) {
+    final Token t = new Token(term, startOffset, endOffset);
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  public void testSingleToken() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+      });
+    assertSameLanguage(s2a("abc"), ts);
+  }
+
+  public void testMultipleHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("b", 3, 1),
+      });
+    assertSameLanguage(join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")), ts);
+  }
+
+  public void testSynOverMultipleHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("x", 0, 3),
+        token("b", 3, 1),
+      });
+    final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
+    final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  // for debugging!
+  /*
+  private static void toDot(Automaton a) throws IOException {
+    final String s = a.toDot();
+    Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
+    w.write(s);
+    w.close();
+    System.out.println("TEST: saved to /x/tmp/out.dot");
+  }
+  */
+
+  private static final Automaton SEP_A = Automata.makeChar(TokenStreamToAutomaton.POS_SEP);
+  private static final Automaton HOLE_A = Automata.makeChar(TokenStreamToAutomaton.HOLE);
+
+  private Automaton join(String ... strings) {
+    List<Automaton> as = new ArrayList<>();
+    for(String s : strings) {
+      as.add(s2a(s));
+      as.add(SEP_A);
+    }
+    as.remove(as.size()-1);
+    return Operations.concatenate(as);
+  }
+
+  private Automaton join(Automaton ... as) {
+    return Operations.concatenate(Arrays.asList(as));
+  }
+
+  private Automaton s2a(String s) {
+    return Automata.makeString(s);
+  }
+
+  public void testTwoTokens() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 1, 1),
+      });
+    assertSameLanguage(join("abc", "def"), ts);
+  }
+
+  public void testHole() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("def", 2, 1),
+      });
+    assertSameLanguage(join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def")), ts);
+  }
+
+  public void testOverlappedTokensSausage() throws Exception {
+
+    // Two tokens on top of each other (sausage):
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 1)
+      });
+    final Automaton a1 = s2a("abc");
+    final Automaton a2 = s2a("xyz");
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  public void testOverlappedTokensLattice() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 2),
+        token("def", 1, 1),
+      });
+    final Automaton a1 = s2a("xyz");
+    final Automaton a2 = join("abc", "def");
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  public void testSynOverHole() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("X", 0, 2),
+        token("b", 2, 1),
+      });
+    final Automaton a1 = Operations.union(join(s2a("a"), SEP_A, HOLE_A), s2a("X"));
+    final Automaton expected = Operations.concatenate(a1, join(SEP_A, s2a("b")));
+    assertSameLanguage(expected, ts);
+  }
+
+  public void testSynOverHole2() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("xyz", 1, 1),
+        token("abc", 0, 3),
+        token("def", 2, 1),
+      });
+    final Automaton expected = Operations.union(
+      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")), s2a("abc"));
+    assertSameLanguage(expected, ts);
+  }
+
+  public void testOverlappedTokensLattice2() throws Exception {
+
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 0, 3),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    final Automaton a1 = s2a("xyz");
+    final Automaton a2 = join("abc", "def", "ghi");
+    assertSameLanguage(Operations.union(a1, a2), ts);
+  }
+
+  public void testToDot() throws Exception {
+    final TokenStream ts = new CannedTokenStream(new Token[] {token("abc", 1, 1, 0, 4)});
+    StringWriter w = new StringWriter();
+    new TokenStreamToDot("abcd", ts, new PrintWriter(w)).toDot();
+    assertTrue(w.toString().indexOf("abc / abcd") != -1);
+  }
+
+  public void testStartsWithHole() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 2, 1),
+      });
+    assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc")), ts);
+  }
+
+  // TODO: testEndsWithHole... but we need posInc to set in TS.end()
+
+  public void testSynHangingOverEnd() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("a", 1, 1),
+        token("X", 0, 10),
+      });
+    assertSameLanguage(Operations.union(s2a("a"), s2a("X")), ts);
+  }
+
+  private void assertSameLanguage(Automaton expected, TokenStream ts) throws IOException {
+    assertSameLanguage(expected, new TokenStreamToAutomaton().toAutomaton(ts));
+  }
+
+  private void assertSameLanguage(Automaton expected, Automaton actual) {
+    assertTrue(Operations.sameLanguage(
+      Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
+      Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
+  }
+
+  public void testTokenStreamGraphWithHoles() throws Exception {
+    final TokenStream ts = new CannedTokenStream(
+      new Token[] {
+        token("abc", 1, 1),
+        token("xyz", 1, 8),
+        token("def", 1, 1),
+        token("ghi", 1, 1),
+      });
+    assertSameLanguage(Operations.union(join(s2a("abc"), SEP_A, s2a("xyz")),
+                                        join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"), SEP_A, s2a("ghi"))), ts);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f985fcaa/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java b/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
deleted file mode 100644
index 8899dd1..0000000
--- a/lucene/test-framework/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
+++ /dev/null
@@ -1,600 +0,0 @@
-/*
- * 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.lucene.analysis;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.Operations;
-
-import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
-
-public class TestGraphTokenizers extends BaseTokenStreamTestCase {
-
-  // Makes a graph TokenStream from the string; separate
-  // positions with single space, multiple tokens at the same
-  // position with /, and add optional position length with
-  // :.  EG "a b c" is a simple chain, "a/x b c" adds 'x'
-  // over 'a' at position 0 with posLen=1, "a/x:3 b c" adds
-  // 'x' over a with posLen=3.  Tokens are in normal-form!
-  // So, offsets are computed based on the first token at a
-  // given position.  NOTE: each token must be a single
-  // character!  We assume this when computing offsets...
-  
-  // NOTE: all input tokens must be length 1!!!  This means
-  // you cannot turn on MockCharFilter when random
-  // testing...
-
-  private static class GraphTokenizer extends Tokenizer {
-    private List<Token> tokens;
-    private int upto;
-    private int inputLength;
-
-    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-    private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
-    private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
-    private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
-
-    @Override
-    public void reset() throws IOException {
-      super.reset();
-      tokens = null;
-      upto = 0;
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (tokens == null) {
-        fillTokens();
-      }
-      //System.out.println("graphTokenizer: incr upto=" + upto + " vs " + tokens.size());
-      if (upto == tokens.size()) {
-        //System.out.println("  END @ " + tokens.size());
-        return false;
-      } 
-      final Token t = tokens.get(upto++);
-      //System.out.println("  return token=" + t);
-      clearAttributes();
-      termAtt.append(t.toString());
-      offsetAtt.setOffset(t.startOffset(), t.endOffset());
-      posIncrAtt.setPositionIncrement(t.getPositionIncrement());
-      posLengthAtt.setPositionLength(t.getPositionLength());
-      return true;
-    }
-
-    @Override
-    public void end() throws IOException {
-      super.end();
-      // NOTE: somewhat... hackish, but we need this to
-      // satisfy BTSTC:
-      final int lastOffset;
-      if (tokens != null && !tokens.isEmpty()) {
-        lastOffset = tokens.get(tokens.size()-1).endOffset();
-      } else {
-        lastOffset = 0;
-      }
-      offsetAtt.setOffset(correctOffset(lastOffset),
-                          correctOffset(inputLength));
-    }
-
-    private void fillTokens() throws IOException {
-      final StringBuilder sb = new StringBuilder();
-      final char[] buffer = new char[256];
-      while (true) {
-        final int count = input.read(buffer);
-        if (count == -1) {
-          break;
-        }
-        sb.append(buffer, 0, count);
-        //System.out.println("got count=" + count);
-      }
-      //System.out.println("fillTokens: " + sb);
-
-      inputLength = sb.length();
-
-      final String[] parts = sb.toString().split(" ");
-
-      tokens = new ArrayList<>();
-      int pos = 0;
-      int maxPos = -1;
-      int offset = 0;
-      //System.out.println("again");
-      for(String part : parts) {
-        final String[] overlapped = part.split("/");
-        boolean firstAtPos = true;
-        int minPosLength = Integer.MAX_VALUE;
-        for(String part2 : overlapped) {
-          final int colonIndex = part2.indexOf(':');
-          final String token;
-          final int posLength;
-          if (colonIndex != -1) {
-            token = part2.substring(0, colonIndex);
-            posLength = Integer.parseInt(part2.substring(1+colonIndex));
-          } else {
-            token = part2;
-            posLength = 1;
-          }
-          maxPos = Math.max(maxPos, pos + posLength);
-          minPosLength = Math.min(minPosLength, posLength);
-          final Token t = new Token(token, offset, offset + 2*posLength - 1);
-          t.setPositionLength(posLength);
-          t.setPositionIncrement(firstAtPos ? 1:0);
-          firstAtPos = false;
-          //System.out.println("  add token=" + t + " startOff=" + t.startOffset() + " endOff=" + t.endOffset());
-          tokens.add(t);
-        }
-        pos += minPosLength;
-        offset = 2 * pos;
-      }
-      assert maxPos <= pos: "input string mal-formed: posLength>1 tokens hang over the end";
-    }
-  }
-
-  public void testMockGraphTokenFilterBasic() throws Exception {
-
-    for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      checkAnalysisConsistency(random(), a, false, "a b c d e f g h i j k");
-    }
-  }
-
-  public void testMockGraphTokenFilterOnGraphInput() throws Exception {
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new GraphTokenizer();
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      checkAnalysisConsistency(random(), a, false, "a/x:3 c/y:2 d e f/z:4 g h i j k");
-    }
-  }
-
-  // Just deletes (leaving hole) token 'a':
-  private final static class RemoveATokens extends TokenFilter {
-    private int pendingPosInc;
-
-    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
-
-    public RemoveATokens(TokenStream in) {
-      super(in);
-    }
-
-    @Override
-    public void reset() throws IOException {
-      super.reset();
-      pendingPosInc = 0;
-    }
-
-    @Override
-    public void end() throws IOException {
-      super.end();
-      posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      while (true) {
-        final boolean gotOne = input.incrementToken();
-        if (!gotOne) {
-          return false;
-        } else if (termAtt.toString().equals("a")) {
-          pendingPosInc += posIncAtt.getPositionIncrement();
-        } else {
-          posIncAtt.setPositionIncrement(pendingPosInc + posIncAtt.getPositionIncrement());
-          pendingPosInc = 0;
-          return true;
-        }
-      }
-    }
-  }
-
-  public void testMockGraphTokenFilterBeforeHoles() throws Exception {
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            final TokenStream t3 = new RemoveATokens(t2);
-            return new TokenStreamComponents(t, t3);
-          }
-        };
-
-      Random random = random();
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
-    }
-  }
-
-  public void testMockGraphTokenFilterAfterHoles() throws Exception {
-    for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new RemoveATokens(t);
-            final TokenStream t3 = new MockGraphTokenFilter(random(), t2);
-            return new TokenStreamComponents(t, t3);
-          }
-        };
-
-      Random random = random();
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "x y a b c d e f g h i j k");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a");
-      checkAnalysisConsistency(random, a, false, "a b c d e f g h i j k a x y");
-    }
-  }
-
-  public void testMockGraphTokenFilterRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  // Two MockGraphTokenFilters
-  public void testDoubleMockGraphTokenFilterRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  public void testMockGraphTokenFilterBeforeHolesRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t1 = new MockGraphTokenFilter(random(), t);
-            final TokenStream t2 = new MockHoleInjectingTokenFilter(random(), t1);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  public void testMockGraphTokenFilterAfterHolesRandom() throws Exception {
-    for(int iter=0;iter<3*RANDOM_MULTIPLIER;iter++) {
-
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter);
-      }
-
-      // Make new analyzer each time, because MGTF has fixed
-      // seed:
-      final Analyzer a = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName) {
-            final Tokenizer t = new MockTokenizer(MockTokenizer.WHITESPACE, false);
-            final TokenStream t1 = new MockHoleInjectingTokenFilter(random(), t);
-            final TokenStream t2 = new MockGraphTokenFilter(random(), t1);
-            return new TokenStreamComponents(t, t2);
-          }
-        };
-      
-      Random random = random();
-      checkRandomData(random, a, 5, atLeast(100));
-    }
-  }
-
-  private static Token token(String term, int posInc, int posLength) {
-    final Token t = new Token(term, 0, 0);
-    t.setPositionIncrement(posInc);
-    t.setPositionLength(posLength);
-    return t;
-  }
-
-  private static Token token(String term, int posInc, int posLength, int startOffset, int endOffset) {
-    final Token t = new Token(term, startOffset, endOffset);
-    t.setPositionIncrement(posInc);
-    t.setPositionLength(posLength);
-    return t;
-  }
-
-  public void testSingleToken() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-      });
-    assertSameLanguage(s2a("abc"), ts);
-  }
-
-  public void testMultipleHoles() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("b", 3, 1),
-      });
-    assertSameLanguage(join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")), ts);
-  }
-
-  public void testSynOverMultipleHoles() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("x", 0, 3),
-        token("b", 3, 1),
-      });
-    final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
-    final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  // for debugging!
-  /*
-  private static void toDot(Automaton a) throws IOException {
-    final String s = a.toDot();
-    Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
-    w.write(s);
-    w.close();
-    System.out.println("TEST: saved to /x/tmp/out.dot");
-  }
-  */
-
-  private static final Automaton SEP_A = Automata.makeChar(TokenStreamToAutomaton.POS_SEP);
-  private static final Automaton HOLE_A = Automata.makeChar(TokenStreamToAutomaton.HOLE);
-
-  private Automaton join(String ... strings) {
-    List<Automaton> as = new ArrayList<>();
-    for(String s : strings) {
-      as.add(s2a(s));
-      as.add(SEP_A);
-    }
-    as.remove(as.size()-1);
-    return Operations.concatenate(as);
-  }
-
-  private Automaton join(Automaton ... as) {
-    return Operations.concatenate(Arrays.asList(as));
-  }
-
-  private Automaton s2a(String s) {
-    return Automata.makeString(s);
-  }
-
-  public void testTwoTokens() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("def", 1, 1),
-      });
-    assertSameLanguage(join("abc", "def"), ts);
-  }
-
-  public void testHole() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("def", 2, 1),
-      });
-    assertSameLanguage(join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def")), ts);
-  }
-
-  public void testOverlappedTokensSausage() throws Exception {
-
-    // Two tokens on top of each other (sausage):
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 0, 1)
-      });
-    final Automaton a1 = s2a("abc");
-    final Automaton a2 = s2a("xyz");
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  public void testOverlappedTokensLattice() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 0, 2),
-        token("def", 1, 1),
-      });
-    final Automaton a1 = s2a("xyz");
-    final Automaton a2 = join("abc", "def");
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  public void testSynOverHole() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("X", 0, 2),
-        token("b", 2, 1),
-      });
-    final Automaton a1 = Operations.union(join(s2a("a"), SEP_A, HOLE_A), s2a("X"));
-    final Automaton expected = Operations.concatenate(a1, join(SEP_A, s2a("b")));
-    assertSameLanguage(expected, ts);
-  }
-
-  public void testSynOverHole2() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("xyz", 1, 1),
-        token("abc", 0, 3),
-        token("def", 2, 1),
-      });
-    final Automaton expected = Operations.union(
-      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")), s2a("abc"));
-    assertSameLanguage(expected, ts);
-  }
-
-  public void testOverlappedTokensLattice2() throws Exception {
-
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 0, 3),
-        token("def", 1, 1),
-        token("ghi", 1, 1),
-      });
-    final Automaton a1 = s2a("xyz");
-    final Automaton a2 = join("abc", "def", "ghi");
-    assertSameLanguage(Operations.union(a1, a2), ts);
-  }
-
-  public void testToDot() throws Exception {
-    final TokenStream ts = new CannedTokenStream(new Token[] {token("abc", 1, 1, 0, 4)});
-    StringWriter w = new StringWriter();
-    new TokenStreamToDot("abcd", ts, new PrintWriter(w)).toDot();
-    assertTrue(w.toString().indexOf("abc / abcd") != -1);
-  }
-
-  public void testStartsWithHole() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 2, 1),
-      });
-    assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc")), ts);
-  }
-
-  // TODO: testEndsWithHole... but we need posInc to set in TS.end()
-
-  public void testSynHangingOverEnd() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("a", 1, 1),
-        token("X", 0, 10),
-      });
-    assertSameLanguage(Operations.union(s2a("a"), s2a("X")), ts);
-  }
-
-  private void assertSameLanguage(Automaton expected, TokenStream ts) throws IOException {
-    assertSameLanguage(expected, new TokenStreamToAutomaton().toAutomaton(ts));
-  }
-
-  private void assertSameLanguage(Automaton expected, Automaton actual) {
-    assertTrue(Operations.sameLanguage(
-      Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
-      Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
-  }
-
-  public void testTokenStreamGraphWithHoles() throws Exception {
-    final TokenStream ts = new CannedTokenStream(
-      new Token[] {
-        token("abc", 1, 1),
-        token("xyz", 1, 8),
-        token("def", 1, 1),
-        token("ghi", 1, 1),
-      });
-    assertSameLanguage(Operations.union(join(s2a("abc"), SEP_A, s2a("xyz")),
-                                        join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"), SEP_A, s2a("ghi"))), ts);
-  }
-}


[36/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9947: Fix CloudSolrClientTest.testNonRetryableRequests failure

Posted by ho...@apache.org.
SOLR-9947: Fix CloudSolrClientTest.testNonRetryableRequests failure


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

Branch: refs/heads/jira/solr-5944
Commit: dee7709dd86c64529ee0455d05805ab41b34c736
Parents: 6c6c077
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Jan 12 20:49:38 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Jan 12 20:49:38 2017 +0530

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/impl/CloudSolrClientTest.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dee7709d/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index cbaaa1b..1698075 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@ -431,7 +431,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
         for (String adminPath : adminPathToMbean.keySet()) {
           long errorsBefore = 0;
           for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
-            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "QUERY", adminPathToMbean.get(adminPath), true);
+            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "ADMIN", adminPathToMbean.get(adminPath), true);
             errorsBefore += numRequests;
             log.info("Found {} requests to {} on {}", numRequests, adminPath, runner.getBaseUrl());
           }
@@ -448,7 +448,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
           }
           long errorsAfter = 0;
           for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
-            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "QUERY", adminPathToMbean.get(adminPath), true);
+            Long numRequests = getNumRequests(runner.getBaseUrl().toString(), "foo", "ADMIN", adminPathToMbean.get(adminPath), true);
             errorsAfter += numRequests;
             log.info("Found {} requests to {} on {}", numRequests, adminPath, runner.getBaseUrl());
           }


[45/50] [abbrv] lucene-solr:jira/solr-5944: Fix compile warning in Lucene Core; make Eclipse happy by moving Java-1-like-side-by-side class to separate file

Posted by ho...@apache.org.
Fix compile warning in Lucene Core; make Eclipse happy by moving Java-1-like-side-by-side class to separate file


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

Branch: refs/heads/jira/solr-5944
Commit: 60d4a554ecd0ac00bf1cd84041f19dc3f8926cf3
Parents: 4eafdb3
Author: Uwe Schindler <us...@apache.org>
Authored: Sat Jan 14 19:46:59 2017 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Sat Jan 14 19:46:59 2017 +0100

----------------------------------------------------------------------
 .../lucene/search/MultiCollectorManager.java    |   6 +-
 .../solr/highlight/LuceneRegexFragmenter.java   | 217 +++++++++++++++++++
 .../apache/solr/highlight/RegexFragmenter.java  | 196 -----------------
 3 files changed, 221 insertions(+), 198 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60d4a554/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
index 9549cde..a8c6d1c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
@@ -31,8 +31,10 @@ public class MultiCollectorManager implements CollectorManager<MultiCollectorMan
 
   final private CollectorManager<Collector, ?>[] collectorManagers;
 
-  public MultiCollectorManager(final CollectorManager... collectorManagers) {
-    this.collectorManagers = collectorManagers;
+  @SafeVarargs
+  @SuppressWarnings({"varargs", "unchecked"})
+  public MultiCollectorManager(final CollectorManager<? extends Collector, ?>... collectorManagers) {
+    this.collectorManagers = (CollectorManager[]) collectorManagers;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60d4a554/solr/core/src/java/org/apache/solr/highlight/LuceneRegexFragmenter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/LuceneRegexFragmenter.java b/solr/core/src/java/org/apache/solr/highlight/LuceneRegexFragmenter.java
new file mode 100644
index 0000000..0dc3340
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/highlight/LuceneRegexFragmenter.java
@@ -0,0 +1,217 @@
+/*
+ * 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.highlight;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.search.highlight.Fragmenter;
+
+/**
+ * Fragmenter that tries to produce snippets that "look" like a regular 
+ * expression.
+ *
+ * NOTE: the default for <code>maxAnalyzedChars</code> is much lower for this 
+ * fragmenter.  After this limit is exhausted, fragments are produced in the
+ * same way as <code>GapFragmenter</code>
+ */
+class LuceneRegexFragmenter implements Fragmenter
+{
+  // ** defaults
+  public static final int DEFAULT_FRAGMENT_SIZE = 70;
+  public static final int DEFAULT_INCREMENT_GAP = 50;
+  public static final float DEFAULT_SLOP = 0.6f;
+  public static final int DEFAULT_MAX_ANALYZED_CHARS = 10000;
+
+  // ** settings
+
+  // desired length of fragments, in characters
+  protected int targetFragChars;
+  // increment gap which indicates a new fragment should occur 
+  // (often due to multi-valued fields)
+  protected int incrementGapThreshold;
+  // factor by which we are allowed to bend the frag size (larger or smaller)
+  protected float slop;
+  // analysis limit (ensures we don't waste too much time on long fields)
+  protected int maxAnalyzedChars;
+  // default desirable pattern for text fragments.
+  protected Pattern textRE;
+  
+
+  // ** state
+  protected int currentNumFrags;
+  protected int currentOffset;
+  protected int targetOffset;
+  protected int[] hotspots;
+
+  private PositionIncrementAttribute posIncAtt;
+  private OffsetAttribute offsetAtt;
+
+  // ** other
+  // note: could dynamically change size of sentences extracted to match
+  // target frag size
+  public static final String 
+    DEFAULT_PATTERN_RAW = "[-\\w ,\\n\"']{20,200}";
+  public static final Pattern 
+    DEFAULT_PATTERN = Pattern.compile(DEFAULT_PATTERN_RAW);
+
+
+  public LuceneRegexFragmenter() {
+    this(DEFAULT_FRAGMENT_SIZE, 
+         DEFAULT_INCREMENT_GAP,
+         DEFAULT_SLOP,
+         DEFAULT_MAX_ANALYZED_CHARS);
+  }
+  public LuceneRegexFragmenter(int targetFragChars) {
+    this(targetFragChars, 
+         DEFAULT_INCREMENT_GAP,
+         DEFAULT_SLOP,
+         DEFAULT_MAX_ANALYZED_CHARS);
+  }
+
+  public LuceneRegexFragmenter(int targetFragChars, 
+                               int incrementGapThreshold,
+                               float slop,
+                               int maxAnalyzedChars ) {
+    this(targetFragChars, incrementGapThreshold, slop, maxAnalyzedChars,
+         DEFAULT_PATTERN);
+         
+  }
+
+  public LuceneRegexFragmenter(int targetFragChars, 
+                               int incrementGapThreshold,
+                               float slop,
+                               int maxAnalyzedChars,
+                               Pattern targetPattern) {
+    this.targetFragChars = targetFragChars;
+    this.incrementGapThreshold = incrementGapThreshold;    
+    this.slop = slop;
+    this.maxAnalyzedChars = maxAnalyzedChars;
+    this.textRE = targetPattern;
+  }
+  
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.search.highlight.TextFragmenter#start(java.lang.String)
+   */
+  @Override
+  public void start(String originalText, TokenStream tokenStream) {
+    currentNumFrags = 1;
+    currentOffset = 0;
+    addHotSpots(originalText);
+    posIncAtt = tokenStream.getAttribute(PositionIncrementAttribute.class);
+    offsetAtt = tokenStream.getAttribute(OffsetAttribute.class);
+  }
+
+  ////////////////////////////////////
+  // pre-analysis
+  ////////////////////////////////////
+
+  protected void addHotSpots(String text) {
+    //System.out.println("hot spotting");
+    ArrayList<Integer> temphs = new ArrayList<>(
+                              text.length() / targetFragChars);
+    Matcher match = textRE.matcher(text);
+    int cur = 0;
+    while(match.find() && cur < maxAnalyzedChars) {
+      int start=match.start(), end=match.end();
+      temphs.add(start);
+      temphs.add(end);
+      cur = end;
+      //System.out.println("Matched " + match.group());
+    }    
+    hotspots = new int[temphs.size()];
+    for(int i = 0; i < temphs.size(); i++) {
+      hotspots[i] = temphs.get(i);
+    }
+    // perhaps not necessary--I don't know if re matches are non-overlapping
+    Arrays.sort(hotspots);
+  }
+
+  ////////////////////////////////////
+  // fragmenting
+  ////////////////////////////////////
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.search.highlight.TextFragmenter#isNewFragment(org.apache.lucene.analysis.Token)
+   */
+  @Override
+  public boolean isNewFragment()
+  {
+    boolean isNewFrag = false;
+    int minFragLen = (int)((1.0f - slop)*targetFragChars);
+    int endOffset = offsetAtt.endOffset();
+    
+    // ** determin isNewFrag
+    if(posIncAtt.getPositionIncrement() > incrementGapThreshold) {
+      // large position gaps always imply new fragments
+      isNewFrag = true;
+
+    } else if(endOffset - currentOffset < minFragLen) {
+      // we're not in our range of flexibility
+      isNewFrag = false;
+
+    } else if(targetOffset > 0) {
+      // we've already decided on a target
+      isNewFrag = endOffset > targetOffset;
+
+    } else {
+      // we might be able to do something
+      int minOffset = currentOffset + minFragLen;
+      int maxOffset = (int)(currentOffset + (1.0f + slop)*targetFragChars);
+      int hotIndex;
+
+      // look for a close hotspot
+      hotIndex = Arrays.binarySearch(hotspots, endOffset);
+      if(hotIndex < 0) hotIndex = -hotIndex;
+      if(hotIndex >= hotspots.length) {
+        // no more hotspots in this input stream
+        targetOffset = currentOffset + targetFragChars;
+
+      } else if(hotspots[hotIndex] > maxOffset) {
+        // no hotspots within slop
+        targetOffset = currentOffset + targetFragChars;
+
+      } else {
+        // try to find hotspot in slop
+        int goal = hotspots[hotIndex];
+        while(goal < minOffset && hotIndex < hotspots.length) {
+          hotIndex++;
+          goal = hotspots[hotIndex];
+        }        
+        targetOffset = goal <= maxOffset ? goal : currentOffset + targetFragChars;
+      }
+
+      isNewFrag = endOffset > targetOffset;
+    }      
+      
+    // ** operate on isNewFrag
+    if(isNewFrag) {
+        currentNumFrags++;
+        currentOffset = endOffset;
+        targetOffset = -1;
+    }
+    return isNewFrag;
+  }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60d4a554/solr/core/src/java/org/apache/solr/highlight/RegexFragmenter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/RegexFragmenter.java b/solr/core/src/java/org/apache/solr/highlight/RegexFragmenter.java
index a80141e..b755b2d 100644
--- a/solr/core/src/java/org/apache/solr/highlight/RegexFragmenter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/RegexFragmenter.java
@@ -16,14 +16,8 @@
  */
 package org.apache.solr.highlight;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.search.highlight.Fragmenter;
 import org.apache.lucene.search.highlight.NullFragmenter;
 import org.apache.solr.common.params.HighlightParams;
@@ -94,193 +88,3 @@ public class RegexFragmenter extends HighlightingPluginBase implements SolrFragm
     return "RegexFragmenter (" + defaultPatternRaw + ")";
   }
 }
-
-
-/**
- * Fragmenter that tries to produce snippets that "look" like a regular 
- * expression.
- *
- * NOTE: the default for <code>maxAnalyzedChars</code> is much lower for this 
- * fragmenter.  After this limit is exhausted, fragments are produced in the
- * same way as <code>GapFragmenter</code>
- */
-class LuceneRegexFragmenter implements Fragmenter
-{
-  // ** defaults
-  public static final int DEFAULT_FRAGMENT_SIZE = 70;
-  public static final int DEFAULT_INCREMENT_GAP = 50;
-  public static final float DEFAULT_SLOP = 0.6f;
-  public static final int DEFAULT_MAX_ANALYZED_CHARS = 10000;
-
-  // ** settings
-
-  // desired length of fragments, in characters
-  protected int targetFragChars;
-  // increment gap which indicates a new fragment should occur 
-  // (often due to multi-valued fields)
-  protected int incrementGapThreshold;
-  // factor by which we are allowed to bend the frag size (larger or smaller)
-  protected float slop;
-  // analysis limit (ensures we don't waste too much time on long fields)
-  protected int maxAnalyzedChars;
-  // default desirable pattern for text fragments.
-  protected Pattern textRE;
-  
-
-  // ** state
-  protected int currentNumFrags;
-  protected int currentOffset;
-  protected int targetOffset;
-  protected int[] hotspots;
-
-  private PositionIncrementAttribute posIncAtt;
-  private OffsetAttribute offsetAtt;
-
-  // ** other
-  // note: could dynamically change size of sentences extracted to match
-  // target frag size
-  public static final String 
-    DEFAULT_PATTERN_RAW = "[-\\w ,\\n\"']{20,200}";
-  public static final Pattern 
-    DEFAULT_PATTERN = Pattern.compile(DEFAULT_PATTERN_RAW);
-
-
-  public LuceneRegexFragmenter() {
-    this(DEFAULT_FRAGMENT_SIZE, 
-         DEFAULT_INCREMENT_GAP,
-         DEFAULT_SLOP,
-         DEFAULT_MAX_ANALYZED_CHARS);
-  }
-  public LuceneRegexFragmenter(int targetFragChars) {
-    this(targetFragChars, 
-         DEFAULT_INCREMENT_GAP,
-         DEFAULT_SLOP,
-         DEFAULT_MAX_ANALYZED_CHARS);
-  }
-
-  public LuceneRegexFragmenter(int targetFragChars, 
-                               int incrementGapThreshold,
-                               float slop,
-                               int maxAnalyzedChars ) {
-    this(targetFragChars, incrementGapThreshold, slop, maxAnalyzedChars,
-         DEFAULT_PATTERN);
-         
-  }
-
-  public LuceneRegexFragmenter(int targetFragChars, 
-                               int incrementGapThreshold,
-                               float slop,
-                               int maxAnalyzedChars,
-                               Pattern targetPattern) {
-    this.targetFragChars = targetFragChars;
-    this.incrementGapThreshold = incrementGapThreshold;    
-    this.slop = slop;
-    this.maxAnalyzedChars = maxAnalyzedChars;
-    this.textRE = targetPattern;
-  }
-  
-
-  /* (non-Javadoc)
-   * @see org.apache.lucene.search.highlight.TextFragmenter#start(java.lang.String)
-   */
-  @Override
-  public void start(String originalText, TokenStream tokenStream) {
-    currentNumFrags = 1;
-    currentOffset = 0;
-    addHotSpots(originalText);
-    posIncAtt = tokenStream.getAttribute(PositionIncrementAttribute.class);
-    offsetAtt = tokenStream.getAttribute(OffsetAttribute.class);
-  }
-
-  ////////////////////////////////////
-  // pre-analysis
-  ////////////////////////////////////
-
-  protected void addHotSpots(String text) {
-    //System.out.println("hot spotting");
-    ArrayList<Integer> temphs = new ArrayList<>(
-                              text.length() / targetFragChars);
-    Matcher match = textRE.matcher(text);
-    int cur = 0;
-    while(match.find() && cur < maxAnalyzedChars) {
-      int start=match.start(), end=match.end();
-      temphs.add(start);
-      temphs.add(end);
-      cur = end;
-      //System.out.println("Matched " + match.group());
-    }    
-    hotspots = new int[temphs.size()];
-    for(int i = 0; i < temphs.size(); i++) {
-      hotspots[i] = temphs.get(i);
-    }
-    // perhaps not necessary--I don't know if re matches are non-overlapping
-    Arrays.sort(hotspots);
-  }
-
-  ////////////////////////////////////
-  // fragmenting
-  ////////////////////////////////////
-
-  /* (non-Javadoc)
-   * @see org.apache.lucene.search.highlight.TextFragmenter#isNewFragment(org.apache.lucene.analysis.Token)
-   */
-  @Override
-  public boolean isNewFragment()
-  {
-    boolean isNewFrag = false;
-    int minFragLen = (int)((1.0f - slop)*targetFragChars);
-    int endOffset = offsetAtt.endOffset();
-    
-    // ** determin isNewFrag
-    if(posIncAtt.getPositionIncrement() > incrementGapThreshold) {
-      // large position gaps always imply new fragments
-      isNewFrag = true;
-
-    } else if(endOffset - currentOffset < minFragLen) {
-      // we're not in our range of flexibility
-      isNewFrag = false;
-
-    } else if(targetOffset > 0) {
-      // we've already decided on a target
-      isNewFrag = endOffset > targetOffset;
-
-    } else {
-      // we might be able to do something
-      int minOffset = currentOffset + minFragLen;
-      int maxOffset = (int)(currentOffset + (1.0f + slop)*targetFragChars);
-      int hotIndex;
-
-      // look for a close hotspot
-      hotIndex = Arrays.binarySearch(hotspots, endOffset);
-      if(hotIndex < 0) hotIndex = -hotIndex;
-      if(hotIndex >= hotspots.length) {
-        // no more hotspots in this input stream
-        targetOffset = currentOffset + targetFragChars;
-
-      } else if(hotspots[hotIndex] > maxOffset) {
-        // no hotspots within slop
-        targetOffset = currentOffset + targetFragChars;
-
-      } else {
-        // try to find hotspot in slop
-        int goal = hotspots[hotIndex];
-        while(goal < minOffset && hotIndex < hotspots.length) {
-          hotIndex++;
-          goal = hotspots[hotIndex];
-        }        
-        targetOffset = goal <= maxOffset ? goal : currentOffset + targetFragChars;
-      }
-
-      isNewFrag = endOffset > targetOffset;
-    }      
-      
-    // ** operate on isNewFrag
-    if(isNewFrag) {
-        currentNumFrags++;
-        currentOffset = endOffset;
-        targetOffset = -1;
-    }
-    return isNewFrag;
-  }
-  
-}


[50/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-5944: refactor away unused (test helper) method param

Posted by ho...@apache.org.
SOLR-5944: refactor away unused (test helper) method param


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

Branch: refs/heads/jira/solr-5944
Commit: 0402c1a06b552de1ffa1957a83c654d74ffb03ca
Parents: 027a92a
Author: Chris Hostetter <ho...@apache.org>
Authored: Sun Jan 15 16:08:44 2017 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Sun Jan 15 16:08:44 2017 -0700

----------------------------------------------------------------------
 .../solr/update/TestInPlaceUpdatesDistrib.java  | 21 +++++++++-----------
 1 file changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0402c1a0/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index c75dd20..8f62b9d 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -244,7 +244,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       ids.add(id);
     }
       
-    buildRandomIndex(101.0F, false, ids);
+    buildRandomIndex(101.0F, ids);
     
     List<Integer> luceneDocids = new ArrayList<>(numDocs);
     List<Float> valuesList = new ArrayList<Float>(numDocs);
@@ -724,7 +724,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     commit();
     
     float inplace_updatable_float = 1F;
-    buildRandomIndex(inplace_updatable_float, false, Collections.singletonList(1));
+    buildRandomIndex(inplace_updatable_float, Collections.singletonList(1));
 
     float newinplace_updatable_float = 100F;
     List<UpdateRequest> updates = new ArrayList<>();
@@ -972,27 +972,24 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
 
   /**
    * Convinience method variant that never uses <code>initFloat</code>
-   * @see #buildRandomIndex(Float,boolean,List)
+   * @see #buildRandomIndex(Float,List)
    */
   protected List<Long> buildRandomIndex(Integer... specialIds) throws Exception {
-    return buildRandomIndex(null, false, Arrays.asList(specialIds));
+    return buildRandomIndex(null, Arrays.asList(specialIds));
   }
                                         
   /** 
    * Helper method to build a randomized index with the fields needed for all test methods in this class.
-   * At a minimum, this index will contain 1 doc per "special" (non-negative) document id.  These special documents will be added with the <code>initFloat</code> specified in the "inplace_updatable_float" based on the <code>useFloatRandomly</code> param.
+   * At a minimum, this index will contain 1 doc per "special" (non-negative) document id.  These special documents will be added with the <code>initFloat</code> specified in the "inplace_updatable_float" field.
    *
    * A random number of documents (with negative ids) will be indexed in between each of the 
    * "special" documents, as well as before/after the first/last special document.
    *
-   * @param initFloat Value to use in the "inplace_updatable_float" for some of the special documents, based on the <code>useFloatRandomly</code> param; will never be used if null
-   * @param useFloatRandomly  If false, all special docs will get the <code>initFloat</code> value; if true, only a random subset of the special docs will get a value.
+   * @param initFloat Value to use in the "inplace_updatable_float" for the special documents; will never be used if null
    * @param specialIds The ids to use for the special documents, all values must be non-negative
    * @return the versions of each of the specials document returned when indexing it
    */
-  protected List<Long> buildRandomIndex(Float initFloat, boolean useFloatRandomly,
-                                        List<Integer> specialIds) throws Exception {
-    // nocommit: optimize away the useFloatRandomly param if it winds up being false in all callers
+  protected List<Long> buildRandomIndex(Float initFloat, List<Integer> specialIds) throws Exception {
     
     int id = -1; // used for non special docs
     final int numPreDocs = rarely() ? TestUtil.nextInt(random(),0,9) : atLeast(10);
@@ -1002,7 +999,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
     final List<Long> versions = new ArrayList<>(specialIds.size());
     for (int special : specialIds) {
-      if (null == initFloat || (useFloatRandomly && random().nextBoolean()) ) {
+      if (null == initFloat) {
         versions.add(addDocAndGetVersion("id", special, "title_s", "title" + special, "id_i", special));
       } else {
         versions.add(addDocAndGetVersion("id", special, "title_s", "title" + special, "id_i", special,
@@ -1032,7 +1029,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     commit();
     
     float inplace_updatable_float = 1F;
-    buildRandomIndex(inplace_updatable_float, false, Collections.singletonList(1));
+    buildRandomIndex(inplace_updatable_float, Collections.singletonList(1));
 
     List<UpdateRequest> updates = new ArrayList<>();
     updates.add(regularUpdateRequest("id", 1, "id_i", 1, "inplace_updatable_float", 12, "title_s", "mytitle"));


[30/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9918: Add SkipExistingDocumentsProcessor that skips duplicate inserts and ignores updates to missing docs

Posted by ho...@apache.org.
SOLR-9918: Add SkipExistingDocumentsProcessor that skips duplicate inserts and ignores updates to missing docs


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

Branch: refs/heads/jira/solr-5944
Commit: d66bfba5dc1bd9154bd48898865f51d9715e8d0c
Parents: 118fc42
Author: koji <ko...@apache.org>
Authored: Wed Jan 11 11:48:33 2017 +0900
Committer: koji <ko...@apache.org>
Committed: Wed Jan 11 11:48:33 2017 +0900

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../SkipExistingDocumentsProcessorFactory.java  | 255 ++++++++++++++
 ...ipExistingDocumentsProcessorFactoryTest.java | 336 +++++++++++++++++++
 3 files changed, 594 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66bfba5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0ee18ba..204ea26 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -126,6 +126,9 @@ Upgrade Notes
 
 New Features
 ----------------------
+* SOLR-9918: Add SkipExistingDocumentsProcessor that skips duplicate inserts and ignores updates to missing docs
+  (Tim Owen via koji)
+
 * SOLR-9293: Solrj client support for hierarchical clusters and other topics 
   marker. (Dawid Weiss)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66bfba5/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
new file mode 100644
index 0000000..ec637a4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
@@ -0,0 +1,255 @@
+/*
+ * 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.update.processor;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.RealTimeGetComponent;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.UpdateCommand;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.plugin.SolrCoreAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+/**
+ * <p>
+ *     This Factory generates an UpdateProcessor that will (by default) skip inserting new documents
+ *     if there already exists a document with the same uniqueKey value in the index. It will also
+ *     skip Atomic Updates to a document if that document does not already exist. This behaviour is applied
+ *     to each document in turn, so adding a batch of documents can result in some being added and some
+ *     ignored, depending on what is already in the index. If all of the documents are skipped, no changes
+ *     to the index will occur.
+ * </p>
+ * These two forms of skipping can be switched on or off independently, by using init params:
+ * <ul>
+ *     <li><code>skipInsertIfExists</code> - This boolean parameter defaults to
+ *          <code>true</code>, but if set to <code>false</code> then inserts (i.e. not Atomic Updates)
+ *          will be passed through unchanged even if the document already exists.</li>
+ *     <li><code>skipUpdateIfMissing</code> - This boolean parameter defaults to
+ *         <code>true</code>, but if set to <code>false</code> then Atomic Updates
+ *          will be passed through unchanged regardless of whether the document exists.</li>
+ * </ul>
+ * <p>
+ *     These params can also be specified per-request, to override the configured behaviour
+ *     for specific updates e.g. <code>/update?skipUpdateIfMissing=true</code>
+ * </p>
+ * <p>
+ *     This implementation is a simpler alternative to {@link DocBasedVersionConstraintsProcessorFactory}
+ *     when you are not concerned with versioning, and just want to quietly ignore duplicate documents and/or
+ *     silently skip updates to non-existent documents (in the same way a database <code>UPDATE</code> would).
+ *
+ *     If your documents do have an explicit version field, and you want to ensure older versions are
+ *     skipped instead of replacing the indexed document, you should consider {@link DocBasedVersionConstraintsProcessorFactory}
+ *     instead.
+ * </p>
+ * <p>
+ *     An example chain configuration to use this for skipping duplicate inserts, but not skipping updates to
+ *     missing documents by default, is:
+ * </p>
+ * <pre class="prettyprint">
+ * &lt;updateRequestProcessorChain name="skipexisting"&gt;
+ *   &lt;processor class="solr.LogUpdateProcessorFactory" /&gt;
+ *   &lt;processor class="solr.SkipExistingDocumentsProcessorFactory"&gt;
+ *     &lt;bool name="skipInsertIfExists"&gt;true&lt;/bool&gt;
+ *     &lt;bool name="skipUpdateIfMissing"&gt;false&lt;/bool&gt; &lt;!-- Can override this per-request --&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.DistributedUpdateProcessorFactory" /&gt;
+ *   &lt;processor class="solr.RunUpdateProcessorFactory" /&gt;
+ * &lt;/updateRequestProcessorChain&gt;
+ * </pre>
+ */
+public class SkipExistingDocumentsProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String PARAM_SKIP_INSERT_IF_EXISTS = "skipInsertIfExists";
+  private static final String PARAM_SKIP_UPDATE_IF_MISSING = "skipUpdateIfMissing";
+
+  private boolean skipInsertIfExists = true;
+  private boolean skipUpdateIfMissing = true;
+
+  @Override
+  public void init(NamedList args)  {
+    Object tmp = args.remove(PARAM_SKIP_INSERT_IF_EXISTS);
+    if (null != tmp) {
+      if (! (tmp instanceof Boolean) ) {
+        throw new SolrException(SERVER_ERROR, "'" + PARAM_SKIP_INSERT_IF_EXISTS + "' must be configured as a <bool>");
+      }
+      skipInsertIfExists = (Boolean)tmp;
+    }
+    tmp = args.remove(PARAM_SKIP_UPDATE_IF_MISSING);
+    if (null != tmp) {
+      if (! (tmp instanceof Boolean) ) {
+        throw new SolrException(SERVER_ERROR, "'" + PARAM_SKIP_UPDATE_IF_MISSING + "' must be configured as a <bool>");
+      }
+      skipUpdateIfMissing = (Boolean)tmp;
+    }
+
+    super.init(args);
+  }
+
+  @Override
+  public SkipExistingDocumentsUpdateProcessor getInstance(SolrQueryRequest req,
+                                                          SolrQueryResponse rsp,
+                                                          UpdateRequestProcessor next) {
+    // Ensure the parameters are forwarded to the leader
+    DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist(req, PARAM_SKIP_INSERT_IF_EXISTS, PARAM_SKIP_UPDATE_IF_MISSING);
+
+    // Allow the particular request to override the plugin's configured behaviour
+    boolean skipInsertForRequest = req.getOriginalParams().getBool(PARAM_SKIP_INSERT_IF_EXISTS, this.skipInsertIfExists);
+    boolean skipUpdateForRequest = req.getOriginalParams().getBool(PARAM_SKIP_UPDATE_IF_MISSING, this.skipUpdateIfMissing);
+
+    return new SkipExistingDocumentsUpdateProcessor(req, next, skipInsertForRequest, skipUpdateForRequest);
+  }
+
+  @Override
+  public void inform(SolrCore core) {
+
+    if (core.getUpdateHandler().getUpdateLog() == null) {
+      throw new SolrException(SERVER_ERROR, "updateLog must be enabled.");
+    }
+
+    if (core.getLatestSchema().getUniqueKeyField() == null) {
+      throw new SolrException(SERVER_ERROR, "schema must have uniqueKey defined.");
+    }
+  }
+
+  static class SkipExistingDocumentsUpdateProcessor extends UpdateRequestProcessor {
+
+    private final boolean skipInsertIfExists;
+    private final boolean skipUpdateIfMissing;
+    private final SolrCore core;
+
+    private DistributedUpdateProcessor distribProc;  // the distributed update processor following us
+    private DistributedUpdateProcessor.DistribPhase phase;
+
+    SkipExistingDocumentsUpdateProcessor(SolrQueryRequest req,
+                                         UpdateRequestProcessor next,
+                                         boolean skipInsertIfExists,
+                                         boolean skipUpdateIfMissing) {
+      super(next);
+      this.skipInsertIfExists = skipInsertIfExists;
+      this.skipUpdateIfMissing = skipUpdateIfMissing;
+      this.core = req.getCore();
+
+      for (UpdateRequestProcessor proc = next ;proc != null; proc = proc.next) {
+        if (proc instanceof DistributedUpdateProcessor) {
+          distribProc = (DistributedUpdateProcessor)proc;
+          break;
+        }
+      }
+
+      if (distribProc == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "DistributedUpdateProcessor must follow SkipExistingDocumentsUpdateProcessor");
+      }
+
+      phase = DistributedUpdateProcessor.DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
+    }
+
+    boolean isSkipInsertIfExists() {
+      return this.skipInsertIfExists;
+    }
+
+    boolean isSkipUpdateIfMissing() {
+      return this.skipUpdateIfMissing;
+    }
+
+    boolean doesDocumentExist(BytesRef indexedDocId) throws IOException {
+      assert null != indexedDocId;
+
+      SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId);
+      if (oldDoc == RealTimeGetComponent.DELETED) {
+        return false;
+      }
+      if (oldDoc != null) {
+        return true;
+      }
+
+      // need to look up in index now...
+      RefCounted<SolrIndexSearcher> newestSearcher = core.getRealtimeSearcher();
+      try {
+        SolrIndexSearcher searcher = newestSearcher.get();
+        return searcher.lookupId(indexedDocId) >= 0L;
+      } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document from index", e);
+      } finally {
+        if (newestSearcher != null) {
+          newestSearcher.decref();
+        }
+      }
+    }
+
+    boolean isLeader(UpdateCommand cmd) {
+      if ((cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0) {
+        return false;
+      }
+      if (phase == DistributedUpdateProcessor.DistribPhase.FROMLEADER) {
+        return false;
+      }
+      return distribProc.isLeader(cmd);
+    }
+
+    @Override
+    public void processAdd(AddUpdateCommand cmd) throws IOException {
+      BytesRef indexedDocId = cmd.getIndexedId();
+
+      boolean isUpdate = AtomicUpdateDocumentMerger.isAtomicUpdate(cmd);
+
+      // boolean existsByLookup = (RealTimeGetComponent.getInputDocument(core, indexedDocId) != null);
+      // if (docExists != existsByLookup) {
+      //   log.error("Found docExists {} but existsByLookup {} for doc {}", docExists, existsByLookup, indexedDocId.utf8ToString());
+      // }
+
+      if (log.isDebugEnabled()) {
+        log.debug("Document ID {} ... exists already? {} ... isAtomicUpdate? {} ... isLeader? {}",
+                  indexedDocId.utf8ToString(), doesDocumentExist(indexedDocId), isUpdate, isLeader(cmd));
+      }
+
+      if (skipInsertIfExists && !isUpdate && isLeader(cmd) && doesDocumentExist(indexedDocId)) {
+        if (log.isDebugEnabled()) {
+          log.debug("Skipping insert for pre-existing document ID {}", indexedDocId.utf8ToString());
+        }
+        return;
+      }
+
+      if (skipUpdateIfMissing && isUpdate && isLeader(cmd) && !doesDocumentExist(indexedDocId)) {
+        if (log.isDebugEnabled()) {
+          log.debug("Skipping update to non-existent document ID {}", indexedDocId.utf8ToString());
+        }
+        return;
+      }
+
+      if (log.isDebugEnabled()) {
+        log.debug("Passing on document ID {}", indexedDocId.utf8ToString());
+      }
+
+      super.processAdd(cmd);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66bfba5/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
new file mode 100644
index 0000000..2afe35c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.update.processor;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.SkipExistingDocumentsProcessorFactory.SkipExistingDocumentsUpdateProcessor;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class SkipExistingDocumentsProcessorFactoryTest {
+
+  private BytesRef docId = new BytesRef();
+  private SolrQueryRequest defaultRequest = new LocalSolrQueryRequest(null, new NamedList());
+
+  // Tests for logic in the factory
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfSkipInsertParamNonBoolean() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", "false");
+    factory.init(initArgs);
+  }
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfSkipUpdateParamNonBoolean() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipUpdateIfMissing", 0);
+    factory.init(initArgs);
+  }
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfNextProcessorIsNull() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), null);
+  }
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfNextProcessorNotDistributed() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor next = new BufferingRequestProcessor(null);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+  }
+
+  @Test
+  public void testNoExceptionIfNextProcessorIsDistributed() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+  }
+
+  @Test
+  public void testNoExceptionIfNextNextProcessorIsDistributed() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor distProcessor = Mockito.mock(DistributedUpdateProcessor.class);
+    UpdateRequestProcessor next = new BufferingRequestProcessor(distProcessor);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+  }
+
+  @Test
+  public void testSkipInsertsAndUpdatesDefaultToTrueIfNotConfigured() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipInsertsFalseIfInInitArgs() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", false);
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertFalse("Expected skipInsertIfExists to be false", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipUpdatesFalseIfInInitArgs() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipUpdateIfMissing", false);
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertFalse("Expected skipUpdateIfMissing to be false", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipBothFalseIfInInitArgs() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", false);
+    initArgs.add("skipUpdateIfMissing", false);
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertFalse("Expected skipInsertIfExists to be false", processor.isSkipInsertIfExists());
+    assertFalse("Expected skipUpdateIfMissing to be false", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipInsertsFalseIfInitArgsTrueButFalseStringInRequest() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", true);
+    factory.init(initArgs);
+    NamedList<String> requestArgs = new NamedList<>();
+    requestArgs.add("skipInsertIfExists", "false");
+    SolrQueryRequest req = new LocalSolrQueryRequest(null, requestArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(req, new SolrQueryResponse(), next);
+    assertFalse("Expected skipInsertIfExists to be false", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipUpdatesFalseIfInitArgsTrueButFalseBooleanInRequest() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipUpdateIfMissing", true);
+    factory.init(initArgs);
+    NamedList<Object> requestArgs = new NamedList<>();
+    requestArgs.add("skipUpdateIfMissing", false);
+    SolrQueryRequest req = new LocalSolrQueryRequest(null, requestArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(req, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertFalse("Expected skipUpdateIfMissing to be false", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipUpdatesTrueIfInitArgsFalseButTrueStringInRequest() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", true);
+    initArgs.add("skipUpdateIfMissing", false);
+    factory.init(initArgs);
+    NamedList<Object> requestArgs = new NamedList<>();
+    requestArgs.add("skipUpdateIfMissing", "true");
+    SolrQueryRequest req = new LocalSolrQueryRequest(null, requestArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(req, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+
+  // Tests for logic in the processor
+
+  @Test
+  public void testSkippableInsertIsNotSkippedIfNotLeader() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, true));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(false).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableInsertIsNotSkippedIfSkipInsertsFalse() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, false));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableInsertIsSkippedIfSkipInsertsTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, false));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next, never()).processAdd(cmd);
+  }
+
+  @Test
+  public void testNonSkippableInsertIsNotSkippedIfSkipInsertsTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, false));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableUpdateIsNotSkippedIfNotLeader() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, true));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(false).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableUpdateIsNotSkippedIfSkipUpdatesFalse() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, false));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableUpdateIsSkippedIfSkipUpdatesTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, true));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next, never()).processAdd(cmd);
+  }
+
+  @Test
+  public void testNonSkippableUpdateIsNotSkippedIfSkipUpdatesTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, true));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  private AddUpdateCommand createInsertUpdateCmd(SolrQueryRequest req) {
+    AddUpdateCommand cmd = new AddUpdateCommand(req);
+    cmd.setIndexedId(docId);
+    cmd.solrDoc = new SolrInputDocument();
+    assertFalse(AtomicUpdateDocumentMerger.isAtomicUpdate(cmd));
+    return cmd;
+  }
+
+  private AddUpdateCommand createAtomicUpdateCmd(SolrQueryRequest req) {
+    AddUpdateCommand cmd = new AddUpdateCommand(req);
+    cmd.setIndexedId(docId);
+    cmd.solrDoc = new SolrInputDocument();
+    cmd.solrDoc.addField("last_name", ImmutableMap.of("set", "Smith"));
+    assertTrue(AtomicUpdateDocumentMerger.isAtomicUpdate(cmd));
+    return cmd;
+  }
+}


[05/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9883: In example schemaless configs' default update chain, move the DUP to after the AddSchemaFields URP (which is now tagged as RunAlways), to avoid invalid buffered tlog entry replays.

Posted by ho...@apache.org.
SOLR-9883: In example schemaless configs' default update chain, move the DUP to after the AddSchemaFields URP (which is now tagged as RunAlways), to avoid invalid buffered tlog entry replays.


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

Branch: refs/heads/jira/solr-5944
Commit: d817fd43eccd67a5d73c3bbc49561de65d3fc9cb
Parents: 67261d2
Author: Steve Rowe <sa...@apache.org>
Authored: Sat Jan 7 16:39:20 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Sat Jan 7 16:39:20 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../java/org/apache/solr/update/UpdateLog.java  |   4 +-
 .../AddSchemaFieldsUpdateProcessorFactory.java  |   3 +-
 ...dd-schema-fields-update-processor-chains.xml |   8 +-
 .../collection1/conf/solrconfig-schemaless.xml  |  45 ++----
 .../schema/TestSchemalessBufferedUpdates.java   | 160 +++++++++++++++++++
 solr/example/files/conf/solrconfig.xml          |   5 +-
 .../basic_configs/conf/solrconfig.xml           |   6 +-
 .../conf/solrconfig.xml                         |   5 +-
 9 files changed, 190 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1f7f09a..0d61730 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -337,6 +337,9 @@ Bug Fixes
 
 * SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules (Tim Owen via noble)
 
+* SOLR-9883: Example schemaless solr config files can lead to invalid tlog replays: when updates are buffered,
+  update processors ordered before DistributedUpdateProcessor, e.g. field normalization, are never run. (Steve Rowe)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 9c0f1cf..b79290d 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -1429,7 +1429,7 @@ public class UpdateLog implements PluginInfoInitialized {
             }
           } catch (IOException ex) {
             recoveryInfo.errors++;
-            loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+            loglog.warn("REPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
           } catch (ClassCastException cl) {
             recoveryInfo.errors++;
@@ -1440,7 +1440,7 @@ public class UpdateLog implements PluginInfoInitialized {
               throw ex;
             }
             recoveryInfo.errors++;
-            loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+            loglog.warn("REPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
           } catch (Exception ex) {
             recoveryInfo.errors++;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
index 4f68bcc..4758972 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
@@ -120,7 +120,8 @@ import static org.apache.solr.core.ConfigSetProperties.IMMUTABLE_CONFIGSET_ARG;
  *   &lt;/lst&gt;
  * &lt;/processor&gt;</pre>
  */
-public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware {
+public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcessorFactory
+    implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final String TYPE_MAPPING_PARAM = "typeMapping";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
index 8d91d28..e574575 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
@@ -66,8 +66,6 @@
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="add-fields">
-    <processor class="solr.LogUpdateProcessorFactory" />
-    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.AddSchemaFieldsUpdateProcessorFactory">
       <str name="defaultFieldType">text</str>
       <lst name="typeMapping">
@@ -96,12 +94,12 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="parse-and-add-fields">
-    <processor class="solr.LogUpdateProcessorFactory" />
-    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>
@@ -154,6 +152,8 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
index 46aa2a4..8247d69 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
@@ -1,5 +1,4 @@
-<?xml version="1.0" ?>
-
+<?xml version="1.0" encoding="UTF-8" ?>
 <!--
  Licensed to the Apache Software Foundation (ASF) under one or more
  contributor license agreements.  See the NOTICE file distributed with
@@ -16,7 +15,7 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
-                                                           
+
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
@@ -33,47 +32,26 @@
   <updateHandler>
     <updateLog enable="true">
       <str name="dir">${solr.ulog.dir:}</str>
-    </updateLog> 
+    </updateLog>
   </updateHandler>
 
   <requestHandler name="standard" class="solr.StandardRequestHandler">
     <bool name="httpCaching">true</bool>
   </requestHandler>
 
-
-  <requestHandler name="/update" class="solr.UpdateRequestHandler">
+  <initParams path="/update/**">
     <lst name="defaults">
       <str name="update.chain">add-unknown-fields-to-the-schema</str>
     </lst>
-  </requestHandler>
-
-  <query>
-    <!-- custom cache currently used by block join -->
-    <cache name="perSegFilter"
-           class="solr.search.LRUCache"
-           size="10"
-           initialSize="0"
-           autowarmCount="10"
-           regenerator="solr.NoOpRegenerator" />
-  </query>
+  </initParams>
 
-  <!-- Add unknown fields to the schema
-  
-       An example field type guessing update processor that will
-       attempt to parse string-typed field values as Booleans, Longs,
-       Doubles, or Dates, and then add schema fields with the guessed
-       field types.  
-       
-       This requires that the schema is both managed and mutable, by
-       declaring schemaFactory as ManagedIndexSchemaFactory, with
-       mutable specified as true. 
-       
-       See http://wiki.apache.org/solr/GuessingFieldTypes
-    -->
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.UUIDUpdateProcessorFactory" />
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
+    <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
+      <str name="pattern">[^\w-\.]</str>
+      <str name="replacement">_</str>
+    </processor>
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>
@@ -119,7 +97,8 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
-
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java b/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
new file mode 100644
index 0000000..c2e8b2e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
@@ -0,0 +1,160 @@
+/*
+ * 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.schema;
+
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.DirectUpdateHandler2;
+import org.apache.solr.update.UpdateLog;
+import org.apache.solr.update.UpdateHandler;
+import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+
+public class TestSchemalessBufferedUpdates extends SolrTestCaseJ4 {
+
+  // means that we've seen the leader and have version info (i.e. we are a non-leader replica)
+  private static final String FROM_LEADER = DistribPhase.FROMLEADER.toString();
+  private static final String UPDATE_CHAIN = "add-unknown-fields-to-the-schema";
+  private static final int TIMEOUT = 10;
+
+  private static final String collection = "collection1";
+  private static final String confDir = collection + "/conf";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    File tmpSolrHome = createTempDir().toFile();
+    File tmpConfDir = new File(tmpSolrHome, confDir);
+    File testHomeConfDir = new File(TEST_HOME(), confDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig-schemaless.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema-add-schema-fields-update-processor.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig.snippet.randomindexconfig.xml"), tmpConfDir);
+    initCore("solrconfig-schemaless.xml", "schema-add-schema-fields-update-processor.xml", tmpSolrHome.getPath());
+  }
+
+  @Test
+  public void test() throws Exception {
+    DirectUpdateHandler2.commitOnClose = false;
+    final Semaphore logReplay = new Semaphore(0);
+    final Semaphore logReplayFinish = new Semaphore(0);
+    UpdateLog.testing_logReplayHook = () -> {
+      try {
+        assertTrue(logReplay.tryAcquire(TIMEOUT, TimeUnit.SECONDS));
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    };
+    UpdateLog.testing_logReplayFinishHook = logReplayFinish::release;
+
+    SolrQueryRequest req = req();
+    UpdateHandler uhandler = req.getCore().getUpdateHandler();
+    UpdateLog ulog = uhandler.getUpdateLog();
+
+    try {
+      assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
+
+      // Invalid date will be normalized by ParseDateField URP
+      updateJ(jsonAdd(processAdd(sdoc("id","1", "f_dt","2017-01-04"))), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+      assertU(commit());
+      assertJQ(req("q", "*:*"), "/response/numFound==1");
+
+      ulog.bufferUpdates();
+      assertEquals(UpdateLog.State.BUFFERING, ulog.getState());
+
+      // If the ParseDateField URP isn't ahead of the DUP, then the date won't be normalized in the buffered tlog entry,
+      // and the doc won't be indexed on the replaying replica - a warning is logged as follows:
+      // WARN [...] o.a.s.u.UpdateLog REYPLAY_ERR: IOException reading log
+      //            org.apache.solr.common.SolrException: Invalid Date String:'2017-01-05'
+      //              at org.apache.solr.util.DateMathParser.parseMath(DateMathParser.java:234)
+      //              at org.apache.solr.schema.TrieField.createField(TrieField.java:725) [...]
+      updateJ(jsonAdd(processAdd(sdoc("id","2", "f_dt","2017-01-05"))), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+
+      Future<UpdateLog.RecoveryInfo> rinfoFuture = ulog.applyBufferedUpdates();
+
+      assertTrue(rinfoFuture != null);
+
+      assertEquals(UpdateLog.State.APPLYING_BUFFERED, ulog.getState());
+
+      logReplay.release(1000);
+
+      UpdateLog.RecoveryInfo rinfo = rinfoFuture.get();
+      assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
+
+      assertU(commit());
+      assertJQ(req("q", "*:*"), "/response/numFound==2");
+    } finally {
+      DirectUpdateHandler2.commitOnClose = true;
+      UpdateLog.testing_logReplayHook = null;
+      UpdateLog.testing_logReplayFinishHook = null;
+      req().close();
+    }
+  }
+
+  private SolrInputDocument processAdd(final SolrInputDocument docIn) throws IOException {
+    UpdateRequestProcessorChain processorChain = h.getCore().getUpdateProcessingChain(UPDATE_CHAIN);
+    assertNotNull("Undefined URP chain '" + UPDATE_CHAIN + "'", processorChain);
+    List <UpdateRequestProcessorFactory> factoriesUpToDUP = new ArrayList<>();
+    for (UpdateRequestProcessorFactory urpFactory : processorChain.getProcessors()) {
+      factoriesUpToDUP.add(urpFactory);
+      if (urpFactory.getClass().equals(DistributedUpdateProcessorFactory.class)) 
+        break;
+    }
+    UpdateRequestProcessorChain chainUpToDUP = new UpdateRequestProcessorChain(factoriesUpToDUP, h.getCore());
+    assertNotNull("URP chain '" + UPDATE_CHAIN + "'", chainUpToDUP);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = req();
+    try {
+      SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
+      AddUpdateCommand cmd = new AddUpdateCommand(req);
+      cmd.solrDoc = docIn;
+      UpdateRequestProcessor processor = chainUpToDUP.createProcessor(req, rsp);
+      processor.processAdd(cmd);
+      if (cmd.solrDoc.get("f_dt").getValue() instanceof Date) {
+        // Non-JSON types (Date in this case) aren't handled properly in noggit-0.6.  Although this is fixed in 
+        // https://github.com/yonik/noggit/commit/ec3e732af7c9425e8f40297463cbe294154682b1 to call obj.toString(), 
+        // Date::toString produces a Date representation that Solr doesn't like, so we convert using Instant::toString
+        cmd.solrDoc.get("f_dt").setValue(((Date) cmd.solrDoc.get("f_dt").getValue()).toInstant().toString(), 1.0f);
+      }
+      return cmd.solrDoc;
+    } finally {
+      SolrRequestInfo.clearRequestInfo();
+      req.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/example/files/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/files/conf/solrconfig.xml b/solr/example/files/conf/solrconfig.xml
index 3fd825e..f83c235 100644
--- a/solr/example/files/conf/solrconfig.xml
+++ b/solr/example/files/conf/solrconfig.xml
@@ -1193,9 +1193,6 @@
   <updateRequestProcessorChain name="files-update-processor">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1261,6 +1258,8 @@
       <!--</lst>-->
     </processor>
 
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
index 4ef902f..e009aab 100644
--- a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
@@ -1187,9 +1187,6 @@
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1239,6 +1236,9 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
+
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
index 4b0899b..2ca1b7f 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
@@ -1186,9 +1186,6 @@
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1238,6 +1235,8 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 


[23/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9856 Collect metrics for shard replication and tlog replay on replicas.

Posted by ho...@apache.org.
SOLR-9856 Collect metrics for shard replication and tlog replay on replicas.


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

Branch: refs/heads/jira/solr-5944
Commit: b8383db06ee194b9195cd95f058dc820cb70baf8
Parents: 1d7379b
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Jan 9 21:00:00 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Jan 9 21:00:00 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../org/apache/solr/core/SolrInfoMBean.java     |  2 +-
 .../java/org/apache/solr/update/PeerSync.java   | 42 +++++++++++++--
 .../java/org/apache/solr/update/UpdateLog.java  | 54 +++++++++++++++++++-
 .../solr/cloud/PeerSyncReplicationTest.java     | 15 ++++++
 .../apache/solr/cloud/TestCloudRecovery.java    | 26 ++++++++++
 .../org/apache/solr/search/TestRecovery.java    | 40 ++++++++++++++-
 7 files changed, 174 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b28df9c..c79b3c6 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -228,6 +228,8 @@ New Features
 * SOLR-9935: Add hl.fragsize support when using the UnifiedHighlighter to avoid snippets/Passages that are too small.
   Defaults to 70. (David Smiley)
 
+* SOLR-9856: Collect metrics for shard replication and tlog replay on replicas (ab).
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
index 421a4ef..04c8395 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -32,7 +32,7 @@ public interface SolrInfoMBean {
   /**
    * Category of {@link SolrCore} component.
    */
-  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, INDEX, DIRECTORY, HTTP, OTHER }
+  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, OTHER }
 
   /**
    * Top-level group of beans for a subsystem.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/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 12ba7ae..861cbf7 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -29,6 +29,8 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Timer;
 import org.apache.http.NoHttpResponseException;
 import org.apache.http.client.HttpClient;
 import org.apache.http.conn.ConnectTimeoutException;
@@ -40,12 +42,15 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
@@ -58,7 +63,7 @@ import static org.apache.solr.update.processor.DistributedUpdateProcessor.Distri
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
 /** @lucene.experimental */
-public class PeerSync  {
+public class PeerSync implements SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private boolean debug = log.isDebugEnabled();
 
@@ -87,6 +92,11 @@ public class PeerSync  {
   private final boolean onlyIfActive;
   private SolrCore core;
 
+  // metrics
+  private Timer syncTime;
+  private Counter syncErrors;
+  private Counter syncSkipped;
+
   // comparator that sorts by absolute value, putting highest first
   public static Comparator<Long> absComparator = (o1, o2) -> {
     long l1 = Math.abs(o1);
@@ -112,7 +122,6 @@ public class PeerSync  {
     return 0;
   };
 
-
   private static class SyncShardRequest extends ShardRequest {
     List<Long> reportedVersions;
     IndexFingerprint fingerprint;
@@ -147,6 +156,15 @@ public class PeerSync  {
     // TODO: close
     shardHandlerFactory = (HttpShardHandlerFactory) core.getCoreDescriptor().getCoreContainer().getShardHandlerFactory();
     shardHandler = shardHandlerFactory.getShardHandler(client);
+
+    core.getCoreMetricManager().registerMetricProducer(SolrInfoMBean.Category.REPLICATION.toString(), this);
+  }
+
+  @Override
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+    syncTime = manager.timer(registry, "time", scope);
+    syncErrors = manager.counter(registry, "errors", scope);
+    syncSkipped = manager.counter(registry, "skipped", scope);
   }
 
   /** optional list of updates we had before possibly receiving new updates */
@@ -208,9 +226,11 @@ public class PeerSync  {
    */
   public PeerSyncResult sync() {
     if (ulog == null) {
+      syncErrors.inc();
       return PeerSyncResult.failure();
     }
     MDCLoggingContext.setCore(core);
+    Timer.Context timerContext = null;
     try {
       log.info(msg() + "START replicas=" + replicas + " nUpdates=" + nUpdates);
       
@@ -221,10 +241,13 @@ public class PeerSync  {
       }
       // check if we already in sync to begin with 
       if(doFingerprint && alreadyInSync()) {
+        syncSkipped.inc();
         return PeerSyncResult.success();
       }
-      
-      
+
+      // measure only when actual sync is performed
+      timerContext = syncTime.time();
+
       // Fire off the requests before getting our own recent updates (for better concurrency)
       // This also allows us to avoid getting updates we don't need... if we got our updates and then got their updates,
       // they would
@@ -242,6 +265,7 @@ public class PeerSync  {
       if (startingVersions != null) {
         if (startingVersions.size() == 0) {
           log.warn("no frame of reference to tell if we've missed updates");
+          syncErrors.inc();
           return PeerSyncResult.failure();
         }
         Collections.sort(startingVersions, absComparator);
@@ -257,6 +281,7 @@ public class PeerSync  {
         if (Math.abs(startingVersions.get(0)) < smallestNewUpdate) {
           log.warn(msg()
               + "too many updates received since start - startingUpdates no longer overlaps with our currentUpdates");
+          syncErrors.inc();
           return PeerSyncResult.failure();
         }
         
@@ -285,10 +310,12 @@ public class PeerSync  {
             if (srsp.getException() == null)  {
               List<Long> otherVersions = (List<Long>)srsp.getSolrResponse().getResponse().get("versions");
               if (otherVersions != null && !otherVersions.isEmpty())  {
+                syncErrors.inc();
                 return PeerSyncResult.failure(true);
               }
             }
           }
+          syncErrors.inc();
           return PeerSyncResult.failure(false);
         }
       }
@@ -304,6 +331,7 @@ public class PeerSync  {
         if (!success) {
           log.info(msg() + "DONE. sync failed");
           shardHandler.cancelAll();
+          syncErrors.inc();
           return PeerSyncResult.failure();
         }
       }
@@ -318,8 +346,14 @@ public class PeerSync  {
       }
 
       log.info(msg() + "DONE. sync " + (success ? "succeeded" : "failed"));
+      if (!success) {
+        syncErrors.inc();
+      }
       return success ?  PeerSyncResult.success() : PeerSyncResult.failure();
     } finally {
+      if (timerContext != null) {
+        timerContext.close();
+      }
       MDCLoggingContext.clear();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index b79290d..c40eafd 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -40,6 +40,8 @@ import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.SolrException;
@@ -50,6 +52,9 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
@@ -71,7 +76,7 @@ import static org.apache.solr.update.processor.DistributingUpdateProcessorFactor
 
 
 /** @lucene.experimental */
-public class UpdateLog implements PluginInfoInitialized {
+public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
   private static final long STATUS_TIME = TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
   public static String LOG_FILENAME_PATTERN = "%s.%019d";
   public static String TLOG_NAME="tlog";
@@ -186,6 +191,14 @@ public class UpdateLog implements PluginInfoInitialized {
   List<Long> startingVersions;
   int startingOperation;  // last operation in the logs on startup
 
+  // metrics
+  protected Gauge<Integer> bufferedOpsGauge;
+  protected Gauge<Integer> replayLogsCountGauge;
+  protected Gauge<Long> replayBytesGauge;
+  protected Gauge<Integer> stateGauge;
+  protected Meter applyingBufferedOpsMeter;
+  protected Meter replayOpsMeter;
+
   public static class LogPtr {
     final long pointer;
     final long version;
@@ -333,7 +346,39 @@ public class UpdateLog implements PluginInfoInitialized {
       }
 
     }
+    core.getCoreMetricManager().registerMetricProducer(SolrInfoMBean.Category.TLOG.toString(), this);
+  }
 
+  @Override
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+    bufferedOpsGauge = () -> {
+      if (tlog == null) {
+        return 0;
+      } else if (state == State.APPLYING_BUFFERED) {
+        // numRecords counts header as a record
+        return tlog.numRecords() - 1 - recoveryInfo.adds - recoveryInfo.deleteByQuery - recoveryInfo.deletes - recoveryInfo.errors;
+      } else if (state == State.BUFFERING) {
+        // numRecords counts header as a record
+        return tlog.numRecords() - 1;
+      } else {
+        return 0;
+      }
+    };
+    replayLogsCountGauge = () -> logs.size();
+    replayBytesGauge = () -> {
+      if (state == State.REPLAYING) {
+        return getTotalLogsSize();
+      } else {
+        return 0L;
+      }
+    };
+    manager.register(registry, bufferedOpsGauge, true, "ops", scope, "buffered");
+    manager.register(registry, replayLogsCountGauge, true, "logs", scope, "replay", "remaining");
+    manager.register(registry, replayBytesGauge, true, "bytes", scope, "replay", "remaining");
+    applyingBufferedOpsMeter = manager.meter(registry, "ops", scope, "applying_buffered");
+    replayOpsMeter = manager.meter(registry, "ops", scope, "replay");
+    stateGauge = () -> state.ordinal();
+    manager.register(registry, stateGauge, true, "state", scope);
   }
 
   /**
@@ -1427,6 +1472,13 @@ public class UpdateLog implements PluginInfoInitialized {
               loglog.error("REPLAY_ERR: Exception replaying log", rsp.getException());
               throw rsp.getException();
             }
+            if (state == State.REPLAYING) {
+              replayOpsMeter.mark();
+            } else if (state == State.APPLYING_BUFFERED) {
+              applyingBufferedOpsMeter.mark();
+            } else {
+              // XXX should not happen?
+            }
           } catch (IOException ex) {
             recoveryInfo.errors++;
             loglog.warn("REPLAY_ERR: IOException reading log", ex);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index 4084ad7..57784b6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -27,9 +27,14 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -172,6 +177,16 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
 
       // make sure leader has not changed after bringing initial leader back
       assertEquals(nodePeerSynced, shardToLeaderJetty.get("shard1"));
+
+      // assert metrics
+      MetricRegistry registry = nodePeerSynced.jetty.getCoreContainer().getMetricManager().registry("solr.core.collection1");
+      Map<String, Metric> metrics = registry.getMetrics();
+      assertTrue("REPLICATION.time present", metrics.containsKey("REPLICATION.time"));
+      assertTrue("REPLICATION.errors present", metrics.containsKey("REPLICATION.errors"));
+      Timer timer = (Timer)metrics.get("REPLICATION.time");
+      assertEquals(1L, timer.getCount());
+      Counter counter = (Counter)metrics.get("REPLICATION.errors");
+      assertEquals(0L, counter.getCount());
       success = true;
     } finally {
       System.clearProperty("solr.disableFingerprint");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
index e2f3bfd..164eeab 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
@@ -23,9 +23,14 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.Timer;
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -35,6 +40,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.cloud.ClusterStateUtil;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.util.TestInjection;
@@ -102,6 +108,26 @@ public class TestCloudRecovery extends SolrCloudTestCase {
     assertEquals(4, resp.getResults().getNumFound());
     // Make sure all nodes is recover from tlog
     assertEquals(4, countReplayLog.get());
+
+    // check metrics
+    int replicationCount = 0;
+    int errorsCount = 0;
+    int skippedCount = 0;
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      SolrMetricManager manager = jetty.getCoreContainer().getMetricManager();
+      List<String> registryNames = manager.registryNames().stream()
+          .filter(s -> s.startsWith("solr.core.")).collect(Collectors.toList());
+      for (String registry : registryNames) {
+        Map<String, Metric> metrics = manager.registry(registry).getMetrics();
+        Timer timer = (Timer)metrics.get("REPLICATION.time");
+        Counter counter = (Counter)metrics.get("REPLICATION.errors");
+        Counter skipped = (Counter)metrics.get("REPLICATION.skipped");
+        replicationCount += timer.getCount();
+        errorsCount += counter.getCount();
+        skippedCount += skipped.getCount();
+      }
+    }
+    assertEquals(2, replicationCount);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b8383db0/solr/core/src/test/org/apache/solr/search/TestRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 9b88ec4..7bd0951 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -19,6 +19,11 @@ package org.apache.solr.search;
 
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.noggit.ObjectBuilder;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
@@ -55,7 +60,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
 
   // TODO: fix this test to not require FSDirectory
   static String savedFactory;
-  
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     savedFactory = System.getProperty("solr.DirectoryFactory");
@@ -72,6 +77,12 @@ public class TestRecovery extends SolrTestCaseJ4 {
     }
   }
 
+  private Map<String, Metric> getMetrics() {
+    SolrMetricManager manager = h.getCoreContainer().getMetricManager();
+    MetricRegistry registry = manager.registry(h.getCore().getCoreMetricManager().getRegistryName());
+    return registry.getMetrics();
+  }
+
   @Test
   public void testLogReplay() throws Exception {
     try {
@@ -107,6 +118,9 @@ public class TestRecovery extends SolrTestCaseJ4 {
 
       h.close();
       createCore();
+
+      Map<String, Metric> metrics = getMetrics(); // live map view
+
       // Solr should kick this off now
       // h.getCore().getUpdateHandler().getUpdateLog().recoverFromLog();
 
@@ -117,6 +131,15 @@ public class TestRecovery extends SolrTestCaseJ4 {
       // make sure we can still access versions after a restart
       assertJQ(req("qt","/get", "getVersions",""+versions.size()),"/versions==" + versions);
 
+      assertEquals(UpdateLog.State.REPLAYING, h.getCore().getUpdateHandler().getUpdateLog().getState());
+      // check metrics
+      Gauge<Integer> state = (Gauge<Integer>)metrics.get("TLOG.state");
+      assertEquals(UpdateLog.State.REPLAYING.ordinal(), state.getValue().intValue());
+      Gauge<Integer> replayingLogs = (Gauge<Integer>)metrics.get("TLOG.replay.remaining.logs");
+      assertTrue(replayingLogs.getValue().intValue() > 0);
+      Gauge<Long> replayingDocs = (Gauge<Long>)metrics.get("TLOG.replay.remaining.bytes");
+      assertTrue(replayingDocs.getValue().longValue() > 0);
+
       // unblock recovery
       logReplay.release(1000);
 
@@ -128,6 +151,10 @@ public class TestRecovery extends SolrTestCaseJ4 {
 
       assertJQ(req("q","*:*") ,"/response/numFound==3");
 
+      Meter replayDocs = (Meter)metrics.get("TLOG.replay.ops");
+      assertEquals(5L, replayDocs.getCount());
+      assertEquals(UpdateLog.State.ACTIVE.ordinal(), state.getValue().intValue());
+
       // make sure we can still access versions after recovery
       assertJQ(req("qt","/get", "getVersions",""+versions.size()) ,"/versions==" + versions);
 
@@ -195,15 +222,20 @@ public class TestRecovery extends SolrTestCaseJ4 {
       clearIndex();
       assertU(commit());
 
+      Map<String, Metric> metrics = getMetrics();
+
       assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
       ulog.bufferUpdates();
       assertEquals(UpdateLog.State.BUFFERING, ulog.getState());
+
       Future<UpdateLog.RecoveryInfo> rinfoFuture = ulog.applyBufferedUpdates();
       assertTrue(rinfoFuture == null);
       assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
 
       ulog.bufferUpdates();
       assertEquals(UpdateLog.State.BUFFERING, ulog.getState());
+      Gauge<Integer> state = (Gauge<Integer>)metrics.get("TLOG.state");
+      assertEquals(UpdateLog.State.BUFFERING.ordinal(), state.getValue().intValue());
 
       // simulate updates from a leader
       updateJ(jsonAdd(sdoc("id","B1", "_version_","1010")), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
@@ -235,6 +267,8 @@ public class TestRecovery extends SolrTestCaseJ4 {
           ,"=={'doc':null}"
       );
 
+      Gauge<Integer> bufferedOps = (Gauge<Integer>)metrics.get("TLOG.buffered.ops");
+      assertEquals(6, bufferedOps.getValue().intValue());
 
       rinfoFuture = ulog.applyBufferedUpdates();
       assertTrue(rinfoFuture != null);
@@ -246,6 +280,8 @@ public class TestRecovery extends SolrTestCaseJ4 {
       UpdateLog.RecoveryInfo rinfo = rinfoFuture.get();
       assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
 
+      Meter applyingBuffered = (Meter)metrics.get("TLOG.applying_buffered.ops");
+      assertEquals(6L, applyingBuffered.getCount());
 
       assertJQ(req("qt","/get", "getVersions","6")
           ,"=={'versions':[-2010,1030,1020,-1017,1015,1010]}"
@@ -312,6 +348,8 @@ public class TestRecovery extends SolrTestCaseJ4 {
       assertEquals(1, recInfo.deleteByQuery);
 
       assertEquals(UpdateLog.State.ACTIVE, ulog.getState()); // leave each test method in a good state
+
+      assertEquals(0, bufferedOps.getValue().intValue());
     } finally {
       DirectUpdateHandler2.commitOnClose = true;
       UpdateLog.testing_logReplayHook = null;


[33/50] [abbrv] lucene-solr:jira/solr-5944: LUCENE-7559: UH: Also expose OffsetsEnum, and test it's exposure

Posted by ho...@apache.org.
LUCENE-7559: UH: Also expose OffsetsEnum, and test it's exposure


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

Branch: refs/heads/jira/solr-5944
Commit: 7435ab18786a43f9004c44713446380711137b79
Parents: 04f45aa
Author: David Smiley <ds...@apache.org>
Authored: Thu Jan 12 00:29:17 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Jan 12 00:29:17 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +-
 .../search/uhighlight/FieldHighlighter.java     |  4 +-
 .../lucene/search/uhighlight/OffsetsEnum.java   | 45 +++++++++++-----
 .../TestUnifiedHighlighterExtensibility.java    | 57 +++++++++++++++++++-
 4 files changed, 92 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7435ab18/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 109a534..4912920 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -252,8 +252,8 @@ Other
 
 * LUCENE-7534: fix smokeTestRelease.py to run on Cygwin (Mikhail Khludnev)
 
-* LUCENE-7559: UnifiedHighlighter: Make Passage more exposed to allow passage creation to
-  be customized. (David Smiley)
+* LUCENE-7559: UnifiedHighlighter: Make Passage and OffsetsEnum more exposed to allow
+  passage creation to be customized. (David Smiley)
 
 * LUCENE-7599: Simplify TestRandomChains using Java's built-in Predicate and
   Function interfaces. (Ahmet Arslan via Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7435ab18/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
index 1caa739..cc9f318 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
@@ -138,7 +138,7 @@ public class FieldHighlighter {
 
     PriorityQueue<OffsetsEnum> offsetsEnumQueue = new PriorityQueue<>(offsetsEnums.size() + 1);
     for (OffsetsEnum off : offsetsEnums) {
-      off.weight = scorer.weight(contentLength, off.postingsEnum.freq());
+      off.setWeight(scorer.weight(contentLength, off.freq()));
       off.nextPosition(); // go to first position
       offsetsEnumQueue.add(off);
     }
@@ -214,7 +214,7 @@ public class FieldHighlighter {
           break;
         }
       }
-      passage.setScore(passage.getScore() + off.weight * scorer.tf(tf, passage.getEndOffset() - passage.getStartOffset()));
+      passage.setScore(passage.getScore() + off.getWeight() * scorer.tf(tf, passage.getEndOffset() - passage.getStartOffset()));
     }
 
     Passage[] passages = passageQueue.toArray(new Passage[passageQueue.size()]);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7435ab18/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
index db1ea1f..708f5c3 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
@@ -18,24 +18,25 @@ package org.apache.lucene.search.uhighlight;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.List;
 import java.util.Objects;
 
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.util.BytesRef;
 
 /**
- * Holds the term &amp; PostingsEnum, and info for tracking the occurrences of a term within the text.
- * It is advanced with the underlying postings and is placed in a priority queue by highlightOffsetsEnums
- * based on the start offset.
+ * Holds the term ({@link BytesRef}), {@link PostingsEnum}, offset iteration tracking.
+ * It is advanced with the underlying postings and is placed in a priority queue by
+ * {@link FieldHighlighter#highlightOffsetsEnums(List)} based on the start offset.
  *
  * @lucene.internal
  */
 public class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable {
   private final BytesRef term;
-  final PostingsEnum postingsEnum; // with offsets
+  private final PostingsEnum postingsEnum; // with offsets
 
-  float weight; // set once in highlightOffsetsEnums
-  private int pos = 0; // the occurrence counter of this term within the text being highlighted.
+  private float weight; // set once in highlightOffsetsEnums
+  private int posCounter = 0; // the occurrence counter of this term within the text being highlighted.
 
   public OffsetsEnum(BytesRef term, PostingsEnum postingsEnum) throws IOException {
     this.term = term; // can be null
@@ -65,29 +66,47 @@ public class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable {
     }
   }
 
-  BytesRef getTerm() throws IOException {
+  /** The term at this position; usually always the same. This term is a reference that is safe to continue to refer to,
+   * even after we move to next position. */
+  public BytesRef getTerm() throws IOException {
     // TODO TokenStreamOffsetStrategy could override OffsetsEnum; then remove this hack here
     return term != null ? term : postingsEnum.getPayload(); // abusing payload like this is a total hack!
   }
 
-  boolean hasMorePositions() throws IOException {
-    return pos < postingsEnum.freq();
+  public PostingsEnum getPostingsEnum() {
+    return postingsEnum;
   }
 
-  void nextPosition() throws IOException {
+  public int freq() throws IOException {
+    return postingsEnum.freq();
+  }
+
+  public boolean hasMorePositions() throws IOException {
+    return posCounter < postingsEnum.freq();
+  }
+
+  public void nextPosition() throws IOException {
     assert hasMorePositions();
-    pos++;
+    posCounter++;
     postingsEnum.nextPosition();
   }
 
-  int startOffset() throws IOException {
+  public int startOffset() throws IOException {
     return postingsEnum.startOffset();
   }
 
-  int endOffset() throws IOException {
+  public int endOffset() throws IOException {
     return postingsEnum.endOffset();
   }
 
+  public float getWeight() {
+    return weight;
+  }
+
+  public void setWeight(float weight) {
+    this.weight = weight;
+  }
+
   @Override
   public void close() throws IOException {
     // TODO TokenStreamOffsetStrategy could override OffsetsEnum; then this base impl would be no-op.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7435ab18/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
index 10757a5..08055a2 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
@@ -40,12 +40,16 @@ import org.apache.lucene.search.uhighlight.Passage;
 import org.apache.lucene.search.uhighlight.PassageFormatter;
 import org.apache.lucene.search.uhighlight.PassageScorer;
 import org.apache.lucene.search.uhighlight.PhraseHelper;
+import org.apache.lucene.search.uhighlight.SplittingBreakIterator;
 import org.apache.lucene.search.uhighlight.UnifiedHighlighter;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.junit.Test;
 
+/**
+ * Helps us be aware of visibility/extensibility concerns.
+ */
 public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
 
   /**
@@ -144,7 +148,19 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
 
       @Override
       protected FieldHighlighter getFieldHighlighter(String field, Query query, Set<Term> allTerms, int maxPassages) {
-        return super.getFieldHighlighter(field, query, allTerms, maxPassages);
+        // THIS IS A COPY of the superclass impl; but use CustomFieldHighlighter
+        BytesRef[] terms = filterExtractedTerms(getFieldMatcher(field), allTerms);
+        Set<HighlightFlag> highlightFlags = getFlags(field);
+        PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
+        CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
+        OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
+        return new CustomFieldHighlighter(field,
+            getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags),
+            new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
+            getScorer(field),
+            maxPassages,
+            getMaxNoHighlightPassages(field),
+            getFormatter(field));
       }
 
       @Override
@@ -185,4 +201,43 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
     assertEquals(fieldHighlighter.getField(), fieldName);
   }
 
+  /** Tests maintaining extensibility/visibility of {@link org.apache.lucene.search.uhighlight.FieldHighlighter} out of package. */
+  private class CustomFieldHighlighter extends FieldHighlighter {
+    CustomFieldHighlighter(String field, FieldOffsetStrategy fieldOffsetStrategy, BreakIterator breakIterator, PassageScorer passageScorer, int maxPassages, int maxNoHighlightPassages, PassageFormatter passageFormatter) {
+      super(field, fieldOffsetStrategy, breakIterator, passageScorer, maxPassages, maxNoHighlightPassages, passageFormatter);
+    }
+
+    @Override
+    public Object highlightFieldForDoc(IndexReader reader, int docId, String content) throws IOException {
+      return super.highlightFieldForDoc(reader, docId, content);
+    }
+
+    @Override
+    protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums) throws IOException {
+      // TEST OffsetsEnums & Passage visibility
+
+      // this code never runs; just for compilation
+      OffsetsEnum oe = new OffsetsEnum(null, EMPTY);
+      oe.getTerm();
+      oe.getPostingsEnum();
+      oe.freq();
+      oe.hasMorePositions();
+      oe.nextPosition();
+      oe.startOffset();
+      oe.endOffset();
+      oe.getWeight();
+      oe.setWeight(2f);
+
+      Passage p = new Passage();
+      p.setStartOffset(0);
+      p.setEndOffset(9);
+      p.setScore(1f);
+      p.addMatch(1, 2, new BytesRef());
+      p.reset();
+      p.sort();
+      //... getters are all exposed; custom PassageFormatter impls uses them
+
+      return super.highlightOffsetsEnums(offsetsEnums);
+    }
+  }
 }