You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/10/24 01:58:51 UTC

[01/12] lucene-solr:jira/http2: SOLR-12729: Unlock the shard on error.

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/http2 a10c09e5a -> 6226f0f38


SOLR-12729: Unlock the shard on error.


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

Branch: refs/heads/jira/http2
Commit: 8c70811f3a2a4deab8186b187909ac5c3615e6fb
Parents: 5de6332
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Oct 22 11:27:35 2018 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Oct 22 11:28:09 2018 +0200

----------------------------------------------------------------------
 .../apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8c70811f/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index 43e12ce..a7471eb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -1156,6 +1156,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     Map<String, Object> props = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>())
         .computeIfAbsent(sliceName.get(), ss -> new ConcurrentHashMap<>());
     if (props.containsKey(BUFFERED_UPDATES)) {
+      SplitShardCmd.unlockForSplit(cloudManager, collectionName, sliceName.get());
       throw new Exception("--- SOLR-12729: Overlapping splitShard commands for " + collectionName + "/" + sliceName.get());
     }
     props.put(BUFFERED_UPDATES, new AtomicLong());


[06/12] lucene-solr:jira/http2: SOLR-11522: Moved the _get methods to a separate interafce and keep MapWriter clean

Posted by da...@apache.org.
SOLR-11522: Moved the _get methods to a separate interafce and keep MapWriter clean


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

Branch: refs/heads/jira/http2
Commit: 576d28f643a89de832b59a783ce729402d70fb9f
Parents: d799fd5
Author: noble <no...@apache.org>
Authored: Tue Oct 23 12:30:54 2018 +1100
Committer: noble <no...@apache.org>
Committed: Tue Oct 23 12:35:23 2018 +1100

----------------------------------------------------------------------
 .../java/org/apache/solr/common/MapWriter.java  | 48 +----------
 .../org/apache/solr/common/NavigableObject.java | 89 ++++++++++++++++++++
 .../java/org/apache/solr/common/util/Utils.java | 12 +--
 3 files changed, 96 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/576d28f6/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/MapWriter.java b/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
index d2f14d8..d6c9efb 100644
--- a/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.function.BiConsumer;
 import java.util.function.BiPredicate;
 
 import org.apache.solr.common.util.Utils;
@@ -33,7 +32,7 @@ import org.apache.solr.common.util.Utils;
  * This avoids creating map instances and is supposed to be memory efficient.
  * If the entries are primitives, unnecessary boxing is also avoided.
  */
-public interface MapWriter extends MapSerializable {
+public interface MapWriter extends MapSerializable , NavigableObject {
 
   default String jsonStr(){
     return Utils.toJSONString(this);
@@ -81,51 +80,6 @@ public interface MapWriter extends MapSerializable {
 
   void writeMap(EntryWriter ew) throws IOException;
 
-  /**Get a child object value using json path
-   *
-   * @param path the full path to that object such as a/b/c[4]/d etc
-   * @param def the default
-   * @return the found value or default
-   */
-  default Object _get(String path, Object def) {
-    Object v = Utils.getObjectByPath(this, false, path);
-    return v == null ? def : v;
-  }
-
-  default String _getStr(String path, String def) {
-    Object v = Utils.getObjectByPath(this, false, path);
-    return v == null ? def : String.valueOf(v);
-  }
-
-  default void _forEachEntry(String path, BiConsumer fun) {
-    Utils.forEachMapEntry(this, path, fun);
-  }
-
-  default void _forEachEntry(List<String> path, BiConsumer fun) {
-    Utils.forEachMapEntry(this, path, fun);
-  }
-
-  default void _forEachEntry(BiConsumer fun) {
-    Utils.forEachMapEntry(this, fun);
-  }
-
-  /**
-   * Get a child object value using json path
-   *
-   * @param path the full path to that object such as ["a","b","c[4]","d"] etc
-   * @param def  the default
-   * @return the found value or default
-   */
-  default Object _get(List<String> path, Object def) {
-    Object v = Utils.getObjectByPath(this, false, path);
-    return v == null ? def : v;
-  }
-
-  default String _getStr(List<String> path, String def) {
-    Object v = Utils.getObjectByPath(this, false, path);
-    return v == null ? def : String.valueOf(v);
-  }
-
   /**
    * An interface to push one entry at a time to the output.
    * The order of the keys is not defined, but we assume they are distinct -- don't call {@code put} more than once

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/576d28f6/solr/solrj/src/java/org/apache/solr/common/NavigableObject.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/NavigableObject.java b/solr/solrj/src/java/org/apache/solr/common/NavigableObject.java
new file mode 100644
index 0000000..be50a17
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/NavigableObject.java
@@ -0,0 +1,89 @@
+/*
+ * 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.common;
+
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import org.apache.solr.common.util.Utils;
+
+/**This class contains helper methods for navigating deeply nested Objects. Keep in mind that
+ * it may be expensive depending on the underlying implementation. each level needs an extra lookup
+ * and the lookup may be as expensive as O(log(n)) to O(o) depending on the underlying impl
+ *
+ */
+public interface NavigableObject {
+  /**Get a child object value using json path. This usually ends up in String split operations
+   *  use a list of strings where performance is important
+   *
+   * @param path the full path to that object such as a/b/c[4]/d etc
+   * @param def the default
+   * @return the found value or default
+   */
+  default Object _get(String path, Object def) {
+    Object v = Utils.getObjectByPath(this, false, path);
+    return v == null ? def : v;
+  }
+
+  /**get the value as a String. useful in tests
+   *
+   * @param path the full path
+   * @param def default value
+   */
+  default String _getStr(String path, String def) {
+    Object v = Utils.getObjectByPath(this, false, path);
+    return v == null ? def : String.valueOf(v);
+  }
+
+  /**Iterate through the entries of a navigable Object at a certain path
+   * @param path the json path
+   */
+  default void _forEachEntry(String path, BiConsumer fun) {
+    Utils.forEachMapEntry(this, path, fun);
+  }
+
+  /**Iterate through the entries of a navigable Object at a certain path
+   * @param path the json path
+   */
+  default void _forEachEntry(List<String> path, BiConsumer fun) {
+    Utils.forEachMapEntry(this, path, fun);
+  }
+
+  /**Iterate through each entry in this object
+   */
+  default void _forEachEntry(BiConsumer fun) {
+    Utils.forEachMapEntry(this, fun);
+  }
+
+  /**
+   * Get a child object value using json path
+   *
+   * @param path the full path to that object such as ["a","b","c[4]","d"] etc
+   * @param def  the default
+   * @return the found value or default
+   */
+  default Object _get(List<String> path, Object def) {
+    Object v = Utils.getObjectByPath(this, false, path);
+    return v == null ? def : v;
+  }
+
+  default String _getStr(List<String> path, String def) {
+    Object v = Utils.getObjectByPath(this, false, path);
+    return v == null ? def : String.valueOf(v);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/576d28f6/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index bdfd03e..a1b34a2 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -107,14 +107,14 @@ public class Utils {
     return mutable ? copy : Collections.unmodifiableMap(copy);
   }
 
-  public static void forEachMapEntry(MapWriter mw, String path, BiConsumer fun) {
-    Object o = Utils.getObjectByPath(mw, false, path);
-    forEachMapEntry(o, fun);
+  public static void forEachMapEntry(Object o, String path, BiConsumer fun) {
+    Object val = Utils.getObjectByPath(o, false, path);
+    forEachMapEntry(val, fun);
   }
 
-  public static void forEachMapEntry(MapWriter mw, List<String> path, BiConsumer fun) {
-    Object o = Utils.getObjectByPath(mw, false, path);
-    forEachMapEntry(o, fun);
+  public static void forEachMapEntry(Object o, List<String> path, BiConsumer fun) {
+    Object val = Utils.getObjectByPath(o, false, path);
+    forEachMapEntry(val, fun);
   }
 
   public static void forEachMapEntry(Object o, BiConsumer fun) {


[09/12] lucene-solr:jira/http2: SOLR-12879 - registered MinHashQParserPlugin to QParserPlugin as min_hash

Posted by da...@apache.org.
SOLR-12879 - registered MinHashQParserPlugin to QParserPlugin as min_hash


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

Branch: refs/heads/jira/http2
Commit: 2e757f6c257687ab713f88b6a07cf4a355e4cf66
Parents: 9df96d2
Author: Tommaso Teofili <te...@adobe.com>
Authored: Tue Oct 23 09:12:46 2018 +0200
Committer: Tommaso Teofili <te...@adobe.com>
Committed: Tue Oct 23 09:12:46 2018 +0200

----------------------------------------------------------------------
 .../src/java/org/apache/solr/search/MinHashQParserPlugin.java     | 3 +++
 solr/core/src/java/org/apache/solr/search/QParserPlugin.java      | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2e757f6c/solr/core/src/java/org/apache/solr/search/MinHashQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/MinHashQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/MinHashQParserPlugin.java
index 4567e25..e63665b 100644
--- a/solr/core/src/java/org/apache/solr/search/MinHashQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/MinHashQParserPlugin.java
@@ -24,6 +24,9 @@ import org.apache.solr.request.SolrQueryRequest;
  * {@link QParserPlugin} based on {@link MinHashQParser}.
  */
 public class MinHashQParserPlugin extends QParserPlugin {
+
+  public static final String NAME = "min_hash";
+
   @Override
   public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
     return new MinHashQParser(qstr, localParams, params, req);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2e757f6c/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
index b20c3c8..d602c7a 100644
--- a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
@@ -84,6 +84,7 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI
     map.put(PayloadScoreQParserPlugin.NAME, new PayloadScoreQParserPlugin());
     map.put(PayloadCheckQParserPlugin.NAME, new PayloadCheckQParserPlugin());
     map.put(BoolQParserPlugin.NAME, new BoolQParserPlugin());
+    map.put(MinHashQParserPlugin.NAME, new MinHashQParserPlugin());
 
     standardPlugins = Collections.unmodifiableMap(map);
   }


[07/12] lucene-solr:jira/http2: SOLR-11812: Remove LIROnShardRestartTest since the transition from old lir to new lir is no longer supported

Posted by da...@apache.org.
SOLR-11812: Remove LIROnShardRestartTest since the transition from old lir to new lir is no longer supported


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

Branch: refs/heads/jira/http2
Commit: 7512cd9425319fb620c1992053a5d4be7cd9229d
Parents: 576d28f
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Oct 23 09:58:59 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Oct 23 09:58:59 2018 +0700

----------------------------------------------------------------------
 .../solr/cloud/LIROnShardRestartTest.java       | 262 -------------------
 1 file changed, 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7512cd94/solr/core/src/test/org/apache/solr/cloud/LIROnShardRestartTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LIROnShardRestartTest.java b/solr/core/src/test/org/apache/solr/cloud/LIROnShardRestartTest.java
deleted file mode 100644
index 54742a9..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/LIROnShardRestartTest.java
+++ /dev/null
@@ -1,262 +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.solr.cloud;
-
-import java.lang.invoke.MethodHandles;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.ClusterStateUtil;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.TimeSource;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.update.processor.DistributedUpdateProcessor;
-import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
-import org.apache.solr.util.TimeOut;
-import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@LuceneTestCase.Nightly
-@LuceneTestCase.Slow
-@Deprecated
-public class LIROnShardRestartTest extends SolrCloudTestCase {
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
-    System.setProperty("solr.ulog.numRecordsToKeep", "1000");
-
-    configureCluster(3)
-        .addConfig("conf", configset("cloud-minimal"))
-        .configure();
-  }
-
-  @AfterClass
-  public static void tearDownCluster() throws Exception {
-    System.clearProperty("solr.directoryFactory");
-    System.clearProperty("solr.ulog.numRecordsToKeep");
-  }
-
-  public void testAllReplicasInLIR() throws Exception {
-    String collection = "allReplicasInLIR";
-    CollectionAdminRequest.createCollection(collection, 1, 3)
-        .process(cluster.getSolrClient());
-    cluster.getSolrClient().add(collection, new SolrInputDocument("id", "1"));
-    cluster.getSolrClient().add(collection, new SolrInputDocument("id", "2"));
-    cluster.getSolrClient().commit(collection);
-
-    DocCollection docCollection = getCollectionState(collection);
-    Slice shard1 = docCollection.getSlice("shard1");
-    Replica newLeader = shard1.getReplicas(rep -> !rep.getName().equals(shard1.getLeader().getName())).get(random().nextInt(2));
-    JettySolrRunner jettyOfNewLeader = cluster.getJettySolrRunners().stream()
-        .filter(jetty -> jetty.getNodeName().equals(newLeader.getNodeName()))
-        .findAny().get();
-    assertNotNull(jettyOfNewLeader);
-
-    // randomly add too many docs to peer sync to one replica so that only one random replica is the valid leader
-    // the versions don't matter, they just have to be higher than what the last 2 docs got
-    try (HttpSolrClient client = getHttpSolrClient(jettyOfNewLeader.getBaseUrl().toString())) {
-      ModifiableSolrParams params = new ModifiableSolrParams();
-      params.set(DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM, DistributedUpdateProcessor.DistribPhase.FROMLEADER.toString());
-
-      for (int i = 0; i < 101; i++) {
-        UpdateRequest ureq = new UpdateRequest();
-        ureq.setParams(new ModifiableSolrParams(params));
-        ureq.add(sdoc("id", 3 + i, "_version_", Long.MAX_VALUE - 1 - i));
-        ureq.process(client, collection);
-      }
-      client.commit(collection);
-    }
-
-    ChaosMonkey.stop(cluster.getJettySolrRunners());
-    assertTrue("Timeout waiting for all not live",
-        ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 45000));
-
-    try (ZkShardTerms zkShardTerms = new ZkShardTerms(collection, "shard1", cluster.getZkClient())) {
-      for (Replica replica : docCollection.getReplicas()) {
-        zkShardTerms.removeTerm(replica.getName());
-      }
-    }
-
-    Map<String,Object> stateObj = Utils.makeMap();
-    stateObj.put(ZkStateReader.STATE_PROP, "down");
-    stateObj.put("createdByNodeName", "test");
-    stateObj.put("createdByCoreNodeName", "test");
-    byte[] znodeData = Utils.toJSON(stateObj);
-
-    for (Replica replica : docCollection.getReplicas()) {
-      try {
-        cluster.getZkClient().makePath("/collections/" + collection + "/leader_initiated_recovery/shard1/" + replica.getName(),
-            znodeData, true);
-      } catch (KeeperException.NodeExistsException e) {
-
-      }
-    }
-
-    ChaosMonkey.start(cluster.getJettySolrRunners());
-    waitForState("Timeout waiting for active replicas", collection, clusterShape(1, 3));
-
-    assertEquals(103, cluster.getSolrClient().query(collection, new SolrQuery("*:*")).getResults().getNumFound());
-
-
-    // now expire each node
-    for (Replica replica : docCollection.getReplicas()) {
-      try {
-        // todo remove the condition for skipping leader after SOLR-12166 is fixed
-        if (newLeader.getName().equals(replica.getName())) continue;
-
-        cluster.getZkClient().makePath("/collections/" + collection + "/leader_initiated_recovery/shard1/" + replica.getName(),
-            znodeData, true);
-      } catch (KeeperException.NodeExistsException e) {
-
-      }
-    }
-
-    // only 2 replicas join the election and all of them are in LIR state, no one should win the election
-    List<String> oldElectionNodes = getElectionNodes(collection, "shard1", cluster.getZkClient());
-
-    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
-      expire(jetty);
-    }
-
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.CURRENT_TIME);
-    while (!timeOut.hasTimedOut()) {
-      List<String> electionNodes = getElectionNodes(collection, "shard1", cluster.getZkClient());
-      electionNodes.retainAll(oldElectionNodes);
-      if (electionNodes.isEmpty()) break;
-    }
-    assertFalse("Timeout waiting for replicas rejoin election", timeOut.hasTimedOut());
-    try {
-      waitForState("Timeout waiting for active replicas", collection, clusterShape(1, 3));
-    } catch (Throwable th) {
-      String electionPath = "/collections/allReplicasInLIR/leader_elect/shard1/election/";
-      List<String> children = zkClient().getChildren(electionPath, null, true);
-      log.info("Election queue {}", children);
-      throw th;
-    }
-
-    assertEquals(103, cluster.getSolrClient().query(collection, new SolrQuery("*:*")).getResults().getNumFound());
-
-    CollectionAdminRequest.deleteCollection(collection).process(cluster.getSolrClient());
-  }
-
-  public void expire(JettySolrRunner jetty) {
-    CoreContainer cores = jetty.getCoreContainer();
-    ChaosMonkey.causeConnectionLoss(jetty);
-    long sessionId = cores.getZkController().getZkClient()
-        .getSolrZooKeeper().getSessionId();
-    cluster.getZkServer().expire(sessionId);
-  }
-
-
-  public void testSeveralReplicasInLIR() throws Exception {
-    String collection = "severalReplicasInLIR";
-    CollectionAdminRequest.createCollection(collection, 1, 3)
-        .process(cluster.getSolrClient());
-    cluster.getSolrClient().add(collection, new SolrInputDocument("id", "1"));
-    cluster.getSolrClient().add(collection, new SolrInputDocument("id", "2"));
-    cluster.getSolrClient().commit(collection);
-
-    DocCollection docCollection = getCollectionState(collection);
-    Map<JettySolrRunner, String> nodeNameToJetty = cluster.getJettySolrRunners().stream()
-        .collect(Collectors.toMap(jetty -> jetty, JettySolrRunner::getNodeName));
-    ChaosMonkey.stop(cluster.getJettySolrRunners());
-    assertTrue("Timeout waiting for all not live",
-        ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 45000));
-
-    try (ZkShardTerms zkShardTerms = new ZkShardTerms(collection, "shard1", cluster.getZkClient())) {
-      for (Replica replica : docCollection.getReplicas()) {
-        zkShardTerms.removeTerm(replica.getName());
-      }
-    }
-
-    Map<String,Object> stateObj = Utils.makeMap();
-    stateObj.put(ZkStateReader.STATE_PROP, "down");
-    stateObj.put("createdByNodeName", "test");
-    stateObj.put("createdByCoreNodeName", "test");
-    byte[] znodeData = Utils.toJSON(stateObj);
-
-    Replica replicaNotInLIR = docCollection.getReplicas().get(random().nextInt(3));
-    for (Replica replica : docCollection.getReplicas()) {
-      if (replica.getName().equals(replicaNotInLIR.getName())) continue;
-      try {
-        cluster.getZkClient().makePath("/collections/" + collection + "/leader_initiated_recovery/shard1/" + replica.getName(),
-            znodeData, true);
-      } catch (KeeperException.NodeExistsException e) {
-
-      }
-    }
-
-    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
-      if (nodeNameToJetty.get(jetty).equals(replicaNotInLIR.getNodeName())) continue;
-      jetty.start();
-    }
-    waitForState("Timeout waiting for no leader", collection, (liveNodes, collectionState) -> {
-      Replica leader = collectionState.getSlice("shard1").getLeader();
-      return leader == null;
-    });
-
-    // only 2 replicas join the election and all of them are in LIR state, no one should win the election
-    List<String> oldElectionNodes = getElectionNodes(collection, "shard1", cluster.getZkClient());
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.CURRENT_TIME);
-    while (!timeOut.hasTimedOut()) {
-      List<String> electionNodes = getElectionNodes(collection, "shard1", cluster.getZkClient());
-      electionNodes.retainAll(oldElectionNodes);
-      if (electionNodes.isEmpty()) break;
-    }
-    assertFalse("Timeout waiting for replicas rejoin election", timeOut.hasTimedOut());
-
-    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
-      if (nodeNameToJetty.get(jetty).equals(replicaNotInLIR.getNodeName())) {
-        jetty.start();
-      }
-    }
-    waitForState("Timeout waiting for new leader", collection, (liveNodes, collectionState) -> {
-      Replica leader = collectionState.getSlice("shard1").getLeader();
-      return leader != null;
-    });
-    waitForState("Timeout waiting for new leader", collection, clusterShape(1, 3));
-
-    assertEquals(2L, cluster.getSolrClient().query(collection, new SolrQuery("*:*")).getResults().getNumFound());
-    CollectionAdminRequest.deleteCollection(collection).process(cluster.getSolrClient());
-  }
-
-  private List<String> getElectionNodes(String collection, String shard, SolrZkClient client) throws KeeperException, InterruptedException {
-    return client.getChildren("/collections/"+collection+"/leader_elect/"+shard+LeaderElector.ELECTION_NODE, null, true);
-  }
-}


[11/12] lucene-solr:jira/http2: Add log.warn when a replica become leader after timeout

Posted by da...@apache.org.
Add log.warn when a replica become leader after timeout


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

Branch: refs/heads/jira/http2
Commit: e083b1501ebe21abaf95bcd93f89af12142fd1ee
Parents: 3e89b7a
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed Oct 24 08:56:01 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed Oct 24 08:56:01 2018 +0700

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/cloud/ElectionContext.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e083b150/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index d4f84f9..6d17de4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -522,6 +522,9 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
     long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS);
     while (!isClosed && !cc.isShutDown()) {
       if (System.nanoTime() > timeoutAt) {
+        log.warn("After waiting for {}ms, no other potential leader was found, {} try to become leader anyway (" +
+                "core_term:{}, highest_term:{})",
+            timeout, coreNodeName, zkShardTerms.getTerm(coreNodeName), zkShardTerms.getHighestTerm());
         return true;
       }
       if (replicasWithHigherTermParticipated(zkShardTerms, coreNodeName)) {


[05/12] lucene-solr:jira/http2: SOLR-5004: Allow a shard to be split into 'n' sub-shards using the collections API

Posted by da...@apache.org.
SOLR-5004: Allow a shard to be split into 'n' sub-shards using the collections API


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

Branch: refs/heads/jira/http2
Commit: d799fd53c7cd3a83442d6010dc48802d2fd8c7fb
Parents: fcaea07
Author: Anshum Gupta <an...@apache.org>
Authored: Mon Oct 22 14:59:10 2018 -0700
Committer: Anshum Gupta <an...@apache.org>
Committed: Mon Oct 22 15:01:24 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../cloud/api/collections/SplitShardCmd.java    |  23 +++--
 .../solr/handler/admin/CollectionsHandler.java  |   9 +-
 .../org/apache/solr/cloud/SplitShardTest.java   | 102 +++++++++++++++++++
 solr/solr-ref-guide/src/collections-api.adoc    |   5 +
 .../solrj/request/CollectionAdminRequest.java   |  12 +++
 .../solr/common/params/CommonAdminParams.java   |   2 +
 7 files changed, 149 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f26e180..81748b3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -151,6 +151,9 @@ New Features
 
 * SOLR-12846: Added support for "host" variable in autoscaling policy rules (noble)
 
+* SOLR-5004: Splitshard collections API now supports splitting into more than 2 sub-shards directly i.e. by providing a
+  numSubShards parameter (Christine Poerschke, Anshum Gupta)
+
 
 Other Changes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
index 2e68f91..aa4909d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
@@ -74,10 +74,15 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.AD
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
 
 
 public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final int MIN_NUM_SUB_SHARDS = 2;
+  // This is an arbitrary number that seems reasonable at this time.
+  private static final int MAX_NUM_SUB_SHARDS = 8;
+  private static final int DEFAULT_NUM_SUB_SHARDS = 2;
 
   private final OverseerCollectionMessageHandler ocmh;
 
@@ -122,7 +127,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
     }
 
     // find the leader for the shard
-    Replica parentShardLeader = null;
+    Replica parentShardLeader;
     try {
       parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice.get(), 10000);
     } catch (InterruptedException e) {
@@ -326,7 +331,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
           requestMap);
       t.stop();
 
-      log.debug("Index on shard: " + nodeName + " split into two successfully");
+      log.debug("Index on shard: {} split into {} successfully", nodeName, subShardNames.size());
 
       t = timings.sub("applyBufferedUpdates");
       // apply buffered updates on sub-shards
@@ -680,13 +685,13 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
                                 List<DocRouter.Range> subRanges, List<String> subSlices, List<String> subShardNames,
                                   boolean firstReplicaNrt) {
     String splitKey = message.getStr("split.key");
+    String rangesStr = message.getStr(CoreAdminParams.RANGES);
+
     DocRouter.Range range = parentSlice.getRange();
     if (range == null) {
       range = new PlainIdRouter().fullRange();
     }
     DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
-
-    String rangesStr = message.getStr(CoreAdminParams.RANGES);
     if (rangesStr != null) {
       String[] ranges = rangesStr.split(",");
       if (ranges.length == 0 || ranges.length == 1) {
@@ -740,8 +745,14 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         }
       }
     } else {
-      // todo: fixed to two partitions?
-      subRanges.addAll(router.partitionRange(2, range));
+      int numSubShards = message.getInt(NUM_SUB_SHARDS, DEFAULT_NUM_SUB_SHARDS);
+      log.info("{} set at: {}", NUM_SUB_SHARDS, numSubShards);
+
+      if(numSubShards < MIN_NUM_SUB_SHARDS || numSubShards > MAX_NUM_SUB_SHARDS)
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "A shard can only be split into "+MIN_NUM_SUB_SHARDS+" to " + MAX_NUM_SUB_SHARDS
+            + " subshards in one split request. Provided "+NUM_SUB_SHARDS+"=" + numSubShards);
+      subRanges.addAll(router.partitionRange(numSubShards, range));
     }
 
     for (int i = 0; i < subRanges.size(); i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/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 276ab94..dfb3c6b 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
@@ -143,6 +143,7 @@ import static org.apache.solr.common.params.CollectionAdminParams.WITH_COLLECTIO
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonAdminParams.IN_PLACE_MOVE;
+import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
 import static org.apache.solr.common.params.CommonAdminParams.SPLIT_METHOD;
 import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
 import static org.apache.solr.common.params.CommonParams.NAME;
@@ -638,6 +639,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       String shard = req.getParams().get(SHARD_ID_PROP);
       String rangesStr = req.getParams().get(CoreAdminParams.RANGES);
       String splitKey = req.getParams().get("split.key");
+      String numSubShards = req.getParams().get(NUM_SUB_SHARDS);
 
       if (splitKey == null && shard == null) {
         throw new SolrException(ErrorCode.BAD_REQUEST, "At least one of shard, or split.key should be specified.");
@@ -650,6 +652,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
         throw new SolrException(ErrorCode.BAD_REQUEST,
             "Only one of 'ranges' or 'split.key' should be specified");
       }
+      if (numSubShards != null && (splitKey != null || rangesStr != null)) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,
+            "numSubShards can not be specified with split.key or ranges parameters");
+      }
 
       Map<String, Object> map = copy(req.getParams(), null,
           COLLECTION_PROP,
@@ -658,7 +664,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           CoreAdminParams.RANGES,
           WAIT_FOR_FINAL_STATE,
           TIMING,
-          SPLIT_METHOD);
+          SPLIT_METHOD,
+          NUM_SUB_SHARDS);
       return copyPropertiesWithPrefix(req.getParams(), map, COLL_PROP_PREFIX);
     }),
     DELETESHARD_OP(DELETESHARD, (req, rsp, h) -> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java b/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java
new file mode 100644
index 0000000..a2a2dca
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.SolrException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SplitShardTest extends SolrCloudTestCase {
+
+  private final String COLLECTION_NAME = "splitshardtest-collection";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    cluster.deleteAllCollections();
+  }
+
+  @Test
+  public void doTest() throws IOException, SolrServerException {
+    CollectionAdminRequest
+        .createCollection(COLLECTION_NAME, "conf", 2, 1)
+        .setMaxShardsPerNode(100)
+        .process(cluster.getSolrClient());
+    CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(COLLECTION_NAME)
+        .setNumSubShards(5)
+        .setShardName("shard1");
+    splitShard.process(cluster.getSolrClient());
+    waitForState("Timed out waiting for sub shards to be active. Number of active shards=" +
+            cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getActiveSlices().size(),
+        COLLECTION_NAME, activeClusterShape(6, 1));
+
+    try {
+      splitShard = CollectionAdminRequest.splitShard(COLLECTION_NAME).setShardName("shard2").setNumSubShards(10);
+      splitShard.process(cluster.getSolrClient());
+      fail("SplitShard should throw an exception when numSubShards > 8");
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getMessage().contains("A shard can only be split into 2 to 8 subshards in one split request."));
+    }
+
+    try {
+      splitShard = CollectionAdminRequest.splitShard(COLLECTION_NAME).setShardName("shard2").setNumSubShards(1);
+      splitShard.process(cluster.getSolrClient());
+      fail("SplitShard should throw an exception when numSubShards < 2");
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getMessage().contains("A shard can only be split into 2 to 8 subshards in one split request. Provided numSubShards=1"));
+    }
+  }
+
+  @Test
+  public void multipleOptionsSplitTest() throws IOException, SolrServerException {
+    CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(COLLECTION_NAME)
+        .setNumSubShards(5)
+        .setRanges("0-c,d-7fffffff")
+        .setShardName("shard1");
+    boolean expectedException = false;
+    try {
+      splitShard.process(cluster.getSolrClient());
+      fail("An exception should have been thrown");
+    } catch (SolrException ex) {
+      expectedException = true;
+    }
+    assertTrue("Expected SolrException but it didn't happen", expectedException);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/solr/solr-ref-guide/src/collections-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index 0e4112d..a5021da 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -287,6 +287,11 @@ This parameter can be used to split a shard using a route key such that all docu
 +
 For example, suppose `split.key=A!` hashes to the range `12-15` and belongs to shard 'shard1' with range `0-20`. Splitting by this route key would yield three sub-shards with ranges `0-11`, `12-15` and `16-20`. Note that the sub-shard with the hash range of the route key may also contain documents for other route keys whose hash ranges overlap.
 
+`numSubShards`::
+The number of sub-shards to split the parent shard into. Allowed values for this are in the range of 2-8 and defaults to 2.
++
+This parameter can only be used when ranges or split.key are not specified.
+
 `splitMethod`::
 Currently two methods of shard splitting are supported:
 * `splitMethod=rewrite` (default) after selecting documents to retain in each partition this method creates sub-indexes from

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 0f6de19..4f26984 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -1153,6 +1153,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     protected String splitKey;
     protected String shard;
     protected String splitMethod;
+    protected Integer numSubShards;
 
     private Properties properties;
 
@@ -1164,6 +1165,15 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public SplitShard setRanges(String ranges) { this.ranges = ranges; return this; }
     public String getRanges() { return ranges; }
 
+    public Integer getNumSubShards() {
+      return numSubShards;
+    }
+
+    public SplitShard setNumSubShards(Integer numSubShards) {
+      this.numSubShards = numSubShards;
+      return this;
+    }
+
     public SplitShard setSplitMethod(String splitMethod) {
       this.splitMethod = splitMethod;
       return this;
@@ -1210,6 +1220,8 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       params.set("split.key", this.splitKey);
       params.set(CoreAdminParams.RANGES, ranges);
       params.set(CommonAdminParams.SPLIT_METHOD, splitMethod);
+      if(numSubShards != null)
+        params.set("numSubShards", numSubShards);
 
       if(properties != null) {
         addProperties(params, properties);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d799fd53/solr/solrj/src/java/org/apache/solr/common/params/CommonAdminParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonAdminParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonAdminParams.java
index c12ee32..c080342 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonAdminParams.java
@@ -27,6 +27,8 @@ public interface CommonAdminParams
   String IN_PLACE_MOVE = "inPlaceMove";
   /** Method to use for shard splitting. */
   String SPLIT_METHOD = "splitMethod";
+  /** **/
+  String NUM_SUB_SHARDS = "numSubShards";
   /** Timeout for replicas to become active. */
   String TIMEOUT = "timeout";
 }


[02/12] lucene-solr:jira/http2: SOLR-12846: Added support for "host" variable in autoscaling policy rules

Posted by da...@apache.org.
SOLR-12846: Added support for "host" variable in autoscaling policy rules


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

Branch: refs/heads/jira/http2
Commit: c31a95d26170c7ffbd7e3177288891d6a14f4ab1
Parents: 8c70811
Author: Noble Paul <no...@apache.org>
Authored: Tue Oct 23 00:03:59 2018 +1100
Committer: Noble Paul <no...@apache.org>
Committed: Tue Oct 23 00:03:59 2018 +1100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../solr/cloud/autoscaling/TestPolicyCloud.java |   2 +
 .../solrj/cloud/autoscaling/Variable.java       |   6 +
 .../solr/autoscaling/testHostAttribute.json     | 119 +++++++++++++++++++
 .../solrj/cloud/autoscaling/TestPolicy2.java    |  22 +++-
 5 files changed, 150 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c31a95d2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 342d68c..f26e180 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -149,6 +149,9 @@ New Features
 * SOLR-12780: Add support for Leaky ReLU and TanH activations in contrib/ltr NeuralNetworkModel class.
   (Kamuela Lau, Christine Poerschke)
 
+* SOLR-12846: Added support for "host" variable in autoscaling policy rules (noble)
+
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c31a95d2/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
index bfd5878..4ff847d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
@@ -389,9 +389,11 @@ public class TestPolicyCloud extends SolrCloudTestCase {
       Map<String, Object> val = cloudManager.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
           "freedisk",
           "cores",
+          "host",
           "heapUsage",
           "sysLoadAvg"));
       assertNotNull(val.get("freedisk"));
+      assertNotNull(val.get("host"));
       assertNotNull(val.get("heapUsage"));
       assertNotNull(val.get("sysLoadAvg"));
       assertTrue(((Number) val.get("cores")).intValue() > 0);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c31a95d2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
index 0dfe282..72e571f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
@@ -205,6 +205,12 @@ public interface Variable {
         min = 0)
     NUMBER,
 
+    @Meta(name = "host",
+        type = String.class,
+        wildCards = Policy.EACH,
+        supportArrayVals = true)
+    HOST,
+
     @Meta(name = "STRING",
         type = String.class,
         wildCards = Policy.EACH,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c31a95d2/solr/solrj/src/test-files/solrj/solr/autoscaling/testHostAttribute.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test-files/solrj/solr/autoscaling/testHostAttribute.json b/solr/solrj/src/test-files/solrj/solr/autoscaling/testHostAttribute.json
new file mode 100644
index 0000000..7afd7ad
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/solr/autoscaling/testHostAttribute.json
@@ -0,0 +1,119 @@
+{"diagnostics":{
+  "sortedNodes":[{
+    "node":"127.0.0.191:63191_solr",
+    "isLive":true,
+    "cores":3.0,
+    "host":"127.0.0.191",
+    "sysprop.zone":"east",
+    "freedisk":1727.1459312438965,
+    "heapUsage":24.97510064011647,
+    "sysLoadAvg":272.75390625,
+    "totaldisk":1037.938980102539,
+    "replicas":{"zonesTest":{"shard1":[{"core_node5":{
+      "core":"zonesTest_shard1_replica_n2",
+      "leader":"true",
+      "base_url":"https://127.0.0.191:63191/solr",
+      "node_name":"127.0.0.191:63191_solr",
+      "state":"active",
+      "type":"NRT",
+      "force_set_state":"false",
+      "INDEX.sizeInGB":6.426125764846802E-8,
+      "shard":"shard1",
+      "collection":"zonesTest"}},
+      {"core_node7":{
+        "core":"zonesTest_shard1_replica_n4",
+        "base_url":"https://127.0.0.191:63191/solr",
+        "node_name":"127.0.0.191:63191_solr",
+        "state":"active",
+        "type":"NRT",
+        "force_set_state":"false",
+        "INDEX.sizeInGB":6.426125764846802E-8,
+        "shard":"shard1",
+        "collection":"zonesTest"}},
+      {"core_node12":{
+        "core":"zonesTest_shard1_replica_n10",
+        "base_url":"https://127.0.0.191:63191/solr",
+        "node_name":"127.0.0.191:63191_solr",
+        "state":"active",
+        "type":"NRT",
+        "force_set_state":"false",
+        "INDEX.sizeInGB":6.426125764846802E-8,
+        "shard":"shard1",
+        "collection":"zonesTest"}}]}}},
+    {
+      "node":"127.0.0.191:63192_solr",
+      "isLive":true,
+      "cores":3.0,
+      "host":"127.0.0.191",
+      "sysprop.zone":"east",
+      "freedisk":1727.1459312438965,
+      "heapUsage":24.98878807983566,
+      "sysLoadAvg":272.75390625,
+      "totaldisk":1037.938980102539,
+      "replicas":{"zonesTest":{"shard2":[{"core_node3":{
+        "core":"zonesTest_shard1_replica_n1",
+        "base_url":"https://127.0.0.191:63192/solr",
+        "node_name":"127.0.0.191:63192_solr",
+        "state":"active",
+        "type":"NRT",
+        "force_set_state":"false",
+        "INDEX.sizeInGB":6.426125764846802E-8,
+        "shard":"shard2",
+        "collection":"zonesTest"}},
+        {"core_node9":{
+          "core":"zonesTest_shard1_replica_n6",
+          "base_url":"https://127.0.0.191:63192/solr",
+          "node_name":"127.0.0.191:63192_solr",
+          "state":"active",
+          "type":"NRT",
+          "force_set_state":"false",
+          "INDEX.sizeInGB":6.426125764846802E-8,
+          "shard":"shard2",
+          "collection":"zonesTest"}},
+        {"core_node11":{
+          "core":"zonesTest_shard1_replica_n8",
+          "base_url":"https://127.0.0.191:63192/solr",
+          "node_name":"127.0.0.191:63192_solr",
+          "state":"active",
+          "type":"NRT",
+          "force_set_state":"false",
+          "INDEX.sizeInGB":6.426125764846802E-8,
+          "shard":"shard2",
+          "collection":"zonesTest"}}]}}},
+    {
+      "node":"127.0.0.219:63219_solr",
+      "isLive":true,
+      "cores":0.0,
+      "host":"127.0.0.219",
+      "sysprop.zone":"west",
+      "freedisk":1768.6174201965332,
+      "heapUsage":24.98878807983566,
+      "sysLoadAvg":272.75390625,
+      "totaldisk":1037.938980102539,
+      "replicas":{}},
+    {
+      "node":"127.0.0.219:63229_solr",
+      "isLive":true,
+      "cores":0.0,
+      "host":"127.0.0.219",
+      "sysprop.zone":"west",
+      "freedisk":1768.6174201965332,
+      "heapUsage":24.98878807983566,
+      "sysLoadAvg":272.75390625,
+      "totaldisk":1037.938980102539,
+      "replicas":{}}],
+  "liveNodes":["127.0.0.191:63191_solr",
+    "127.0.0.191:63192_solr",
+    "127.0.0.219:63219_solr",
+    "127.0.0.219:63229_solr"],
+  "config":{
+    "cluster-preferences":[{
+      "minimize":"cores",
+      "precision":1},
+      {"maximize":"freedisk",
+        "precision":100}],
+    "cluster-policy":[{
+      "replica":"<3",
+      "shard":"#EACH",
+      "host":["127.0.0.191",
+        "127.0.0.219:63219"]}]}}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c31a95d2/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
index c2ce4fa..cbcf3cd 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
@@ -291,7 +291,7 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
         return new DelegatingClusterStateProvider(null) {
 
           @Override
-          public ClusterState getClusterState() throws IOException {
+          public ClusterState getClusterState() {
             return clusterState;
           }
 
@@ -309,7 +309,25 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
     };
   }
 
-  public void testSysPropSuggestions() throws IOException {
+  public void testHostAttribute() {
+    Map<String, Object> m = (Map<String, Object>) loadFromResource("testHostAttribute.json");
+    Map<String, Object> conf = (Map<String, Object>) Utils.getObjectByPath(m, false, "diagnostics/config");
+    Policy policy = new Policy(conf);
+    SolrCloudManager cloudManagerFromDiagnostics = createCloudManagerFromDiagnostics(m);
+    Policy.Session session = policy.createSession(cloudManagerFromDiagnostics);
+    List<Violation> violations = session.getViolations();
+    for (Violation violation : violations) {
+      assertEquals(1.0d, violation.replicaCountDelta.doubleValue(), 0.0001);
+    }
+    assertEquals(2, violations.size());
+    List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(new AutoScalingConfig(conf), cloudManagerFromDiagnostics);
+    assertEquals(2, suggestions.size());
+    for (Suggester.SuggestionInfo suggestion : suggestions) {
+      assertTrue(ImmutableSet.of("127.0.0.219:63219_solr", "127.0.0.219:63229_solr").contains(
+          suggestion._get("operation/command/move-replica/targetNode", null)));
+    }
+  }
+  public void testSysPropSuggestions() {
 
     Map<String, Object> m = (Map<String, Object>) loadFromResource("testSysPropSuggestions.json");
 


[08/12] lucene-solr:jira/http2: SOLR-12879 - added missing attribution in CHANGES.txt

Posted by da...@apache.org.
SOLR-12879 - added missing attribution in 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/9df96d25
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/9df96d25
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/9df96d25

Branch: refs/heads/jira/http2
Commit: 9df96d2530ed7098549cbd8bda2b347f8c26042b
Parents: 7512cd9
Author: Tommaso Teofili <te...@adobe.com>
Authored: Tue Oct 23 09:07:38 2018 +0200
Committer: Tommaso Teofili <te...@adobe.com>
Committed: Tue Oct 23 09:07:38 2018 +0200

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9df96d25/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 81748b3..c14ac0a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -65,7 +65,7 @@ New Features
   locale was changed from ROOT to en_US since well-known patterns assume this locale.
   (David Smiley, Bar Rotstein)
 
-* SOLR-12879: MinHash query parser that builds queries providing a measure of Jaccard similarity
+* SOLR-12879: MinHash query parser that builds queries providing a measure of Jaccard similarity (Andy Hind via Tommaso Teofili)
 
 * SOLR-12593: The default configSet now includes an "ignored_*" dynamic field.  (David Smiley)
 


[03/12] lucene-solr:jira/http2: Three minor variables.adoc edits.

Posted by da...@apache.org.
Three minor variables.adoc edits.


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

Branch: refs/heads/jira/http2
Commit: c9776d88f90df2ae77a9e37e36d87e069bfde6ed
Parents: c31a95d
Author: Christine Poerschke <cp...@apache.org>
Authored: Fri Oct 19 20:01:17 2018 -0400
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Oct 22 17:33:11 2018 +0100

----------------------------------------------------------------------
 solr/solr-ref-guide/src/variables.adoc | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9776d88/solr/solr-ref-guide/src/variables.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/variables.adoc b/solr/solr-ref-guide/src/variables.adoc
index 894a5ac..2c1f905 100644
--- a/solr/solr-ref-guide/src/variables.adoc
+++ b/solr/solr-ref-guide/src/variables.adoc
@@ -189,7 +189,7 @@ When this expression is sent to the `/stream` handler it responds with:
 
 The `getCache` function retrieves an object from the cache by its workspace and key.
 
-In the example below the `getCache` function retrieves the array the was cached above and assigns it to variable *`a`*.
+In the example below the `getCache` function retrieves the array that was cached above and assigns it to variable *`a`*.
 
 [source,text]
 ----
@@ -279,8 +279,8 @@ When this expression is sent to the `/stream` handler it responds with:
 }
 ----
 
-The `removeCache` function can be used to remove a a key from a specific
-workspace. This `removeCache` function removes the key from the cache
+The `removeCache` function can be used to remove a key from a specific
+workspace. The `removeCache` function removes the key from the cache
 and returns the object that was removed.
 
 In the example below the array that was cached above is removed from the cache.


[10/12] lucene-solr:jira/http2: SOLR-12754: New hl.weightMatches for UnifiedHighlighter WEIGHT_MATCHES (defaults to true in master/8)

Posted by da...@apache.org.
SOLR-12754: New hl.weightMatches for UnifiedHighlighter WEIGHT_MATCHES
(defaults to true in master/8)


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

Branch: refs/heads/jira/http2
Commit: 3e89b7a771639aacaed6c21406624a2b27231dd7
Parents: 2e757f6
Author: David Smiley <ds...@apache.org>
Authored: Tue Oct 23 13:28:10 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Tue Oct 23 13:28:10 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  5 +++
 .../solr/highlight/UnifiedSolrHighlighter.java  | 33 ++++++++++++--------
 .../highlight/TestUnifiedSolrHighlighter.java   | 14 +++++++--
 solr/solr-ref-guide/src/highlighting.adoc       | 31 +++++++++++++++---
 .../solr/common/params/HighlightParams.java     |  1 +
 5 files changed, 64 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e89b7a7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c14ac0a..32c8ae2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -56,6 +56,9 @@ Upgrade Notes
   "date.formats" configuration.  To ensure date strings are properly parsed, use ParseDateFieldUpdateProcessorFactory
   (an URP) commonly registered with the name "parse-date" in "schemaless mode".  (David Smiley, Bar Rotstein)
 
+* SOLR-12754: The UnifiedHighlighter hl.weightMatches now defaults to true.  If there are unforseen highlight problems,
+  this may be the culprit.
+
 New Features
 ----------------------
 
@@ -154,6 +157,8 @@ New Features
 * SOLR-5004: Splitshard collections API now supports splitting into more than 2 sub-shards directly i.e. by providing a
   numSubShards parameter (Christine Poerschke, Anshum Gupta)
 
+* SOLR-12754: The UnifiedHighlighter has a new hl.weightMatches param defaulting to false (will be true in 8.0).  It's
+  the highest query accuracy mode, and furthermore phrase queries are highlighted as one.  (David Smiley)
 
 Other Changes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e89b7a7/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 a4beaba..fc43687 100644
--- a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
@@ -19,6 +19,7 @@ package org.apache.solr.highlight;
 import java.io.IOException;
 import java.text.BreakIterator;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -43,7 +44,6 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocIterator;
@@ -80,6 +80,7 @@ import org.apache.solr.util.plugin.PluginInfoInitialized;
  * &lt;bool name="hl.usePhraseHighlighter"&gt;true&lt;/bool&gt;
  * &lt;int name="hl.cacheFieldValCharsThreshold"&gt;524288&lt;/int&gt;
  * &lt;str name="hl.offsetSource"&gt;&lt;/str&gt;
+ * &lt;bool name="hl.weightMatches"&gt;true&lt;/bool&gt;
  * &lt;/lst&gt;
  * &lt;/requestHandler&gt;
  * </pre>
@@ -109,6 +110,7 @@ import org.apache.solr.util.plugin.PluginInfoInitialized;
  * <li>hl.usePhraseHighlighter (bool) enables phrase highlighting. default is true
  * <li>hl.cacheFieldValCharsThreshold (int) controls how many characters from a field are cached. default is 524288 (1MB in 2 byte chars)
  * <li>hl.offsetSource (string) specifies which offset source to use, prefers postings, but will use what's available if not specified
+ * <li>hl.weightMatches (bool) enables Lucene Weight Matches mode</li>
  * </ul>
  *
  * @lucene.experimental
@@ -241,12 +243,9 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
       this.setCacheFieldValCharsThreshold(
           params.getInt(HighlightParams.CACHE_FIELD_VAL_CHARS_THRESHOLD, DEFAULT_CACHE_CHARS_THRESHOLD));
 
-      // SolrRequestInfo is a thread-local singleton providing access to the ResponseBuilder to code that
-      //   otherwise can't get it in a nicer way.
-      SolrQueryRequest request = SolrRequestInfo.getRequestInfo().getReq();
       final RTimerTree timerTree;
-      if (request.getRequestTimer() != null) { //It may be null if not used in a search context.
-        timerTree = request.getRequestTimer();
+      if (req.getRequestTimer() != null) { //It may be null if not used in a search context.
+        timerTree = req.getRequestTimer();
       } else {
         timerTree = new RTimerTree(); // since null checks are annoying
       }
@@ -394,20 +393,28 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
     }
 
     @Override
-    protected boolean shouldHandleMultiTermQuery(String field) {
-      return params.getFieldBool(field, HighlightParams.HIGHLIGHT_MULTI_TERM, true);
-    }
+    protected Set<HighlightFlag> getFlags(String field) {
+      Set<HighlightFlag> flags = EnumSet.noneOf(HighlightFlag.class);
+      if (params.getFieldBool(field, HighlightParams.HIGHLIGHT_MULTI_TERM, true)) {
+        flags.add(HighlightFlag.MULTI_TERM_QUERY);
+      }
+      if (params.getFieldBool(field, HighlightParams.USE_PHRASE_HIGHLIGHTER, true)) {
+        flags.add(HighlightFlag.PHRASES);
+      }
+      flags.add(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED);
 
-    @Override
-    protected boolean shouldHighlightPhrasesStrictly(String field) {
-      return params.getFieldBool(field, HighlightParams.USE_PHRASE_HIGHLIGHTER, true);
+      if (params.getFieldBool(field, HighlightParams.WEIGHT_MATCHES, true)
+          && flags.contains(HighlightFlag.PHRASES) && flags.contains(HighlightFlag.MULTI_TERM_QUERY)) {
+        flags.add(HighlightFlag.WEIGHT_MATCHES);
+      }
+      return flags;
     }
 
     @Override
     protected Predicate<String> getFieldMatcher(String field) {
       // TODO define hl.queryFieldPattern as a more advanced alternative to hl.requireFieldMatch.
 
-      // note that the UH & PH at Lucene level default to effectively "true"
+      // note that the UH at Lucene level default to effectively "true"
       if (params.getFieldBool(field, HighlightParams.FIELD_MATCH, false)) {
         return field::equals; // requireFieldMatch
       } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e89b7a7/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 aeffed1..2a3e3a7 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -103,7 +103,7 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
     assertQ("strict phrase handling",
         req("q", "text:\"strict phrases\"", "sort", "id asc", "hl", "true"),
         "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/*)=1",
-        "//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>Strict</em> <em>phrases</em> should be enabled for phrases'");
+        "//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>Strict phrases</em> should be enabled for phrases'");
   }
 
   public void testStrictPhrasesCanBeDisabled() {
@@ -291,5 +291,15 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
     assertQ(req("q", "id:101", "hl", "true", "hl.q", "text:document", "hl.fl", "text3", "hl.requireFieldMatch", "true"),
         "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text3']/*)=0");
   }
-  
+
+  public void testWeightMatchesDisabled() {
+    clearIndex();
+    assertU(adoc("text", "alpha bravo charlie", "id", "101"));
+    assertU(commit());
+    assertQ("weight matches disabled, phrase highlights separately",
+        req("q", "text:\"alpha bravo\"", "hl", "true", "hl.weightMatches", "false"),
+        "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/*)=1",
+        "//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>alpha</em> <em>bravo</em> charlie'");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e89b7a7/solr/solr-ref-guide/src/highlighting.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/highlighting.adoc b/solr/solr-ref-guide/src/highlighting.adoc
index 2a832ee..6764343 100644
--- a/solr/solr-ref-guide/src/highlighting.adoc
+++ b/solr/solr-ref-guide/src/highlighting.adoc
@@ -147,19 +147,32 @@ There are many parameters supported by more than one highlighter, and sometimes
 
 There are four highlighters available that can be chosen at runtime with the `hl.method` parameter, in order of general recommendation:
 
+
 <<The Unified Highlighter,Unified Highlighter>>:: (`hl.method=unified`)
 +
-The Unified Highlighter is the newest highlighter (as of Solr 6.4), which stands out as the most flexible and performant of the options. We recommend that you try this highlighter even though it isn't the default (yet).
+The Unified Highlighter is the newest highlighter (as of Solr 6.4), which stands out as the most performant and accurate of the options.
+It can handle typical requirements and others possibly via plugins/extension.
+We recommend that you try this highlighter even though it isn't the default (yet).
++
+The UH highlights a query very _accurately_ and thus is true to what the underlying Lucene query actually matches.
+Other highlighters highlight terms more liberally (over-highlight).
+A strong benefit to this highlighter is that you can opt to configure Solr to put more information in the underlying index to speed up highlighting of large documents; multiple configurations are supported, even on a per-field basis.
+There is little or no such flexibility of offset sources for the other highlighters.
+More on this below.
 +
-This highlighter supports the most common highlighting parameters and can handle just about any query accurately, even SpanQueries (e.g., as seen from the `surround` parser). A strong benefit to this highlighter is that you can opt to configure Solr to put more information in the underlying index to speed up highlighting of large documents; multiple configurations are supported, even on a per-field basis. There is little or no such flexibility for the other highlighters. More on this below.
+There are some reasons not to choose this highlighter: The `surround` query parser doesn't yet work here -- SOLR-12895.
+Passage scoring does not consider boosts in the query.
+Some people want more/better passage breaking flexibility.
 
 <<The Original Highlighter,Original Highlighter>>:: (`hl.method=original`, the default)
 +
-The Original Highlighter, sometimes called the "Standard Highlighter" or "Default Highlighter", is Lucene's original highlighter – a venerable option with a high degree of customization options. Its ability to highlight just about any query accurately is a strength shared with the Unified Highlighter (they share some code for this in fact).
+The Original Highlighter, sometimes called the "Standard Highlighter" or "Default Highlighter", is Lucene's original highlighter – a venerable option with a high degree of customization options.
+It's query accuracy is good enough for most needs, although it's not quite as good/perfect as the Unified Highlighter.
 +
-The Original Highlighter will normally analyze stored text on the fly in order to highlight. It will use full term vectors if available, however in this mode it isn't as fast as the Unified Highlighter or FastVector Highlighter.
+The Original Highlighter will normally analyze stored text on the fly in order to highlight. It will use full term vectors if available.
 +
-This highlighter is a good choice for a wide variety of search use-cases. Where it falls short is performance; it's often twice as slow as the Unified Highlighter. And despite being the most customizable, it doesn't have a BreakIterator based fragmenter (all the others do), which could pose a challenge for some languages.
+Where this highlighter falls short is performance; it's often twice as slow as the Unified Highlighter. And despite being the most customizable, it doesn't have a BreakIterator based fragmenter (all the others do), which could pose a challenge for some languages.
+
 
 <<The FastVector Highlighter,FastVector Highlighter>>:: (`hl.method=fastVector`)
 +
@@ -171,6 +184,7 @@ This highlighter's query-representation is less advanced than the Original or Un
 +
 Note that both the FastVector and Original Highlighters can be used in conjunction in a search request to highlight some fields with one and some the other. In contrast, the other highlighters can only be chosen exclusively.
 
+
 The Unified Highlighter is exclusively configured via search parameters. In contrast, some settings for the Original and FastVector Highlighters are set in `solrconfig.xml`. There's a robust example of the latter in the "```techproducts```" configset.
 
 In addition to further information below, more information can be found in the {solr-javadocs}/solr-core/org/apache/solr/highlight/package-summary.html[Solr javadocs].
@@ -242,6 +256,13 @@ Indicates which character to break the text on. Use only if you have defined `hl
 +
 This is useful when the text has already been manipulated in advance to have a special delineation character at desired highlight passage boundaries. This character will still appear in the text as the last character of a passage.
 
+`hl.weightMatches`::
+Tells the UH to use Lucene's new "Weight Matches" API instead of doing SpanQuery conversion.
+This is the most accurate highlighting mode reflecting the query.
+Furthermore, phrases will be highlighted as a whole instead of word by word.
++
+The default is `true`.
+However if either `hl.usePhraseHighlighter` or `hl.multiTermQuery` are set to false, then this setting is effectively false no matter what you set it to.
 
 == The Original Highlighter
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e89b7a7/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 e09a2dc..c4825b7 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
@@ -91,4 +91,5 @@ public interface HighlightParams {
   public static final String PHRASE_LIMIT = HIGHLIGHT + ".phraseLimit"; // FVH
   public static final String OFFSET_SOURCE = HIGHLIGHT + ".offsetSource"; // UH
   public static final String CACHE_FIELD_VAL_CHARS_THRESHOLD = HIGHLIGHT + ".cacheFieldValCharsThreshold"; // UH
+  public static final String WEIGHT_MATCHES = HIGHLIGHT + ".weightMatches"; // UH
 }


[04/12] lucene-solr:jira/http2: SOLR-12829: Add plist (parallel list) Streaming Expression

Posted by da...@apache.org.
SOLR-12829: Add plist (parallel list) Streaming Expression


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

Branch: refs/heads/jira/http2
Commit: fcaea07f3c8cba34906ca02f40fb1d2c40badc08
Parents: c9776d8
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Oct 22 15:20:13 2018 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Oct 22 15:20:13 2018 -0400

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/io/Lang.java   |   1 +
 .../solr/client/solrj/io/stream/ListStream.java |   4 +
 .../solrj/io/stream/ParallelListStream.java     | 205 +++++++++++++++++++
 .../solr/client/solrj/io/stream/TupStream.java  |  89 ++++----
 .../apache/solr/client/solrj/io/TestLang.java   |   2 +-
 .../solrj/io/stream/MathExpressionTest.java     |   8 +-
 6 files changed, 260 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fcaea07f/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
index 7cc842f..2be48e3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
@@ -90,6 +90,7 @@ public class Lang {
         .withFunctionName("timeseries", TimeSeriesStream.class)
         .withFunctionName("tuple", TupStream.class)
         .withFunctionName("sql", SqlStream.class)
+        .withFunctionName("plist", ParallelListStream.class)
 
             // metrics
         .withFunctionName("min", MinMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fcaea07f/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ListStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ListStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ListStream.java
index 826e948..33f8fd5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ListStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ListStream.java
@@ -108,6 +108,10 @@ public class ListStream extends TupleStream implements Expressible {
       if (currentStream == null) {
         if (streamIndex < streams.length) {
           currentStream = streams[streamIndex];
+          // Set the stream to null in the array of streams once its been set to the current stream.
+          // This will remove the reference to the stream
+          // and should allow it to be garbage collected once it's no longer the current stream.
+          streams[streamIndex] = null;
           currentStream.open();
         } else {
           HashMap map = new HashMap();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fcaea07f/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelListStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelListStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelListStream.java
new file mode 100644
index 0000000..ef02ffa
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelListStream.java
@@ -0,0 +1,205 @@
+/*
+ * 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.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrjNamedThreadFactory;
+
+public class ParallelListStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+  private TupleStream[] streams;
+  private TupleStream currentStream;
+  private int streamIndex;
+
+  public ParallelListStream(TupleStream... streams) throws IOException {
+    init(streams);
+  }
+
+  public ParallelListStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    TupleStream[] streams = new TupleStream[streamExpressions.size()];
+    for(int idx = 0; idx < streamExpressions.size(); ++idx){
+      streams[idx] = factory.constructStream(streamExpressions.get(idx));
+    }
+
+    init(streams);
+  }
+
+  private void init(TupleStream ... tupleStreams) {
+    this.streams = tupleStreams;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException{
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(includeStreams) {
+      for(TupleStream stream : streams) {
+        expression.addParameter(((Expressible)stream).toExpression(factory));
+      }
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    StreamExplanation explanation = new StreamExplanation(getStreamNodeId().toString());
+    explanation.setFunctionName(factory.getFunctionName(this.getClass()));
+    explanation.setImplementingClass(this.getClass().getName());
+    explanation.setExpressionType(ExpressionType.STREAM_DECORATOR);
+    explanation.setExpression(toExpression(factory, false).toString());
+    for(TupleStream stream : streams) {
+      explanation.addChild(stream.toExplanation(factory));
+    }
+
+    return explanation;
+  }
+
+  public void setStreamContext(StreamContext context) {
+    for(TupleStream stream : streams) {
+      stream.setStreamContext(context);
+    }
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    for(TupleStream stream : streams) {
+      l.add(stream);
+    }
+    return l;
+  }
+
+  public Tuple read() throws IOException {
+    while(true) {
+      if (currentStream == null) {
+        if (streamIndex < streams.length) {
+          currentStream = streams[streamIndex];
+        } else {
+          HashMap map = new HashMap();
+          map.put("EOF", true);
+          return new Tuple(map);
+        }
+      }
+
+      Tuple tuple = currentStream.read();
+      if (tuple.EOF) {
+        currentStream.close();
+        currentStream = null;
+        ++streamIndex;
+      } else {
+        return tuple;
+      }
+    }
+  }
+
+  public void close() throws IOException {
+  }
+
+  public void open() throws IOException {
+    openStreams();
+  }
+
+  private void openStreams() throws IOException {
+    ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("ParallelListStream"));
+    try {
+      List<Future<StreamIndex>> futures = new ArrayList();
+      int i=0;
+      for (TupleStream tupleStream : streams) {
+        StreamOpener so = new StreamOpener(new StreamIndex(tupleStream, i++));
+        Future<StreamIndex> future = service.submit(so);
+        futures.add(future);
+      }
+
+      try {
+        for (Future<StreamIndex> f : futures) {
+          StreamIndex streamIndex = f.get();
+          this.streams[streamIndex.getIndex()] = streamIndex.getTupleStream();
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    } finally {
+      service.shutdown();
+    }
+  }
+
+  protected class StreamOpener implements Callable<StreamIndex> {
+
+    private StreamIndex streamIndex;
+
+    public StreamOpener(StreamIndex streamIndex) {
+      this.streamIndex = streamIndex;
+    }
+
+    public StreamIndex call() throws Exception {
+      streamIndex.getTupleStream().open();
+      return streamIndex;
+    }
+  }
+
+  protected class StreamIndex {
+    private TupleStream tupleStream;
+    private int index;
+
+    public StreamIndex(TupleStream tupleStream, int index) {
+      this.tupleStream = tupleStream;
+      this.index = index;
+    }
+
+    public int getIndex() {
+      return this.index;
+    }
+
+    public TupleStream getTupleStream() {
+      return this.tupleStream;
+    }
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return null;
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fcaea07f/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupStream.java
index c87dc24..fde8298 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupStream.java
@@ -51,6 +51,7 @@ public class TupStream extends TupleStream implements Expressible {
   private Map<String,TupleStream> streamParams = new HashMap<>();
   private List<String> fieldNames = new ArrayList();
   private Map<String, String> fieldLabels = new HashMap();
+  private Tuple tup = null;
   
   private boolean finished;
 
@@ -151,50 +152,6 @@ public class TupStream extends TupleStream implements Expressible {
       return new Tuple(m);
     } else {
       finished = true;
-      Map<String, Object> values = new HashMap<>();
-      
-      // add all string based params
-      // these could come from the context, or they will just be treated as straight strings
-      for(Entry<String,String> param : stringParams.entrySet()){
-        if(streamContext.getLets().containsKey(param.getValue())){
-          values.put(param.getKey(), streamContext.getLets().get(param.getValue()));
-        }
-        else{
-          values.put(param.getKey(), param.getValue());
-        }
-      }
-      
-      // add all evaluators
-      for(Entry<String,StreamEvaluator> param : evaluatorParams.entrySet()){
-        values.put(param.getKey(), param.getValue().evaluateOverContext());
-      }
-      
-      // Add all streams
-      for(Entry<String,TupleStream> param : streamParams.entrySet()){
-        
-        try{
-          List<Tuple> streamTuples = new ArrayList();
-          // open the stream, closed in finally block
-          param.getValue().open();
-          
-          // read all values from stream (memory expensive)
-          Tuple streamTuple = param.getValue().read();
-          while(!streamTuple.EOF){
-            streamTuples.add(streamTuple);
-            streamTuple = param.getValue().read();
-          }
-          
-          values.put(param.getKey(), streamTuples);
-        }
-        finally{
-          // safely close the stream
-          param.getValue().close();
-        }        
-      }
-
-      Tuple tup = new Tuple(values);
-      tup.fieldNames = fieldNames;
-      tup.fieldLabels = fieldLabels;
       return tup;
     }
   }
@@ -204,6 +161,50 @@ public class TupStream extends TupleStream implements Expressible {
   }
 
   public void open() throws IOException {
+    Map<String, Object> values = new HashMap<>();
+
+    // add all string based params
+    // these could come from the context, or they will just be treated as straight strings
+    for(Entry<String,String> param : stringParams.entrySet()){
+      if(streamContext.getLets().containsKey(param.getValue())){
+        values.put(param.getKey(), streamContext.getLets().get(param.getValue()));
+      }
+      else{
+        values.put(param.getKey(), param.getValue());
+      }
+    }
+
+    // add all evaluators
+    for(Entry<String,StreamEvaluator> param : evaluatorParams.entrySet()){
+      values.put(param.getKey(), param.getValue().evaluateOverContext());
+    }
+
+    // Add all streams
+    for(Entry<String,TupleStream> param : streamParams.entrySet()){
+
+      try{
+        List<Tuple> streamTuples = new ArrayList();
+        // open the stream, closed in finally block
+        param.getValue().open();
+
+        // read all values from stream (memory expensive)
+        Tuple streamTuple = param.getValue().read();
+        while(!streamTuple.EOF){
+          streamTuples.add(streamTuple);
+          streamTuple = param.getValue().read();
+        }
+
+        values.put(param.getKey(), streamTuples);
+      }
+      finally{
+        // safely close the stream
+        param.getValue().close();
+      }
+    }
+
+    this.tup = new Tuple(values);
+    tup.fieldNames = fieldNames;
+    tup.fieldLabels = fieldLabels;
     // nothing to do here
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fcaea07f/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
index 07b0938..e06b973 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
@@ -73,7 +73,7 @@ public class TestLang extends LuceneTestCase {
       "outliers", "stream", "getCache", "putCache", "listCache", "removeCache", "zscores", "latlonVectors",
       "convexHull", "getVertices", "getBaryCenter", "getArea", "getBoundarySize","oscillate",
       "getAmplitude", "getPhase", "getAngularFrequency", "enclosingDisk", "getCenter", "getRadius",
-      "getSupportPoints", "pairSort", "log10"};
+      "getSupportPoints", "pairSort", "log10", "plist"};
 
   @Test
   public void testLang() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fcaea07f/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
index 78fc2ce..2bff1ab 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
@@ -235,7 +235,7 @@ public class MathExpressionTest extends SolrCloudTestCase {
   @Test
   public void testMemsetSize() throws Exception {
     String expr = "let(echo=\"b, c\"," +
-        "              a=memset(list(tuple(field1=val(1), field2=val(10)), tuple(field1=val(2), field2=val(20))), " +
+        "              a=memset(plist(tuple(field1=val(1), field2=val(10)), tuple(field1=val(2), field2=val(20))), " +
         "                       cols=\"field1, field2\", " +
         "                       vars=\"f1, f2\"," +
         "                       size=1)," +
@@ -1974,7 +1974,7 @@ public class MathExpressionTest extends SolrCloudTestCase {
     //Test exclude. This should drop off the term jim
 
     cexpr = "let(echo=true," +
-        "        a=select(list(tuple(id=\"1\", text=\"hello world\"), " +
+        "        a=select(plist(tuple(id=\"1\", text=\"hello world\"), " +
         "                      tuple(id=\"2\", text=\"hello steve\"), " +
         "                      tuple(id=\"3\", text=\"hello jim jim\"), " +
         "                      tuple(id=\"4\", text=\"hello jack\")), id, analyze(text, test_t) as terms)," +
@@ -2046,7 +2046,7 @@ public class MathExpressionTest extends SolrCloudTestCase {
     //Test minDocFreq attribute at .5. This should eliminate all but the term hello
 
     cexpr = "let(echo=true," +
-        "a=select(list(tuple(id=\"1\", text=\"hello world\"), " +
+        "a=select(plist(tuple(id=\"1\", text=\"hello world\"), " +
         "tuple(id=\"2\", text=\"hello steve\"), " +
         "tuple(id=\"3\", text=\"hello jim jim\"), " +
         "tuple(id=\"4\", text=\"hello jack\")), id, analyze(text, test_t) as terms)," +
@@ -2100,7 +2100,7 @@ public class MathExpressionTest extends SolrCloudTestCase {
     //Test maxDocFreq attribute at 0. This should eliminate all terms
 
     cexpr = "let(echo=true," +
-        "a=select(list(tuple(id=\"1\", text=\"hello world\"), " +
+        "a=select(plist(tuple(id=\"1\", text=\"hello world\"), " +
         "tuple(id=\"2\", text=\"hello steve\"), " +
         "tuple(id=\"3\", text=\"hello jim jim\"), " +
         "tuple(id=\"4\", text=\"hello jack\")), id, analyze(text, test_t) as terms)," +


[12/12] lucene-solr:jira/http2: Merge with master

Posted by da...@apache.org.
Merge with master


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

Branch: refs/heads/jira/http2
Commit: 6226f0f383d48642d3a0ba38e5a9336d3345feeb
Parents: a10c09e e083b15
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed Oct 24 08:58:38 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed Oct 24 08:58:38 2018 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  13 +-
 .../org/apache/solr/cloud/ElectionContext.java  |   3 +
 .../cloud/api/collections/SplitShardCmd.java    |  23 +-
 .../solr/handler/admin/CollectionsHandler.java  |   9 +-
 .../solr/highlight/UnifiedSolrHighlighter.java  |  33 ++-
 .../solr/search/MinHashQParserPlugin.java       |   3 +
 .../org/apache/solr/search/QParserPlugin.java   |   1 +
 .../solr/cloud/LIROnShardRestartTest.java       | 262 -------------------
 .../org/apache/solr/cloud/SplitShardTest.java   | 102 ++++++++
 .../solr/cloud/autoscaling/TestPolicyCloud.java |   2 +
 .../sim/SimClusterStateProvider.java            |   1 +
 .../highlight/TestUnifiedSolrHighlighter.java   |  14 +-
 solr/solr-ref-guide/src/collections-api.adoc    |   5 +
 solr/solr-ref-guide/src/highlighting.adoc       |  31 ++-
 solr/solr-ref-guide/src/variables.adoc          |   6 +-
 .../solrj/cloud/autoscaling/Variable.java       |   6 +
 .../org/apache/solr/client/solrj/io/Lang.java   |   1 +
 .../solr/client/solrj/io/stream/ListStream.java |   4 +
 .../solrj/io/stream/ParallelListStream.java     | 205 +++++++++++++++
 .../solr/client/solrj/io/stream/TupStream.java  |  89 +++----
 .../solrj/request/CollectionAdminRequest.java   |  12 +
 .../java/org/apache/solr/common/MapWriter.java  |  48 +---
 .../org/apache/solr/common/NavigableObject.java |  89 +++++++
 .../solr/common/params/CommonAdminParams.java   |   2 +
 .../solr/common/params/HighlightParams.java     |   1 +
 .../java/org/apache/solr/common/util/Utils.java |  12 +-
 .../solr/autoscaling/testHostAttribute.json     | 119 +++++++++
 .../solrj/cloud/autoscaling/TestPolicy2.java    |  22 +-
 .../apache/solr/client/solrj/io/TestLang.java   |   2 +-
 .../solrj/io/stream/MathExpressionTest.java     |   8 +-
 30 files changed, 731 insertions(+), 397 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6226f0f3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --cc solr/CHANGES.txt
index 88ca1fd,32c8ae2..1a7db9d
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@@ -63,26 -56,9 +63,29 @@@ Upgrade Note
    "date.formats" configuration.  To ensure date strings are properly parsed, use ParseDateFieldUpdateProcessorFactory
    (an URP) commonly registered with the name "parse-date" in "schemaless mode".  (David Smiley, Bar Rotstein)
  
 +* SOLR-12639: Solr is now using HTTP/2 (Http2SolrClient) for most of internal requests
 +  (UpdateShardHandler, HttpShardHandler).
 +
 +* SOLR-12643: Since Http2SolrClient does not support exposing connections related metrics. These metrics are no longer
 +  available 'QUERY.httpShardHandler.{availableConnections, leasedConnections, maxConnections, pendingConnections}',
 +  'UPDATE.updateShardHandler.{availableConnections, leasedConnections, maxConnections, pendingConnections}'
 +  (Cao Manh Dat)
 +
 +* SOLR-12605: UpdateShardHandler's updateOnlyClient is now a Http2SolrClient (previous HttpSolrClient). This new
 +  client does not support 'maxConnections' parameter (Cao Manh Dat). StreamingsSolrClient which is removed and
 +  ConcurrentUpdateSolrClient are no longer used to send updates, updates will be sent asynchronously (without-batching)
 +  by Http2SolrClient.
 +
 +* SOLR-12640: HttpShardHandlerFactory's defaultClient is now a Http2SolrClient (previous HttpSolrClient). This new
 +  client does not support 'maxConnections' parameter (Cao Manh Dat). LBHttpSolrClient.Req and LBHttpSolrClient.Rsp
 +  are marked as deprecated, uses LBSolrClient.Req and LBSolrClient.Rsp instead.
 +
 +* SOLR-12644: Custom AuthenticationPlugin must provide its own setup for Http2SolrClient through
 +  implementing HttpClientBuilderPlugin.setup, if not internal requests can't be authenticated. (Cao Manh Dat)
 +
+ * SOLR-12754: The UnifiedHighlighter hl.weightMatches now defaults to true.  If there are unforseen highlight problems,
+   this may be the culprit.
+ 
  New Features
  ----------------------