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 2017/12/27 15:03:54 UTC

[01/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-11702 81efe5209 -> a7b32b080


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
index c285452..3041a13 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ClusterStateProvider.java
@@ -16,15 +16,15 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.Map;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.cloud.ClusterState;
 
-public interface ClusterStateProvider extends Closeable {
+public interface ClusterStateProvider extends SolrCloseable {
 
   /**
    * Obtain the state of the collection (cluster status).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
index d617d16..b623157 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClusterStateProvider.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,7 +91,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
   public CollectionRef getState(String collection) {
     for (String nodeName: liveNodes) {
       try (HttpSolrClient client = new HttpSolrClient.Builder().
-          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
           withHttpClient(httpClient).build()) {
         ClusterState cs = fetchClusterState(client, collection, null);
         return cs.getCollectionRef(collection);
@@ -102,7 +103,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
           return null;
         }
         log.warn("Attempt to fetch cluster state from " +
-            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "
@@ -160,7 +161,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
     if (TimeUnit.SECONDS.convert((System.nanoTime() - liveNodesTimestamp), TimeUnit.NANOSECONDS) > getCacheTimeout()) {
       for (String nodeName: liveNodes) {
         try (HttpSolrClient client = new HttpSolrClient.Builder().
-            withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+            withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
             withHttpClient(httpClient).build()) {
           Set<String> liveNodes = fetchLiveNodes(client);
           this.liveNodes = (liveNodes);
@@ -168,7 +169,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
           return liveNodes;
         } catch (Exception e) {
           log.warn("Attempt to fetch live_nodes from " +
-              ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+              Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
         }
       }
       throw new RuntimeException("Tried fetching live_nodes using all the node names we knew of, i.e. " + liveNodes +". However, "
@@ -209,7 +210,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
         TimeUnit.SECONDS.convert((System.nanoTime() - aliasesTimestamp), TimeUnit.NANOSECONDS) > getCacheTimeout()) {
       for (String nodeName: liveNodes) {
         try (HttpSolrClient client = new HttpSolrClient.Builder().
-            withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+            withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
             withHttpClient(httpClient).build()) {
 
           Map<String, List<String>> aliases = new CollectionAdminRequest.ListAliases().process(client).getAliasesAsLists();
@@ -226,7 +227,7 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
             return aliases;
           }
           log.warn("Attempt to fetch cluster state from " +
-              ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+              Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
         }
       }
 
@@ -244,13 +245,13 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
   public ClusterState getClusterState() throws IOException {
     for (String nodeName: liveNodes) {
       try (HttpSolrClient client = new HttpSolrClient.Builder().
-          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
           withHttpClient(httpClient).build()) {
         ClusterState cs = fetchClusterState(client, null, null);
         return cs;
       } catch (SolrServerException | RemoteSolrException | IOException e) {
         log.warn("Attempt to fetch cluster state from " +
-            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "
@@ -264,14 +265,14 @@ public class HttpClusterStateProvider implements ClusterStateProvider {
   public Map<String, Object> getClusterProperties() {
     for (String nodeName: liveNodes) {
       try (HttpSolrClient client = new HttpSolrClient.Builder().
-          withBaseSolrUrl(ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme)).
+          withBaseSolrUrl(Utils.getBaseUrlForNodeName(nodeName, urlScheme)).
           withHttpClient(httpClient).build()) {
         Map<String, Object> clusterProperties = new HashMap<>();
         fetchClusterState(client, null, clusterProperties);
         return clusterProperties;
       } catch (SolrServerException | RemoteSolrException | IOException e) {
         log.warn("Attempt to fetch cluster state from " +
-            ZkStateReader.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
+            Utils.getBaseUrlForNodeName(nodeName, urlScheme) + " failed.", e);
       }
     }
     throw new RuntimeException("Tried fetching cluster state using the node names we knew of, i.e. " + liveNodes +". However, "

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
index c67cdc8..4fe345b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
@@ -45,6 +45,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -89,6 +90,11 @@ public class SolrClientCloudManager implements SolrCloudManager {
   }
 
   @Override
+  public TimeSource getTimeSource() {
+    return TimeSource.NANO_TIME;
+  }
+
+  @Override
   public ClusterStateProvider getClusterStateProvider() {
     return solrClient.getClusterStateProvider();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
index f2782a3..66f54dd 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
@@ -67,6 +67,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
   public static SolrClientNodeStateProvider INST;
 
 
+
   private final CloudSolrClient solrClient;
   private final ZkStateReader zkStateReader;
   private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> nodeVsCollectionVsShardVsReplicaInfo = new HashMap<>();
@@ -155,6 +156,11 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
     }
   }
 
+  @Override
+  public void close() throws IOException {
+
+  }
+
   //uses metrics API to get node information
   static class AutoScalingSnitch extends ImplicitSnitch {
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
index 7664ce2..b0cc710 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkDistribStateManager.java
@@ -61,9 +61,9 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
-  public List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+  public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
     try {
-      return zkClient.getChildren(path, null, true);
+      return zkClient.getChildren(path, watcher, true);
     } catch (KeeperException.NoNodeException e) {
       throw new NoSuchElementException(path);
     } catch (InterruptedException e) {
@@ -72,6 +72,11 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
+  public List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+    return listData(path, null);
+  }
+
+  @Override
   public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
     Stat stat = new Stat();
     try {
@@ -96,9 +101,22 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
-  public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+  public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    try {
+      zkClient.makePath(path, data, createMode, null, failOnExists, true);
+    } catch (KeeperException.NodeExistsException e) {
+      throw new AlreadyExistsException(path);
+    } catch (InterruptedException e) {
+      throw e;
+    }
+  }
+
+  @Override
+  public String createData(String path, byte[] data, CreateMode mode) throws NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
     try {
       return zkClient.create(path, data, mode, true);
+    } catch (KeeperException.NoNodeException e) {
+      throw new NoSuchElementException(path);
     } catch (KeeperException.NodeExistsException e) {
       throw new AlreadyExistsException(path);
     } catch (InterruptedException e) {
@@ -107,11 +125,13 @@ public class ZkDistribStateManager implements DistribStateManager {
   }
 
   @Override
-  public void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+  public void removeData(String path, int version) throws NoSuchElementException, BadVersionException, IOException, KeeperException, InterruptedException {
     try {
       zkClient.delete(path, version, true);
     } catch (KeeperException.NoNodeException e) {
       throw new NoSuchElementException(path);
+    } catch (KeeperException.BadVersionException e) {
+      throw new BadVersionException(version, path);
     } catch (InterruptedException e) {
       throw e;
     }
@@ -163,4 +183,8 @@ public class ZkDistribStateManager implements DistribStateManager {
     return new AutoScalingConfig(map);
   }
 
+  @Override
+  public void close() throws IOException {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
index bfe33c5..4333a00 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
@@ -92,6 +92,13 @@ public class ClusterState implements JSONWriter.Writable {
   }
 
   /**
+   * Returns the zNode version that was used to construct this instance.
+   */
+  public int getZNodeVersion() {
+    return znodeVersion;
+  }
+
+  /**
    * Returns true if the specified collection name exists, false otherwise.
    *
    * Implementation note: This method resolves the collection reference by calling
@@ -196,7 +203,10 @@ public class ClusterState implements JSONWriter.Writable {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
+    sb.append("znodeVersion: " + znodeVersion);
+    sb.append("\n");
     sb.append("live nodes:" + liveNodes);
+    sb.append("\n");
     sb.append("collections:" + collectionStates);
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index 16a4c60..02beb97 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -124,6 +124,17 @@ public class Replica extends ZkNodeProps {
     type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    Replica replica = (Replica) o;
+
+    return name.equals(replica.name);
+  }
+
   public String getName() {
     return name;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 4c2be1a..8ab7ecb 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -17,9 +17,7 @@
 package org.apache.solr.common.cloud;
 
 import java.io.Closeable;
-import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
-import java.net.URLDecoder;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -79,6 +77,7 @@ public class ZkStateReader implements Closeable {
   public static final String CORE_NODE_NAME_PROP = "core_node_name";
   public static final String ROLES_PROP = "roles";
   public static final String STATE_PROP = "state";
+  /**  SolrCore name. */
   public static final String CORE_NAME_PROP = "core";
   public static final String COLLECTION_PROP = "collection";
   public static final String ELECTION_NODE_PROP = "election_node";
@@ -949,21 +948,7 @@ public class ZkStateReader implements Closeable {
    * @lucene.experimental
    */
   public String getBaseUrlForNodeName(final String nodeName) {
-    return getBaseUrlForNodeName(nodeName, getClusterProperty(URL_SCHEME, "http"));
-  }
-
-  public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
-    final int _offset = nodeName.indexOf("_");
-    if (_offset < 0) {
-      throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName);
-    }
-    final String hostAndPort = nodeName.substring(0,_offset);
-    try {
-      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
-      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
-    } catch (UnsupportedEncodingException e) {
-      throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
-    }
+    return Utils.getBaseUrlForNodeName(nodeName, getClusterProperty(URL_SCHEME, "http"));
   }
 
   /** Watches a single collection's format2 state.json. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java b/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
new file mode 100644
index 0000000..ca535cb
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/TimeSource.java
@@ -0,0 +1,161 @@
+/*
+ * 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.util;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Source of time. NOTE: depending on implementation returned values may not be related in any way to the
+ * current Epoch or calendar time, and they may even be negative - but the API guarantees that they are
+ * always monotonically increasing.
+ */
+public abstract class TimeSource {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /** Implementation that uses {@link System#currentTimeMillis()}. */
+  public static final class CurrentTimeSource extends TimeSource {
+
+    @Override
+    @SuppressForbidden(reason = "Needed to provide timestamps based on currentTimeMillis.")
+    public long getTime() {
+      return TimeUnit.NANOSECONDS.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      Thread.sleep(ms);
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      return toUnit.convert(value, fromUnit);
+    }
+  }
+
+  /** Implementation that uses {@link System#nanoTime()}. */
+  public static final class NanoTimeSource extends TimeSource {
+
+    @Override
+    public long getTime() {
+      return System.nanoTime();
+    }
+
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      Thread.sleep(ms);
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      return toUnit.convert(value, fromUnit);
+    }
+  }
+
+  /** Implementation that uses {@link #NANO_TIME} accelerated by a double multiplier. */
+  public static final class SimTimeSource extends TimeSource {
+
+    final double multiplier;
+    long start;
+
+    /**
+     * Create a simulated time source that runs faster than real time by a multipler.
+     * @param multiplier must be greater than 0.0
+     */
+    public SimTimeSource(double multiplier) {
+      this.multiplier = multiplier;
+      start = NANO_TIME.getTime();
+    }
+
+    public void advance(long delta) {
+      start = getTime() + delta;
+    }
+
+    @Override
+    public long getTime() {
+      return start + Math.round((double)(NANO_TIME.getTime() - start) * multiplier);
+    }
+
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      ms = Math.round((double)ms / multiplier);
+      Thread.sleep(ms);
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      long nano = Math.round((double)TimeUnit.NANOSECONDS.convert(value, fromUnit) / multiplier);
+      return toUnit.convert(nano, TimeUnit.NANOSECONDS);
+    }
+  }
+
+  /** This instance uses {@link CurrentTimeSource} for generating timestamps. */
+  public static final TimeSource CURRENT_TIME = new CurrentTimeSource();
+
+  /** This instance uses {@link NanoTimeSource} for generating timestamps. */
+  public static final TimeSource NANO_TIME = new NanoTimeSource();
+
+  private static Map<String, SimTimeSource> simTimeSources = new ConcurrentHashMap<>();
+
+  /**
+   * Obtain an instance of time source.
+   * @param type supported types: <code>currentTime</code>, <code>nanoTime</code> and accelerated
+   *             time with a double factor in the form of <code>simTime:FACTOR</code>, eg.
+   *             <code>simTime:2.5</code>
+   * @return one of the supported types
+   */
+  public static TimeSource get(String type) {
+    if (type == null) {
+      return NANO_TIME;
+    } else if (type.equals("currentTime")) {
+      return CURRENT_TIME;
+    } else if (type.equals("nanoTime")) {
+      return NANO_TIME;
+    } else if (type.startsWith("simTime")) {
+      return simTimeSources.computeIfAbsent(type, t -> {
+        String[] parts = t.split(":");
+        double mul = 1.0;
+        if (parts.length != 2) {
+          log.warn("Invalid simTime specification, assuming multiplier==1.0: '" + type + "'");
+        } else {
+          try {
+            mul = Double.parseDouble(parts[1]);
+          } catch (Exception e) {
+            log.warn("Invalid simTime specification, assuming multiplier==1.0: '" + type + "'");
+          }
+        }
+        return new SimTimeSource(mul);
+      });
+    } else {
+      throw new UnsupportedOperationException("Unsupported time source type '" + type + "'.");
+    }
+  }
+
+  /**
+   * Return a time value, in nanosecond unit.
+   */
+  public abstract long getTime();
+
+  public abstract void sleep(long ms) throws InterruptedException;
+
+  public abstract long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 93af8c3..4ab24d2 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
@@ -21,7 +21,9 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.Reader;
 import java.io.StringReader;
+import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
+import java.net.URLDecoder;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
@@ -445,12 +447,25 @@ public class Utils {
     }
   }
 
-  public static long time(TimeUnit unit) {
-    return unit.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
+  public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
+    final int _offset = nodeName.indexOf("_");
+    if (_offset < 0) {
+      throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName);
+    }
+    final String hostAndPort = nodeName.substring(0,_offset);
+    try {
+      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
+      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
+    }
   }
 
-  public static long timeElapsed(long start, TimeUnit unit) {
-    return unit.convert(System.nanoTime() - NANOSECONDS.convert(start, unit), NANOSECONDS);
+  public static long time(TimeSource timeSource, TimeUnit unit) {
+    return unit.convert(timeSource.getTime(), TimeUnit.NANOSECONDS);
   }
 
+  public static long timeElapsed(TimeSource timeSource, long start, TimeUnit unit) {
+    return unit.convert(timeSource.getTime() - NANOSECONDS.convert(start, unit), NANOSECONDS);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
index cda9961..7dc14de 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
@@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
@@ -146,7 +147,7 @@ abstract public class SolrExampleTestsBase extends SolrJettyTestBase {
     Assert.assertEquals(1, rsp.getResults().getNumFound());
     
     // check if the doc has been deleted every 250 ms for 30 seconds
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     do {
       Thread.sleep(250); // wait 250 ms
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
index b3a6518..84aff76 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
@@ -33,6 +33,7 @@ import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.SolrResponseBase;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -228,7 +229,7 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 {
   
   // wait maximum ms for serverName to come back up
   private void waitForServer(int maxSeconds, LBHttpSolrClient client, int nServers, String serverName) throws Exception {
-    final TimeOut timeout = new TimeOut(maxSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(maxSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (! timeout.hasTimedOut()) {
       QueryResponse resp;
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
index 396234b..e89f10e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
@@ -47,6 +47,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.ValidatingJsonMap;
 import org.junit.Test;
@@ -1164,6 +1165,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
       public ObjectCache getObjectCache() {
         return objectCache;
       }
+
+      @Override
+      public TimeSource getTimeSource() {
+        return TimeSource.NANO_TIME;
+      }
+
+      @Override
+      public void close() throws IOException {
+
+      }
+
       @Override
       public ClusterStateProvider getClusterStateProvider() {
         return new DelegatingClusterStateProvider(null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index fb79cf5..a597f3d 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -71,6 +71,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.Diagnostics;
@@ -1797,7 +1798,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       List<Integer> numShardsNumReplicaList,
       List<String> nodesAllowedToRunShards) throws Exception {
     // check for an expectedSlices new collection - we poll the state
-    final TimeOut timeout = new TimeOut(120, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(120, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     boolean success = false;
     String checkResult = "Didnt get to perform a single check";
     while (! timeout.hasTimedOut()) {
@@ -1856,7 +1857,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
  public static void waitForNon403or404or503(HttpSolrClient collectionClient)
       throws Exception {
     SolrException exp = null;
-    final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     while (! timeout.hasTimedOut()) {
       boolean missing = false;
@@ -2207,7 +2208,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   static RequestStatusState getRequestStateAfterCompletion(String requestId, int waitForSeconds, SolrClient client)
       throws IOException, SolrServerException {
     RequestStatusState state = null;
-    final TimeOut timeout = new TimeOut(waitForSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(waitForSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     while (!timeout.hasTimedOut())  {
       state = getRequestState(requestId, client);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
index b8873fa..abb6255 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.servlet.SolrDispatchFilter;
@@ -698,7 +699,7 @@ public class ChaosMonkey {
    * @param zkStateReader current state reader
    */
   public static void wait(long runLength, String collectionName, ZkStateReader zkStateReader) throws InterruptedException {
-    TimeOut t = new TimeOut(runLength, TimeUnit.MILLISECONDS);
+    TimeOut t = new TimeOut(runLength, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (!t.hasTimedOut()) {
       Thread.sleep(Math.min(1000, t.timeLeft(TimeUnit.MILLISECONDS)));
       logCollectionStateSummary(collectionName, zkStateReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
index f437863..75418c6 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
 
 import com.google.common.util.concurrent.AtomicLongMap;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -531,7 +532,7 @@ public class ZkTestServer {
   }
   
   public static boolean waitForServerDown(String hp, long timeoutMs) {
-    final TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    final TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (true) {
       try {
         HostPort hpobj = parseHostPortList(hp).get(0);


[36/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11785: Add multiVariateNormalDistribution Stream Evaluator

Posted by da...@apache.org.
SOLR-11785: Add multiVariateNormalDistribution Stream Evaluator


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

Branch: refs/heads/jira/solr-11702
Commit: 960a5fd793aa26546379a17ee29c026fcb198a37
Parents: d9695cc
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Dec 20 13:51:34 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Dec 20 13:51:47 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  4 +-
 ...MultiVariateNormalDistributionEvaluator.java | 54 ++++++++++++++++++++
 .../client/solrj/io/eval/SampleEvaluator.java   | 27 +++++++++-
 .../solrj/io/stream/StreamExpressionTest.java   | 47 +++++++++++++++++
 4 files changed, 128 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/960a5fd7/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 8a83160..949a040 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -283,11 +283,11 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("spline", SplineEvaluator.class)
         .withFunctionName("ttest", TTestEvaluator.class)
         .withFunctionName("pairedTtest", PairedTTestEvaluator.class)
-
+        .withFunctionName("multiVariateNormalDistribution", MultiVariateNormalDistributionEvaluator.class)
 
         // Boolean Stream Evaluators
 
-        .withFunctionName("and", AndEvaluator.class)
+            .withFunctionName("and", AndEvaluator.class)
         .withFunctionName("eor", ExclusiveOrEvaluator.class)
         .withFunctionName("eq", EqualToEvaluator.class)
         .withFunctionName("gt", GreaterThanEvaluator.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/960a5fd7/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiVariateNormalDistributionEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiVariateNormalDistributionEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiVariateNormalDistributionEvaluator.java
new file mode 100644
index 0000000..bc2fbcb
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/MultiVariateNormalDistributionEvaluator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.List;
+
+import org.apache.commons.math3.distribution.MultivariateNormalDistribution;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class MultiVariateNormalDistributionEvaluator extends RecursiveObjectEvaluator implements TwoValueWorker {
+
+  private static final long serialVersionUID = 1;
+
+  public MultiVariateNormalDistributionEvaluator(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  @Override
+  public Object doWork(Object first, Object second) throws IOException{
+    if(null == first){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - null found for the first value",toExpression(constructingFactory)));
+    }
+    if(null == second){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - null found for the second value",toExpression(constructingFactory)));
+    }
+
+    List<Number> means = (List<Number>)first;
+    Matrix covar = (Matrix)second;
+
+    double[] m = new double[means.size()];
+    for(int i=0; i< m.length; i++) {
+      m[i] = means.get(i).doubleValue();
+    }
+
+    return new MultivariateNormalDistribution(m, covar.getData());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/960a5fd7/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
index 9b7aca5..5ea29e6 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SampleEvaluator.java
@@ -18,12 +18,16 @@
 package org.apache.solr.client.solrj.io.eval;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Locale;
 import java.util.stream.Collectors;
+import java.util.List;
 
 import org.apache.commons.math3.distribution.IntegerDistribution;
+import org.apache.commons.math3.distribution.MultivariateRealDistribution;
 import org.apache.commons.math3.distribution.RealDistribution;
+import org.apache.commons.math3.distribution.MultivariateNormalDistribution;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
@@ -43,7 +47,7 @@ public class SampleEvaluator extends RecursiveObjectEvaluator implements ManyVal
 
     Object first = objects[0];
 
-    if(!(first instanceof RealDistribution) && !(first instanceof IntegerDistribution) && !(first instanceof MarkovChainEvaluator.MarkovChain)){
+    if(!(first instanceof MultivariateRealDistribution) && !(first instanceof RealDistribution) && !(first instanceof IntegerDistribution) && !(first instanceof MarkovChainEvaluator.MarkovChain)){
       throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - found type %s for the first value, expecting a Markov Chain, Real or Integer Distribution",toExpression(constructingFactory), first.getClass().getSimpleName()));
     }
 
@@ -61,11 +65,30 @@ public class SampleEvaluator extends RecursiveObjectEvaluator implements ManyVal
       }
     } else if (first instanceof RealDistribution) {
       RealDistribution realDistribution = (RealDistribution) first;
-      if(second != null) {
+      if (second != null) {
         return Arrays.stream(realDistribution.sample(((Number) second).intValue())).mapToObj(item -> item).collect(Collectors.toList());
       } else {
         return realDistribution.sample();
       }
+    }else if(first instanceof MultivariateNormalDistribution) {
+      if(second != null) {
+        MultivariateNormalDistribution multivariateNormalDistribution = (MultivariateNormalDistribution)first;
+        int size = ((Number)second).intValue();
+        double[][] samples = new double[size][];
+        for(int i=0; i<size; ++i) {
+          samples[i] =  multivariateNormalDistribution.sample();
+        }
+
+        return new Matrix(samples);
+      } else {
+        MultivariateNormalDistribution multivariateNormalDistribution = (MultivariateNormalDistribution)first;
+        double[] sample = multivariateNormalDistribution.sample();
+        List<Number> sampleList = new ArrayList(sample.length);
+        for(int i=0; i<sample.length; i++) {
+          sampleList.add(sample[i]);
+        }
+        return sampleList;
+      }
     } else {
       IntegerDistribution integerDistribution = (IntegerDistribution) first;
       if(second != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/960a5fd7/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index c8fe2ff..01ba1dd 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -7179,6 +7179,53 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertEquals(pval3.doubleValue(), 0.0404907407662755, .0001);
   }
 
+  @Test
+  public void testMultiVariateNormalDistribution() throws Exception {
+    String cexpr = "let(echo=true," +
+        "     a=array(1,2,3,4,5,6,7)," +
+        "     b=array(100, 110, 120, 130,140,150,180)," +
+        "     c=transpose(matrix(a, b))," +
+        "     d=array(mean(a), mean(b))," +
+        "     e=cov(c)," +
+        "     f=multiVariateNormalDistribution(d, e)," +
+        "     g=sample(f, 10000)," +
+        "     h=cov(g)," +
+        "     i=sample(f))";
+
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    List<List<Number>> cov = (List<List<Number>>)tuples.get(0).get("h");
+    assertEquals(cov.size(), 2);
+    List<Number> row1 = cov.get(0);
+    assertEquals(row1.size(), 2);
+
+    double a = row1.get(0).doubleValue();
+    double b = row1.get(1).doubleValue();
+    assertEquals(a, 4.666666666666667, 2.5);
+    assertEquals(b, 56.66666666666667, 7);
+
+    List<Number> row2 = cov.get(1);
+
+    double c = row2.get(0).doubleValue();
+    double d = row2.get(1).doubleValue();
+    assertEquals(c, 56.66666666666667, 7);
+    assertEquals(d, 723.8095238095239, 50);
+
+    List<Number> sample = (List<Number>)tuples.get(0).get("i");
+    assertEquals(sample.size(), 2);
+    Number sample1 = sample.get(0);
+    Number sample2 = sample.get(1);
+    assertTrue(sample1.doubleValue() > -30 && sample1.doubleValue() < 30);
+    assertTrue(sample2.doubleValue() > 50 && sample2.doubleValue() < 250);
+  }
+
 
   @Test
   public void testLoess() throws Exception {


[31/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: OpenNLPPOSFilter: fix reset() to fully reset

Posted by da...@apache.org.
LUCENE-2899: OpenNLPPOSFilter: fix reset() to fully reset


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

Branch: refs/heads/jira/solr-11702
Commit: f8fb13965612142e9ee91631c6ce80a7b255e348
Parents: 8275952
Author: Steve Rowe <sa...@apache.org>
Authored: Mon Dec 18 19:03:08 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Mon Dec 18 19:03:29 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8fb1396/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
index a5bea28..275b29c 100644
--- a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
@@ -86,6 +86,7 @@ public final class OpenNLPPOSFilter extends TokenFilter {
   public void reset() throws IOException {
     super.reset();
     moreTokensAvailable = true;
+    clear();
   }
 
   private void clear() {


[44/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11789: Add integrate Stream Evaluator

Posted by da...@apache.org.
SOLR-11789: Add integrate Stream Evaluator


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

Branch: refs/heads/jira/solr-11702
Commit: b5d55b86dfca27d783915e4523cae76869cf67c9
Parents: 091f45d
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Dec 21 20:42:52 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 22 07:57:41 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  1 +
 .../solrj/io/eval/IntegrateEvaluator.java       | 70 ++++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 26 ++++++++
 3 files changed, 97 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b5d55b86/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 949a040..67d6f7f 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -284,6 +284,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("ttest", TTestEvaluator.class)
         .withFunctionName("pairedTtest", PairedTTestEvaluator.class)
         .withFunctionName("multiVariateNormalDistribution", MultiVariateNormalDistributionEvaluator.class)
+        .withFunctionName("integrate", IntegrateEvaluator.class)
 
         // Boolean Stream Evaluators
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b5d55b86/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IntegrateEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IntegrateEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IntegrateEvaluator.java
new file mode 100644
index 0000000..277748c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/IntegrateEvaluator.java
@@ -0,0 +1,70 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.Locale;
+
+import org.apache.commons.math3.analysis.UnivariateFunction;
+import org.apache.commons.math3.analysis.integration.RombergIntegrator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class IntegrateEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public IntegrateEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+
+  @Override
+  public Object doWork(Object... values) throws IOException {
+
+    if(values.length != 3) {
+      throw new IOException("The integrate function requires 3 parameters");
+    }
+
+    if (!(values[0] instanceof VectorFunction)) {
+      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - found type %s for the first value, expecting a FunctionVector", toExpression(constructingFactory), values[0].getClass().getSimpleName()));
+    }
+
+    VectorFunction vectorFunction = (VectorFunction) values[0];
+    if(!(vectorFunction.getFunction() instanceof UnivariateFunction)) {
+      throw new IOException("Cannot evaluate integral from parameter.");
+    }
+
+    Number min = null;
+    Number max = null;
+
+    if(values[1] instanceof Number) {
+      min = (Number) values[1];
+    } else {
+      throw new IOException("The second parameter of the integrate function must be a number");
+    }
+
+    if(values[2] instanceof Number ) {
+      max = (Number) values[2];
+    } else {
+      throw new IOException("The third parameter of the integrate function must be a number");
+    }
+
+    UnivariateFunction func = (UnivariateFunction)vectorFunction.getFunction();
+
+    RombergIntegrator rombergIntegrator = new RombergIntegrator();
+    return rombergIntegrator.integrate(5000, func, min.doubleValue(), max.doubleValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b5d55b86/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 01ba1dd..a9f0f66 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -7228,6 +7228,32 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
 
   @Test
+  public void testIntegrate() throws Exception {
+    String cexpr = "let(echo=true, " +
+                       "a=sequence(50, 1, 0), " +
+                       "b=spline(a), " +
+                       "c=integrate(b, 0, 49), " +
+                       "d=integrate(b, 0, 20), " +
+                       "e=integrate(b, 20, 49))";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    Number integral = (Number)tuples.get(0).get("c");
+    assertEquals(integral.doubleValue(), 49, 0.0);
+    integral = (Number)tuples.get(0).get("d");
+    assertEquals(integral.doubleValue(), 20, 0.0);
+    integral = (Number)tuples.get(0).get("e");
+    assertEquals(integral.doubleValue(), 29, 0.0);
+  }
+
+
+  @Test
   public void testLoess() throws Exception {
     String cexpr = "let(echo=true," +
                    "    a=array(0,1,2,3,4,5,6,7)," +


[10/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8093: Make TrimFilterFactory multi-term aware

Posted by da...@apache.org.
LUCENE-8093: Make TrimFilterFactory multi-term aware


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

Branch: refs/heads/jira/solr-11702
Commit: 8eb5c696e41595d8699a991bce99767a97707d5a
Parents: d6d2e3b
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 13 14:39:15 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 14 13:49:14 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                                      |  2 ++
 .../analysis/miscellaneous/TrimFilterFactory.java       | 12 +++++++++---
 .../lucene/analysis/util/MultiTermAwareComponent.java   |  5 +++++
 3 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8eb5c696/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c08e215..0fbf446 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -76,6 +76,8 @@ Improvements
   configurable precision for non-spherical planet models.
   (Ignacio Vera via David Smiley)
 
+* LUCENE-8093: TrimFilterFactory implements MultiTermAwareComponent (Alan Woodward)
+
 ======================= Lucene 7.2.0 =======================
 
 API Changes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8eb5c696/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
index aa3e8bd..1aad81d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TrimFilterFactory.java
@@ -21,6 +21,8 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.miscellaneous.TrimFilter;
+import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
+import org.apache.lucene.analysis.util.MultiTermAwareComponent;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 
 /**
@@ -35,7 +37,7 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
  *
  * @see TrimFilter
  */
-public class TrimFilterFactory extends TokenFilterFactory {
+public class TrimFilterFactory extends TokenFilterFactory implements MultiTermAwareComponent {
   
   /** Creates a new TrimFilterFactory */
   public TrimFilterFactory(Map<String,String> args) {
@@ -47,7 +49,11 @@ public class TrimFilterFactory extends TokenFilterFactory {
   
   @Override
   public TrimFilter create(TokenStream input) {
-    final TrimFilter filter = new TrimFilter(input);
-    return filter;
+    return new TrimFilter(input);
+  }
+
+  @Override
+  public AbstractAnalysisFactory getMultiTermComponent() {
+    return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8eb5c696/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/MultiTermAwareComponent.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/MultiTermAwareComponent.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/MultiTermAwareComponent.java
index 700da3c..1c1c4c8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/MultiTermAwareComponent.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/MultiTermAwareComponent.java
@@ -21,6 +21,11 @@ package org.apache.lucene.analysis.util;
  * analysis component factory for use with partial terms in prefix queries,
  * wildcard queries, range query endpoints, regex queries, etc.
  *
+ * Components implementing this interface should not add or remove tokens from
+ * the token stream, and should be able to deal with special characters
+ * indicating that multi-term queries are required (eg slashes for regex, wildcard
+ * characters, etc)
+ *
  * @lucene.experimental
  */
 public interface MultiTermAwareComponent {


[23/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8099: Remove deprecated CustomScoreQuery, BoostedQuery, BoostingQuery

Posted by da...@apache.org.
LUCENE-8099: Remove deprecated CustomScoreQuery, BoostedQuery, BoostingQuery


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

Branch: refs/heads/jira/solr-11702
Commit: c27099b4d1578dd37c284a4c23f3f812d98fe939
Parents: b01e602
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Dec 14 13:47:47 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Dec 18 17:47:03 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 lucene/MIGRATE.txt                              |  13 +
 .../highlight/WeightedSpanTermExtractor.java    |   3 -
 .../search/vectorhighlight/FieldQuery.java      |   6 -
 .../search/highlight/HighlighterTest.java       |  23 --
 .../FastVectorHighlighterTest.java              |  31 --
 .../apache/lucene/queries/BoostingQuery.java    | 198 ----------
 .../lucene/queries/CustomScoreProvider.java     | 162 ---------
 .../apache/lucene/queries/CustomScoreQuery.java | 334 -----------------
 .../lucene/queries/function/BoostedQuery.java   | 195 ----------
 .../lucene/queries/BoostingQueryTest.java       |  56 ---
 .../queries/TestCustomScoreExplanations.java    |  91 -----
 .../lucene/queries/TestCustomScoreQuery.java    | 361 -------------------
 .../queries/function/TestBoostedQuery.java      |  98 -----
 .../queryparser/xml/CorePlusQueriesParser.java  |   2 -
 .../xml/builders/BoostingQueryBuilder.java      |  58 ---
 16 files changed, 16 insertions(+), 1618 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ac70c4e..d8e1237 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -26,6 +26,9 @@ API Changes
 * LUCENE-7996: Queries are now required to produce positive scores.
   (Adrien Grand)
 
+* LUCENE-8099: CustomScoreQuery, BoostedQuery and BoostingQuery have been
+  removed (Alan Woodward)
+
 Changes in Runtime Behavior
 
 * LUCENE-7837: Indices that were created before the previous major version

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/MIGRATE.txt
----------------------------------------------------------------------
diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt
index 6151abd..9d02db8 100644
--- a/lucene/MIGRATE.txt
+++ b/lucene/MIGRATE.txt
@@ -15,3 +15,16 @@ negative scores.
 
 As a side-effect of this change, negative boosts are now rejected and
 FunctionScoreQuery maps negative values to 0.
+
+
+## CustomScoreQuery, BoostedQuery and BoostingQuery removed (LUCENE-8099) ##
+
+Instead use FunctionScoreQuery and a DoubleValuesSource implementation.  For example,
+to replace the functionality of BoostedQuery, you could do the following, using
+the lucene-expressions module:
+
+SimpleBindings bindings = new SimpleBindings();
+bindings.add("score", DoubleValuesSource.SCORES);
+bindings.add("boost", DoubleValuesSource.fromIntField("myboostfield"));
+Expression expr = JavascriptCompiler.compile("score * boost");
+FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
index 8e6bf24..004c06e 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
@@ -39,7 +39,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.queries.CommonTermsQuery;
-import org.apache.lucene.queries.CustomScoreQuery;
 import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -212,8 +211,6 @@ public class WeightedSpanTermExtractor {
       }
     } else if (query instanceof MatchAllDocsQuery) {
       //nothing
-    } else if (query instanceof CustomScoreQuery) {
-      extract(((CustomScoreQuery) query).getSubQuery(), boost, terms);
     } else if (query instanceof FunctionScoreQuery) {
       extract(((FunctionScoreQuery) query).getWrappedQuery(), boost, terms);
     } else if (isQueryUnsupported(query.getClass())) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
index 6daa791..cb502343 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
@@ -27,7 +27,6 @@ import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.CustomScoreQuery;
 import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -141,11 +140,6 @@ public class FieldQuery {
       if (q != null) {
         flatten( q, reader, flatQueries, boost);
       }
-    } else if (sourceQuery instanceof CustomScoreQuery) {
-      final Query q = ((CustomScoreQuery) sourceQuery).getSubQuery();
-      if (q != null) {
-        flatten(q, reader, flatQueries, boost);
-      }
     } else if (sourceQuery instanceof FunctionScoreQuery) {
       final Query q = ((FunctionScoreQuery)sourceQuery).getWrappedQuery();
       if (q != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
index cf14e27..5c2fe35 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
@@ -58,7 +58,6 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.CommonTermsQuery;
-import org.apache.lucene.queries.CustomScoreQuery;
 import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.queries.payloads.SpanPayloadCheckQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
@@ -146,28 +145,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
         searcher.doc(docId).get(fieldName), analyzer, -1);
   }
 
-  public void testCustomScoreQueryHighlight() throws Exception{
-    TermQuery termQuery = new TermQuery(new Term(FIELD_NAME, "very"));
-    CustomScoreQuery query = new CustomScoreQuery(termQuery);
-
-    searcher = newSearcher(reader);
-    TopDocs hits = searcher.search(query, 10, new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE));
-    assertEquals(2, hits.totalHits);
-    QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
-    Highlighter highlighter = new Highlighter(scorer);
-
-    final int docId0 = hits.scoreDocs[0].doc;
-    Document doc = searcher.doc(docId0);
-    String storedField = doc.get(FIELD_NAME);
-
-    TokenStream stream = getAnyTokenStream(FIELD_NAME, docId0);
-    Fragmenter fragmenter = new SimpleSpanFragmenter(scorer);
-    highlighter.setTextFragmenter(fragmenter);
-    String fragment = highlighter.getBestFragment(stream, storedField);
-    assertEquals("Hello this is a piece of text that is <B>very</B> long and contains too much preamble and the meat is really here which says kennedy has been shot", fragment);
-
-  }
-
   public void testFunctionScoreQuery() throws Exception {
     TermQuery termQuery = new TermQuery(new Term(FIELD_NAME, "very"));
     FunctionScoreQuery query = new FunctionScoreQuery(termQuery, DoubleValuesSource.constant(1));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
index 732f62f..0683e8d 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
@@ -38,7 +38,6 @@ import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.CommonTermsQuery;
-import org.apache.lucene.queries.CustomScoreQuery;
 import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanClause;
@@ -92,36 +91,6 @@ public class FastVectorHighlighterTest extends LuceneTestCase {
     dir.close();
   }
 
-  public void testCustomScoreQueryHighlight() throws IOException {
-    Directory dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-    Document doc = new Document();
-    FieldType type = new FieldType(TextField.TYPE_STORED);
-    type.setStoreTermVectorOffsets(true);
-    type.setStoreTermVectorPositions(true);
-    type.setStoreTermVectors(true);
-    type.freeze();
-    Field field = new Field("field", "This is a test where foo is highlighed and should be highlighted", type);
-
-    doc.add(field);
-    writer.addDocument(doc);
-    FastVectorHighlighter highlighter = new FastVectorHighlighter();
-
-    IndexReader reader = DirectoryReader.open(writer);
-    int docId = 0;
-    FieldQuery fieldQuery  = highlighter.getFieldQuery( new CustomScoreQuery(new TermQuery(new Term("field", "foo"))), reader );
-    String[] bestFragments = highlighter.getBestFragments(fieldQuery, reader, docId, "field", 54, 1);
-    // highlighted results are centered
-    assertEquals("This is a test where <b>foo</b> is highlighed and should be highlighted", bestFragments[0]);
-    bestFragments = highlighter.getBestFragments(fieldQuery, reader, docId, "field", 52, 1);
-    assertEquals("This is a test where <b>foo</b> is highlighed and should be", bestFragments[0]);
-    bestFragments = highlighter.getBestFragments(fieldQuery, reader, docId, "field", 30, 1);
-    assertEquals("a test where <b>foo</b> is highlighed", bestFragments[0]);
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-
   public void testFunctionScoreQueryHighlight() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
deleted file mode 100644
index 452cbe6..0000000
--- a/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-import java.util.Objects;
-import java.util.Set;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.FilterScorer;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.TwoPhaseIterator;
-import org.apache.lucene.search.Weight;
-
-/**
- * The BoostingQuery class can be used to effectively demote results that match a given query. 
- * Unlike the "NOT" clause, this still selects documents that contain undesirable terms, 
- * but reduces their overall score:
- *
- *     Query balancedQuery = new BoostingQuery(positiveQuery, negativeQuery, 0.01f);
- * In this scenario the positiveQuery contains the mandatory, desirable criteria which is used to 
- * select all matching documents, and the negativeQuery contains the undesirable elements which 
- * are simply used to lessen the scores. Documents that match the negativeQuery have their score 
- * multiplied by the supplied "boost" parameter, so this should be less than 1 to achieve a 
- * demoting effect
- * 
- * This code was originally made available here: 
- *   <a href="http://marc.theaimsgroup.com/?l=lucene-user&amp;m=108058407130459&amp;w=2">http://marc.theaimsgroup.com/?l=lucene-user&amp;m=108058407130459&amp;w=2</a>
- * and is documented here: http://wiki.apache.org/lucene-java/CommunityContributions
- *
- * Clients should instead use FunctionScoreQuery and the lucene-expressions library:
- * <pre>
- *   SimpleBindings bindings = new SimpleBindings();
- *   bindings.add("score", DoubleValuesSource.SCORES);
- *   bindings.add("context", DoubleValuesSource.fromQuery(new ConstantScoreQuery(myContextQuery, boost)));
- *   Expression expr = JavascriptCompiler.compile("score * context");
- *   FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));
- * </pre>
- *
- * @deprecated Use {@link org.apache.lucene.queries.function.FunctionScoreQuery}
- */
-@Deprecated
-public class BoostingQuery extends Query {
-    private final float boost;                            // the amount to boost by
-    private final Query match;                            // query to match
-    private final Query context;                          // boost when matches too
-
-    public BoostingQuery(Query match, Query context, float boost) {
-      this.match = match;
-      this.context = context; // ignore context-only matches
-      if (Float.isFinite(boost) == false || Float.compare(boost, 0f) < 0) {
-        // otherwise scores could be negative
-        throw new IllegalArgumentException("boost must be a non-negative float, got " + boost);
-      }
-      this.boost = boost;
-    }
-
-    @Override
-    public Query rewrite(IndexReader reader) throws IOException {
-      Query matchRewritten = match.rewrite(reader);
-      Query contextRewritten = context.rewrite(reader);
-      if (match != matchRewritten || context != contextRewritten) {
-        return new BoostingQuery(matchRewritten, contextRewritten, boost);
-      }
-      return super.rewrite(reader);
-    }
-
-    @Override
-    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-      if (scoreMode.needsScores() == false) {
-        return match.createWeight(searcher, scoreMode, boost);
-      }
-      final Weight matchWeight = searcher.createWeight(match, scoreMode, boost);
-      final Weight contextWeight = searcher.createWeight(context, ScoreMode.COMPLETE_NO_SCORES, boost);
-      return new Weight(this) {
-
-        @Override
-        public void extractTerms(Set<Term> terms) {
-          matchWeight.extractTerms(terms);
-          if (boost >= 1) {
-            contextWeight.extractTerms(terms);
-          }
-        }
-
-        @Override
-        public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-          final Explanation matchExplanation = matchWeight.explain(context, doc);
-          final Explanation contextExplanation = contextWeight.explain(context, doc);
-          if (matchExplanation.isMatch() == false || contextExplanation.isMatch() == false) {
-            return matchExplanation;
-          }
-          return Explanation.match(matchExplanation.getValue() * boost, "product of:",
-              matchExplanation,
-              Explanation.match(boost, "boost"));
-        }
-
-        @Override
-        public Scorer scorer(LeafReaderContext context) throws IOException {
-          final Scorer matchScorer = matchWeight.scorer(context);
-          if (matchScorer == null) {
-            return null;
-          }
-          final Scorer contextScorer = contextWeight.scorer(context);
-          if (contextScorer == null) {
-            return matchScorer;
-          }
-          TwoPhaseIterator contextTwoPhase = contextScorer.twoPhaseIterator();
-          DocIdSetIterator contextApproximation = contextTwoPhase == null
-              ? contextScorer.iterator()
-              : contextTwoPhase.approximation();
-          return new FilterScorer(matchScorer) {
-            @Override
-            public float score() throws IOException {
-              if (contextApproximation.docID() < docID()) {
-                contextApproximation.advance(docID());
-              }
-              assert contextApproximation.docID() >= docID();
-              float score = super.score();
-              if (contextApproximation.docID() == docID()
-                  && (contextTwoPhase == null || contextTwoPhase.matches())) {
-                score *= boost;
-              }
-              return score;
-            }
-            @Override
-            public float maxScore() {
-              float maxScore = matchScorer.maxScore();
-              if (boost > 1) {
-                maxScore *= boost;
-              }
-              return maxScore;
-            }
-          };
-        }
-
-        @Override
-        public boolean isCacheable(LeafReaderContext ctx) {
-          return matchWeight.isCacheable(ctx) && contextWeight.isCacheable(ctx);
-        }
-
-      };
-    }
-
-    public Query getMatch() {
-      return match;
-    }
-
-    public Query getContext() {
-      return context;
-    }
-
-    public float getBoost() {
-      return boost;
-    }
-
-    @Override
-    public int hashCode() {
-      return 31 * classHash() + Objects.hash(match, context, boost);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      return sameClassAs(other) &&
-             equalsTo(getClass().cast(other));
-    }
-
-    private boolean equalsTo(BoostingQuery other) {
-      return match.equals(other.match)
-          && context.equals(other.context)
-          && Float.floatToIntBits(boost) == Float.floatToIntBits(other.boost);
-    }
-
-    @Override
-    public String toString(String field) {
-      return match.toString(field) + "/" + context.toString(field);
-    }
-  }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreProvider.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreProvider.java b/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreProvider.java
deleted file mode 100644
index db67b94..0000000
--- a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreProvider.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.index.IndexReader; // for javadocs
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionQuery;
-import org.apache.lucene.search.Explanation;
-
-/**
- * An instance of this subclass should be returned by
- * {@link CustomScoreQuery#getCustomScoreProvider}, if you want
- * to modify the custom score calculation of a {@link CustomScoreQuery}.
- * <p>Since Lucene 2.9, queries operate on each segment of an index separately,
- * so the protected {@link #context} field can be used to resolve doc IDs,
- * as the supplied <code>doc</code> ID is per-segment and without knowledge
- * of the IndexReader you cannot access the document or DocValues.
- * 
- * @lucene.experimental
- * @since 2.9.2
- */
-public class CustomScoreProvider {
-
-  protected final LeafReaderContext context;
-
-  /**
-   * Creates a new instance of the provider class for the given {@link IndexReader}.
-   */
-  public CustomScoreProvider(LeafReaderContext context) {
-    this.context = context;
-  }
-
-  /**
-   * Compute a custom score by the subQuery score and a number of 
-   * {@link org.apache.lucene.queries.function.FunctionQuery} scores.
-   * <p> 
-   * Subclasses can override this method to modify the custom score.  
-   * <p>
-   * If your custom scoring is different than the default herein you 
-   * should override at least one of the two customScore() methods.
-   * If the number of {@link FunctionQuery function queries} is always &lt; 2 it is 
-   * sufficient to override the other 
-   * {@link #customScore(int, float, float) customScore()} 
-   * method, which is simpler. 
-   * <p>
-   * The default computation herein is a multiplication of given scores:
-   * <pre>
-   *     ModifiedScore = valSrcScore * valSrcScores[0] * valSrcScores[1] * ...
-   * </pre>
-   * 
-   * @param doc id of scored doc. 
-   * @param subQueryScore score of that doc by the subQuery.
-   * @param valSrcScores scores of that doc by the {@link FunctionQuery}.
-   * @return custom score.
-   */
-  public float customScore(int doc, float subQueryScore, float valSrcScores[]) throws IOException {
-    if (valSrcScores.length == 1) {
-      return customScore(doc, subQueryScore, valSrcScores[0]);
-    }
-    if (valSrcScores.length == 0) {
-      return customScore(doc, subQueryScore, 1);
-    }
-    float score = subQueryScore;
-    for (float valSrcScore : valSrcScores) {
-      score *= valSrcScore;
-    }
-    return score;
-  }
-
-  /**
-   * Compute a custom score by the subQuery score and the {@link FunctionQuery} score.
-   * <p> 
-   * Subclasses can override this method to modify the custom score.
-   * <p>
-   * If your custom scoring is different than the default herein you 
-   * should override at least one of the two customScore() methods.
-   * If the number of {@link FunctionQuery function queries} is always &lt; 2 it is 
-   * sufficient to override this customScore() method, which is simpler. 
-   * <p>
-   * The default computation herein is a multiplication of the two scores:
-   * <pre>
-   *     ModifiedScore = subQueryScore * valSrcScore
-   * </pre>
-   *
-   * @param doc id of scored doc. 
-   * @param subQueryScore score of that doc by the subQuery.
-   * @param valSrcScore score of that doc by the {@link FunctionQuery}.
-   * @return custom score.
-   */
-  public float customScore(int doc, float subQueryScore, float valSrcScore) throws IOException {
-    return subQueryScore * valSrcScore;
-  }
-
-  /**
-   * Explain the custom score.
-   * Whenever overriding {@link #customScore(int, float, float[])}, 
-   * this method should also be overridden to provide the correct explanation
-   * for the part of the custom scoring.
-   *  
-   * @param doc doc being explained.
-   * @param subQueryExpl explanation for the sub-query part.
-   * @param valSrcExpls explanation for the value source part.
-   * @return an explanation for the custom score
-   */
-  public Explanation customExplain(int doc, Explanation subQueryExpl, Explanation valSrcExpls[]) throws IOException {
-    if (valSrcExpls.length == 1) {
-      return customExplain(doc, subQueryExpl, valSrcExpls[0]);
-    }
-    if (valSrcExpls.length == 0) {
-      return subQueryExpl;
-    }
-    float valSrcScore = 1;
-    for (Explanation valSrcExpl : valSrcExpls) {
-      valSrcScore *= valSrcExpl.getValue();
-    }
-    
-    List<Explanation> subs = new ArrayList<>();
-    subs.add(subQueryExpl);
-    for (Explanation valSrcExpl : valSrcExpls) {
-      subs.add(valSrcExpl);
-    }
-    return Explanation.match(valSrcScore * subQueryExpl.getValue(), "custom score: product of:", subs);
-  }
-  
-  /**
-   * Explain the custom score.
-   * Whenever overriding {@link #customScore(int, float, float)}, 
-   * this method should also be overridden to provide the correct explanation
-   * for the part of the custom scoring.
-   *  
-   * @param doc doc being explained.
-   * @param subQueryExpl explanation for the sub-query part.
-   * @param valSrcExpl explanation for the value source part.
-   * @return an explanation for the custom score
-   */
-  public Explanation customExplain(int doc, Explanation subQueryExpl, Explanation valSrcExpl) throws IOException {
-    float valSrcScore = 1;
-    if (valSrcExpl != null) {
-      valSrcScore *= valSrcExpl.getValue();
-    }
-    return Explanation.match(valSrcScore * subQueryExpl.getValue(), "custom score: product of:", subQueryExpl, valSrcExpl);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
deleted file mode 100644
index f82cf50..0000000
--- a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.FunctionQuery;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.FilterScorer;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
-
-/**
- * Query that sets document score as a programmatic function of several (sub) scores:
- * <ol>
- *    <li>the score of its subQuery (any query)</li>
- *    <li>(optional) the score of its {@link FunctionQuery} (or queries).</li>
- * </ol>
- * Subclasses can modify the computation by overriding {@link #getCustomScoreProvider}.
- * 
- * @lucene.experimental
- *
- * Clients should instead use FunctionScoreQuery and the lucene-expressions library
- * <pre>
- *   SimpleBindings bindings = new SimpleBindings();
- *   bindings.add("score", DoubleValuesSource.SCORES);
- *   bindings.add("boost", DoubleValuesSource.fromIntField("myboostfield"));
- *   Expression expr = JavascriptCompiler.compile("score * boost");
- *   FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));
- * </pre>
- *
- * @deprecated use {@link org.apache.lucene.queries.function.FunctionScoreQuery}
- */
-@Deprecated
-public class CustomScoreQuery extends Query implements Cloneable {
-
-  private Query subQuery;
-  private Query[] scoringQueries; // never null (empty array if there are no valSrcQueries).
-
-  /**
-   * Create a CustomScoreQuery over input subQuery.
-   * @param subQuery the sub query whose scored is being customized. Must not be null. 
-   */
-  public CustomScoreQuery(Query subQuery) {
-    this(subQuery, new FunctionQuery[0]);
-  }
-
-  /**
-   * Create a CustomScoreQuery over input subQuery and a {@link org.apache.lucene.queries.function.FunctionQuery}.
-   * @param subQuery the sub query whose score is being customized. Must not be null.
-   * @param scoringQuery a value source query whose scores are used in the custom score
-   * computation.  This parameter is optional - it can be null.
-   */
-  public CustomScoreQuery(Query subQuery, FunctionQuery scoringQuery) {
-    this(subQuery, scoringQuery!=null ? // don't want an array that contains a single null..
-        new FunctionQuery[] {scoringQuery} : new FunctionQuery[0]);
-  }
-
-  /**
-   * Create a CustomScoreQuery over input subQuery and a {@link org.apache.lucene.queries.function.FunctionQuery}.
-   * @param subQuery the sub query whose score is being customized. Must not be null.
-   * @param scoringQueries value source queries whose scores are used in the custom score
-   * computation.  This parameter is optional - it can be null or even an empty array.
-   */
-  public CustomScoreQuery(Query subQuery, FunctionQuery... scoringQueries) {
-    this.subQuery = subQuery;
-    this.scoringQueries = scoringQueries !=null?
-        scoringQueries : new Query[0];
-    if (subQuery == null) throw new IllegalArgumentException("<subquery> must not be null!");
-  }
-
-  /*(non-Javadoc) @see org.apache.lucene.search.Query#rewrite(org.apache.lucene.index.IndexReader) */
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    CustomScoreQuery clone = null;
-    
-    final Query sq = subQuery.rewrite(reader);
-    if (sq != subQuery) {
-      clone = clone();
-      clone.subQuery = sq;
-    }
-
-    for(int i = 0; i < scoringQueries.length; i++) {
-      final Query v = scoringQueries[i].rewrite(reader);
-      if (v != scoringQueries[i]) {
-        if (clone == null) clone = clone();
-        clone.scoringQueries[i] = v;
-      }
-    }
-    
-    return (clone == null) ? this : clone;
-  }
-
-  /*(non-Javadoc) @see org.apache.lucene.search.Query#clone() */
-  @Override
-  public CustomScoreQuery clone() {
-    CustomScoreQuery clone;
-    try {
-      clone = (CustomScoreQuery)super.clone();
-    } catch (CloneNotSupportedException bogus) {
-      // cannot happen
-      throw new Error(bogus);
-    }
-    clone.subQuery = subQuery;
-    clone.scoringQueries = new Query[scoringQueries.length];
-    for(int i = 0; i < scoringQueries.length; i++) {
-      clone.scoringQueries[i] = scoringQueries[i];
-    }
-    return clone;
-  }
-
-  /* (non-Javadoc) @see org.apache.lucene.search.Query#toString(java.lang.String) */
-  @Override
-  public String toString(String field) {
-    StringBuilder sb = new StringBuilder(name()).append("(");
-    sb.append(subQuery.toString(field));
-    for (Query scoringQuery : scoringQueries) {
-      sb.append(", ").append(scoringQuery.toString(field));
-    }
-    sb.append(")");
-    return sb.toString();
-  }
-
-  /** Returns true if <code>o</code> is equal to this. */
-  @Override
-  public boolean equals(Object other) {
-    return sameClassAs(other) &&
-           equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(CustomScoreQuery other) {
-    return subQuery.equals(other.subQuery) &&
-           scoringQueries.length == other.scoringQueries.length &&
-           Arrays.equals(scoringQueries, other.scoringQueries);
-  }
-
-  /** Returns a hash code value for this object. */
-  @Override
-  public int hashCode() {
-    // Didn't change this hashcode, but it looks suspicious.
-    return (classHash() + 
-        subQuery.hashCode() + 
-        Arrays.hashCode(scoringQueries));
-  }
-  
-  /**
-   * Returns a {@link CustomScoreProvider} that calculates the custom scores
-   * for the given {@link IndexReader}. The default implementation returns a default
-   * implementation as specified in the docs of {@link CustomScoreProvider}.
-   * @since 2.9.2
-   */
-  protected CustomScoreProvider getCustomScoreProvider(LeafReaderContext context) throws IOException {
-    return new CustomScoreProvider(context);
-  }
-
-  //=========================== W E I G H T ============================
-  
-  private class CustomWeight extends Weight {
-    final Weight subQueryWeight;
-    final Weight[] valSrcWeights;
-    final float queryWeight;
-
-    public CustomWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-      super(CustomScoreQuery.this);
-      // note we DONT incorporate our boost, nor pass down any boost 
-      // (e.g. from outer BQ), as there is no guarantee that the CustomScoreProvider's 
-      // function obeys the distributive law... it might call sqrt() on the subQuery score
-      // or some other arbitrary function other than multiplication.
-      // so, instead boosts are applied directly in score()
-      this.subQueryWeight = subQuery.createWeight(searcher, scoreMode, 1f);
-      this.valSrcWeights = new Weight[scoringQueries.length];
-      for(int i = 0; i < scoringQueries.length; i++) {
-        this.valSrcWeights[i] = scoringQueries[i].createWeight(searcher, scoreMode, 1f);
-      }
-      this.queryWeight = boost;
-    }
-
-    @Override
-    public void extractTerms(Set<Term> terms) {
-      subQueryWeight.extractTerms(terms);
-      for (Weight scoringWeight : valSrcWeights) {
-        scoringWeight.extractTerms(terms);
-      }
-    }
-
-    @Override
-    public Scorer scorer(LeafReaderContext context) throws IOException {
-      Scorer subQueryScorer = subQueryWeight.scorer(context);
-      if (subQueryScorer == null) {
-        return null;
-      }
-      Scorer[] valSrcScorers = new Scorer[valSrcWeights.length];
-      for(int i = 0; i < valSrcScorers.length; i++) {
-         valSrcScorers[i] = valSrcWeights[i].scorer(context);
-      }
-      return new CustomScorer(CustomScoreQuery.this.getCustomScoreProvider(context), this, queryWeight, subQueryScorer, valSrcScorers);
-    }
-
-    @Override
-    public boolean isCacheable(LeafReaderContext ctx) {
-      if (subQueryWeight.isCacheable(ctx) == false)
-        return false;
-      for (Weight w : valSrcWeights) {
-        if (w.isCacheable(ctx) == false)
-          return false;
-      }
-      return true;
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Explanation explain = doExplain(context, doc);
-      return explain == null ? Explanation.noMatch("no matching docs") : explain;
-    }
-    
-    private Explanation doExplain(LeafReaderContext info, int doc) throws IOException {
-      Explanation subQueryExpl = subQueryWeight.explain(info, doc);
-      if (!subQueryExpl.isMatch()) {
-        return subQueryExpl;
-      }
-      // match
-      Explanation[] valSrcExpls = new Explanation[valSrcWeights.length];
-      for(int i = 0; i < valSrcWeights.length; i++) {
-        valSrcExpls[i] = valSrcWeights[i].explain(info, doc);
-      }
-      Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(info).customExplain(doc,subQueryExpl,valSrcExpls);
-      float sc = queryWeight * customExp.getValue();
-      return Explanation.match(
-        sc, CustomScoreQuery.this.toString() + ", product of:",
-        customExp, Explanation.match(queryWeight, "queryWeight"));
-    }
-    
-  }
-
-
-  //=========================== S C O R E R ============================
-  
-  /**
-   * A scorer that applies a (callback) function on scores of the subQuery.
-   */
-  private static class CustomScorer extends FilterScorer {
-    private final float qWeight;
-    private final Scorer subQueryScorer;
-    private final Scorer[] valSrcScorers;
-    private final CustomScoreProvider provider;
-    private final float[] vScores; // reused in score() to avoid allocating this array for each doc
-    private int valSrcDocID = -1; // we lazily advance subscorers.
-
-    // constructor
-    private CustomScorer(CustomScoreProvider provider, CustomWeight w, float qWeight,
-        Scorer subQueryScorer, Scorer[] valSrcScorers) {
-      super(subQueryScorer, w);
-      this.qWeight = qWeight;
-      this.subQueryScorer = subQueryScorer;
-      this.valSrcScorers = valSrcScorers;
-      this.vScores = new float[valSrcScorers.length];
-      this.provider = provider;
-    }
-    
-    @Override
-    public float score() throws IOException {
-      // lazily advance to current doc.
-      int doc = docID();
-      if (doc > valSrcDocID) {
-        for (Scorer valSrcScorer : valSrcScorers) {
-          valSrcScorer.iterator().advance(doc);
-        }
-        valSrcDocID = doc;
-      }
-      // TODO: this thing technically takes any Query, so what about when subs don't match?
-      for (int i = 0; i < valSrcScorers.length; i++) {
-        vScores[i] = valSrcScorers[i].score();
-      }
-      return qWeight * provider.customScore(subQueryScorer.docID(), subQueryScorer.score(), vScores);
-    }
-
-    @Override
-    public float maxScore() {
-      return Float.POSITIVE_INFINITY;
-    }
-    
-    @Override
-    public Collection<ChildScorer> getChildren() {
-      return Collections.singleton(new ChildScorer(subQueryScorer, "CUSTOM"));
-    }
-  }
-
-  @Override
-  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    return new CustomWeight(searcher, scoreMode, boost);
-  }
-
-  /** The sub-query that CustomScoreQuery wraps, affecting both the score and which documents match. */
-  public Query getSubQuery() {
-    return subQuery;
-  }
-
-  /** The scoring queries that only affect the score of CustomScoreQuery. */
-  public Query[] getScoringQueries() {
-    return scoringQueries;
-  }
-
-  /**
-   * A short name of this query, used in {@link #toString(String)}.
-   */
-  public String name() {
-    return "custom";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
deleted file mode 100644
index ddb261a..0000000
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries.function;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.FilterScorer;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
-
-/**
- * Query that is boosted by a ValueSource
- *
- * Instead of using this query, clients can use a {@link FunctionScoreQuery} and the
- * lucene-expressions library:
- * <pre>
- *   SimpleBindings bindings = new SimpleBindings();
- *   bindings.add("score", DoubleValuesSource.SCORES);
- *   bindings.add("boost", DoubleValuesSource.fromIntField("myboostfield"));
- *   Expression expr = JavascriptCompiler.compile("score * boost");
- *   FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));
- * </pre>
- *
- * @deprecated Use {@link FunctionScoreQuery}
- */
-@Deprecated
-public final class BoostedQuery extends Query {
-  private final Query q;
-  private final ValueSource boostVal; // optional, can be null
-
-  public BoostedQuery(Query subQuery, ValueSource boostVal) {
-    this.q = subQuery;
-    this.boostVal = boostVal;
-  }
-
-  public Query getQuery() { return q; }
-  public ValueSource getValueSource() { return boostVal; }
-
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    Query newQ = q.rewrite(reader);
-    if (newQ != q) {
-      return new BoostedQuery(newQ, boostVal);
-    }
-    return super.rewrite(reader);
-  }
-
-  @Override
-  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    return new BoostedQuery.BoostedWeight(searcher, scoreMode, boost);
-  }
-
-  private class BoostedWeight extends Weight {
-    Weight qWeight;
-    Map fcontext;
-
-    public BoostedWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-      super(BoostedQuery.this);
-      this.qWeight = searcher.createWeight(q, scoreMode, boost);
-      this.fcontext = ValueSource.newContext(searcher);
-      boostVal.createWeight(fcontext,searcher);
-    }
-
-    @Override
-    public void extractTerms(Set<Term> terms) {
-      qWeight.extractTerms(terms);
-    }
-
-    @Override
-    public Scorer scorer(LeafReaderContext context) throws IOException {
-      Scorer subQueryScorer = qWeight.scorer(context);
-      if (subQueryScorer == null) {
-        return null;
-      }
-      return new BoostedQuery.CustomScorer(context, this, subQueryScorer, boostVal);
-    }
-
-    @Override
-    public boolean isCacheable(LeafReaderContext ctx) {
-      return false;
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext readerContext, int doc) throws IOException {
-      Explanation subQueryExpl = qWeight.explain(readerContext,doc);
-      if (!subQueryExpl.isMatch()) {
-        return subQueryExpl;
-      }
-      FunctionValues vals = boostVal.getValues(fcontext, readerContext);
-      float factor = vals.floatVal(doc);
-      Explanation factorExpl = vals.explain(doc);
-      if (factor < 0) {
-        factor = 0;
-        factorExpl = Explanation.match(0, "truncated score, max of:",
-            Explanation.match(0f, "minimum score"), factorExpl);
-      } else if (Float.isNaN(factor)) {
-        factor = 0;
-        factorExpl = Explanation.match(0, "score, computed as (score == NaN ? 0 : score) since NaN is an illegal score from:", factorExpl);
-      }
-      
-      float sc = subQueryExpl.getValue() * factor;
-      return Explanation.match(sc, BoostedQuery.this.toString() + ", product of:",
-          subQueryExpl, factorExpl);
-    }
-  }
-
-
-  private class CustomScorer extends FilterScorer {
-    private final BoostedQuery.BoostedWeight weight;
-    private final ValueSource vs;
-    private final FunctionValues vals;
-    private final LeafReaderContext readerContext;
-
-    private CustomScorer(LeafReaderContext readerContext, BoostedQuery.BoostedWeight w,
-        Scorer scorer, ValueSource vs) throws IOException {
-      super(scorer);
-      this.weight = w;
-      this.readerContext = readerContext;
-      this.vs = vs;
-      this.vals = vs.getValues(weight.fcontext, readerContext);
-    }
-
-    @Override   
-    public float score() throws IOException {
-      float factor = vals.floatVal(in.docID());
-      if (factor >= 0 == false) { // covers NaN as well
-        factor = 0;
-      }
-      return in.score() * factor;
-    }
-
-    @Override
-    public float maxScore() {
-      return Float.POSITIVE_INFINITY;
-    }
-
-    @Override
-    public Collection<ChildScorer> getChildren() {
-      return Collections.singleton(new ChildScorer(in, "CUSTOM"));
-    }
-  }
-
-
-  @Override
-  public String toString(String field) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("boost(").append(q.toString(field)).append(',').append(boostVal).append(')');
-    return sb.toString();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    return sameClassAs(other) &&
-           equalsTo(getClass().cast(other));
-  }
-
-  private boolean equalsTo(BoostedQuery other) {
-    return q.equals(other.q) &&
-           boostVal.equals(other.boostVal);
-  }
-
-  @Override
-  public int hashCode() {
-    int h = classHash();
-    h = 31 * h + q.hashCode();
-    h = 31 * h + boostVal.hashCode();
-    return h;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/test/org/apache/lucene/queries/BoostingQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/BoostingQueryTest.java b/lucene/queries/src/test/org/apache/lucene/queries/BoostingQueryTest.java
deleted file mode 100644
index a224bda..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/BoostingQueryTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.MultiReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.MatchNoDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryUtils;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.util.LuceneTestCase;
-
-public class BoostingQueryTest extends LuceneTestCase {
-  // TODO: this suite desperately needs more tests!
-  // ... like ones that actually run the query
-  
-  public void testBoostingQueryEquals() {
-    TermQuery q1 = new TermQuery(new Term("subject:", "java"));
-    TermQuery q2 = new TermQuery(new Term("subject:", "java"));
-    assertEquals("Two TermQueries with same attributes should be equal", q1, q2);
-    BoostingQuery bq1 = new BoostingQuery(q1, q2, 0.1f);
-    QueryUtils.check(bq1);
-    BoostingQuery bq2 = new BoostingQuery(q1, q2, 0.1f);
-    assertEquals("BoostingQuery with same attributes is not equal", bq1, bq2);
-  }
-
-  public void testRewrite() throws IOException {
-    IndexReader reader = new MultiReader();
-    BoostingQuery q = new BoostingQuery(new BooleanQuery.Builder().build(), new MatchAllDocsQuery(), 3);
-    Query rewritten = new IndexSearcher(reader).rewrite(q);
-    Query expectedRewritten = new BoostingQuery(new MatchNoDocsQuery(), new MatchAllDocsQuery(), 3);
-    assertEquals(expectedRewritten, rewritten);
-    assertSame(rewritten, rewritten.rewrite(reader));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreExplanations.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreExplanations.java b/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreExplanations.java
deleted file mode 100644
index ab3cced..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreExplanations.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.FunctionQuery;
-import org.apache.lucene.queries.function.valuesource.ConstValueSource;
-import org.apache.lucene.search.BaseExplanationTestCase;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.similarities.BM25Similarity;
-import org.apache.lucene.search.similarities.ClassicSimilarity;
-
-public class TestCustomScoreExplanations extends BaseExplanationTestCase {
-  public void testOneTerm() throws Exception {
-    Query q = new TermQuery(new Term(FIELD, "w1"));
-    CustomScoreQuery csq = new CustomScoreQuery(q, new FunctionQuery(new ConstValueSource(5)));
-    qtest(csq, new int[] { 0,1,2,3 });
-  }
-  
-  public void testBoost() throws Exception {
-    Query q = new TermQuery(new Term(FIELD, "w1"));
-    CustomScoreQuery csq = new CustomScoreQuery(q, new FunctionQuery(new ConstValueSource(5)));
-    qtest(new BoostQuery(csq, 4), new int[] { 0,1,2,3 });
-  }
-  
-  public void testTopLevelBoost() throws Exception {
-    Query q = new TermQuery(new Term(FIELD, "w1"));
-    CustomScoreQuery csq = new CustomScoreQuery(q, new FunctionQuery(new ConstValueSource(5)));
-    BooleanQuery.Builder bqB = new BooleanQuery.Builder();
-    bqB.add(new MatchAllDocsQuery(), BooleanClause.Occur.MUST);
-    bqB.add(csq, BooleanClause.Occur.MUST);
-    BooleanQuery bq = bqB.build();
-    qtest(new BoostQuery(bq, 6), new int[] { 0,1,2,3 });
-  }
-
-  public void testSubExplanations() throws IOException {
-    Query query = new FunctionQuery(new ConstValueSource(5));
-    IndexSearcher searcher = newSearcher(BaseExplanationTestCase.searcher.getIndexReader());
-    searcher.setSimilarity(new BM25Similarity());
-
-    Explanation expl = searcher.explain(query, 0);
-    assertEquals(2, expl.getDetails().length);
-    // function
-    assertEquals(5f, expl.getDetails()[0].getValue(), 0f);
-    // boost
-    assertEquals("boost", expl.getDetails()[1].getDescription());
-    assertEquals(1f, expl.getDetails()[1].getValue(), 0f);
-
-    query = new BoostQuery(query, 2);
-    expl = searcher.explain(query, 0);
-    assertEquals(2, expl.getDetails().length);
-    // function
-    assertEquals(5f, expl.getDetails()[0].getValue(), 0f);
-    // boost
-    assertEquals("boost", expl.getDetails()[1].getDescription());
-    assertEquals(2f, expl.getDetails()[1].getValue(), 0f);
-
-    searcher.setSimilarity(new ClassicSimilarity()); // in order to have a queryNorm != 1
-    expl = searcher.explain(query, 0);
-    assertEquals(2, expl.getDetails().length);
-    // function
-    assertEquals(5f, expl.getDetails()[0].getValue(), 0f);
-    // boost
-    assertEquals("boost", expl.getDetails()[1].getDescription());
-    assertEquals(2f, expl.getDetails()[1].getValue(), 0f);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java
deleted file mode 100644
index d65e646..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java
+++ /dev/null
@@ -1,361 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.FunctionQuery;
-import org.apache.lucene.queries.function.FunctionTestSetup;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryUtils;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TermRangeQuery;
-import org.apache.lucene.search.TopDocs;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Test CustomScoreQuery search.
- */
-public class TestCustomScoreQuery extends FunctionTestSetup {
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    createIndex(true);
-  }
-
-  /**
-   * Test that CustomScoreQuery of Type.INT returns the expected scores.
-   */
-  @Test
-  public void testCustomScoreInt() throws Exception {
-    doTestCustomScore(INT_VALUESOURCE, 1f);
-    doTestCustomScore(INT_VALUESOURCE, 4f);
-  }
-
-  /**
-   * Test that CustomScoreQuery of Type.FLOAT returns the expected scores.
-   */
-  @Test
-  public void testCustomScoreFloat() throws Exception {
-    doTestCustomScore(FLOAT_VALUESOURCE, 1f);
-    doTestCustomScore(FLOAT_VALUESOURCE, 6f);
-  }
-
-  // must have static class otherwise serialization tests fail
-  private static class CustomAddQuery extends CustomScoreQuery {
-    // constructor
-    CustomAddQuery(Query q, FunctionQuery qValSrc) {
-      super(q, qValSrc);
-    }
-
-    /*(non-Javadoc) @see org.apache.lucene.search.function.CustomScoreQuery#name() */
-    @Override
-    public String name() {
-      return "customAdd";
-    }
-    
-    @Override
-    protected CustomScoreProvider getCustomScoreProvider(LeafReaderContext context) {
-      return new CustomScoreProvider(context) {
-        @Override
-        public float customScore(int doc, float subQueryScore, float valSrcScore) {
-          return subQueryScore + valSrcScore;
-        }
-
-        @Override
-        public Explanation customExplain(int doc, Explanation subQueryExpl, Explanation valSrcExpl) {
-          List<Explanation> subs = new ArrayList<>();
-          subs.add(subQueryExpl);
-          if (valSrcExpl != null) {
-            subs.add(valSrcExpl);
-          }
-          float valSrcScore = valSrcExpl == null ? 0 : valSrcExpl.getValue();
-          return Explanation.match(valSrcScore + subQueryExpl.getValue(), "custom score: sum of:", subs);
-        }
-      };
-    }
-  }
-
-  // must have static class otherwise serialization tests fail
-  private static class CustomMulAddQuery extends CustomScoreQuery {
-    // constructor
-    CustomMulAddQuery(Query q, FunctionQuery qValSrc1, FunctionQuery qValSrc2) {
-      super(q, qValSrc1, qValSrc2);
-    }
-
-    /*(non-Javadoc) @see org.apache.lucene.search.function.CustomScoreQuery#name() */
-    @Override
-    public String name() {
-      return "customMulAdd";
-    }
-
-    @Override
-    protected CustomScoreProvider getCustomScoreProvider(LeafReaderContext context) {
-      return new CustomScoreProvider(context) {
-        @Override
-        public float customScore(int doc, float subQueryScore, float valSrcScores[]) {
-          if (valSrcScores.length == 0) {
-            return subQueryScore;
-          }
-          if (valSrcScores.length == 1) {
-            return subQueryScore + valSrcScores[0];
-            // confirm that skipping beyond the last doc, on the
-            // previous reader, hits NO_MORE_DOCS
-          }
-          return (subQueryScore + valSrcScores[0]) * valSrcScores[1]; // we know there are two
-        }
-
-        @Override
-        public Explanation customExplain(int doc, Explanation subQueryExpl, Explanation valSrcExpls[]) {
-          if (valSrcExpls.length == 0) {
-            return subQueryExpl;
-          }
-          if (valSrcExpls.length == 1) {
-            return Explanation.match(valSrcExpls[0].getValue() + subQueryExpl.getValue(), "CustomMulAdd, sum of:", subQueryExpl, valSrcExpls[0]);
-          } else {
-            Explanation exp = Explanation.match(valSrcExpls[0].getValue() + subQueryExpl.getValue(), "sum of:", subQueryExpl, valSrcExpls[0]);
-            return Explanation.match(valSrcExpls[1].getValue() * exp.getValue(), "custom score: product of:", valSrcExpls[1], exp);
-          }
-        }
-      };
-    }
-  }
-
-  private static final class CustomExternalQuery extends CustomScoreQuery {
-
-    @Override
-    protected CustomScoreProvider getCustomScoreProvider(LeafReaderContext context) throws IOException {
-      final NumericDocValues values = DocValues.getNumeric(context.reader(), INT_FIELD);
-      return new CustomScoreProvider(context) {
-        @Override
-        public float customScore(int doc, float subScore, float valSrcScore) throws IOException {
-          assertTrue(doc <= context.reader().maxDoc());
-          if (values.docID() < doc) {
-            values.advance(doc);
-          }
-          if (doc == values.docID()) {
-            return values.longValue();
-          } else {
-            return 0;
-          }
-        }
-      };
-    }
-
-    public CustomExternalQuery(Query q) {
-      super(q);
-    }
-  }
-
-  @Test
-  public void testCustomExternalQuery() throws Exception {
-    BooleanQuery.Builder q1 = new BooleanQuery.Builder();
-    q1.add(new TermQuery(new Term(TEXT_FIELD, "first")), BooleanClause.Occur.SHOULD);
-    q1.add(new TermQuery(new Term(TEXT_FIELD, "aid")), BooleanClause.Occur.SHOULD);
-    q1.add(new TermQuery(new Term(TEXT_FIELD, "text")), BooleanClause.Occur.SHOULD);
-    
-    final Query q = new CustomExternalQuery(q1.build());
-    log(q);
-
-    IndexReader r = DirectoryReader.open(dir);
-    IndexSearcher s = newSearcher(r);
-    TopDocs hits = s.search(q, 1000);
-    assertEquals(N_DOCS, hits.totalHits);
-    for(int i=0;i<N_DOCS;i++) {
-      final int doc = hits.scoreDocs[i].doc;
-      final float score = hits.scoreDocs[i].score;
-      assertEquals("doc=" + doc, (float) 1+(4*doc) % N_DOCS, score, 0.0001);
-    }
-    r.close();
-  }
-  
-  @Test
-  public void testRewrite() throws Exception {
-    IndexReader r = DirectoryReader.open(dir);
-    final IndexSearcher s = newSearcher(r);
-
-    Query q = new TermQuery(new Term(TEXT_FIELD, "first"));
-    CustomScoreQuery original = new CustomScoreQuery(q);
-    CustomScoreQuery rewritten = (CustomScoreQuery) original.rewrite(s.getIndexReader());
-    assertTrue("rewritten query should be identical, as TermQuery does not rewrite", original == rewritten);
-    assertTrue("no hits for query", s.search(rewritten,1).totalHits > 0);
-    assertEquals(s.search(q,1).totalHits, s.search(rewritten,1).totalHits);
-
-    q = new TermRangeQuery(TEXT_FIELD, null, null, true, true); // everything
-    original = new CustomScoreQuery(q);
-    rewritten = (CustomScoreQuery) original.rewrite(s.getIndexReader());
-    assertTrue("rewritten query should not be identical, as TermRangeQuery rewrites", original != rewritten);
-    assertTrue("no hits for query", s.search(rewritten,1).totalHits > 0);
-    assertEquals(s.search(q,1).totalHits, s.search(original,1).totalHits);
-    assertEquals(s.search(q,1).totalHits, s.search(rewritten,1).totalHits);
-    
-    r.close();
-  }
-  
-  // Test that FieldScoreQuery returns docs with expected score.
-  private void doTestCustomScore(ValueSource valueSource, float boost) throws Exception {
-    FunctionQuery functionQuery = new FunctionQuery(valueSource);
-    IndexReader r = DirectoryReader.open(dir);
-    IndexSearcher s = newSearcher(r);
-
-    // regular (boolean) query.
-    BooleanQuery.Builder q1b = new BooleanQuery.Builder();
-    q1b.add(new TermQuery(new Term(TEXT_FIELD, "first")), BooleanClause.Occur.SHOULD);
-    q1b.add(new TermQuery(new Term(TEXT_FIELD, "aid")), BooleanClause.Occur.SHOULD);
-    q1b.add(new TermQuery(new Term(TEXT_FIELD, "text")), BooleanClause.Occur.SHOULD);
-    Query q1 = q1b.build();
-    log(q1);
-
-    // custom query, that should score the same as q1.
-    BooleanQuery.Builder q2CustomNeutralB = new BooleanQuery.Builder();
-    Query q2CustomNeutralInner = new CustomScoreQuery(q1);
-    q2CustomNeutralB.add(new BoostQuery(q2CustomNeutralInner, (float)Math.sqrt(boost)), BooleanClause.Occur.SHOULD);
-    // a little tricky: we split the boost across an outer BQ and CustomScoreQuery
-    // this ensures boosting is correct across all these functions (see LUCENE-4935)
-    Query q2CustomNeutral = q2CustomNeutralB.build();
-    q2CustomNeutral = new BoostQuery(q2CustomNeutral, (float)Math.sqrt(boost));
-    log(q2CustomNeutral);
-
-    // custom query, that should (by default) multiply the scores of q1 by that of the field
-    Query q3CustomMul;
-    {
-      CustomScoreQuery csq = new CustomScoreQuery(q1, functionQuery);
-      q3CustomMul = csq;
-    }
-    q3CustomMul = new BoostQuery(q3CustomMul, boost);
-    log(q3CustomMul);
-
-    // custom query, that should add the scores of q1 to that of the field
-    Query q4CustomAdd;
-    {
-      CustomScoreQuery csq = new CustomAddQuery(q1, functionQuery);
-      q4CustomAdd = csq;
-    }
-    q4CustomAdd = new BoostQuery(q4CustomAdd, boost);
-    log(q4CustomAdd);
-
-    // custom query, that multiplies and adds the field score to that of q1
-    Query q5CustomMulAdd;
-    {
-      CustomScoreQuery csq = new CustomMulAddQuery(q1, functionQuery, functionQuery);
-      q5CustomMulAdd = csq;
-    }
-    q5CustomMulAdd = new BoostQuery(q5CustomMulAdd, boost);
-    log(q5CustomMulAdd);
-
-    // do al the searches 
-    TopDocs td1 = s.search(q1, 1000);
-    TopDocs td2CustomNeutral = s.search(q2CustomNeutral, 1000);
-    TopDocs td3CustomMul = s.search(q3CustomMul, 1000);
-    TopDocs td4CustomAdd = s.search(q4CustomAdd, 1000);
-    TopDocs td5CustomMulAdd = s.search(q5CustomMulAdd, 1000);
-
-    // put results in map so we can verify the scores although they have changed
-    Map<Integer,Float> h1               = topDocsToMap(td1);
-    Map<Integer,Float> h2CustomNeutral  = topDocsToMap(td2CustomNeutral);
-    Map<Integer,Float> h3CustomMul      = topDocsToMap(td3CustomMul);
-    Map<Integer,Float> h4CustomAdd      = topDocsToMap(td4CustomAdd);
-    Map<Integer,Float> h5CustomMulAdd   = topDocsToMap(td5CustomMulAdd);
-    
-    verifyResults(boost, s, 
-        h1, h2CustomNeutral, h3CustomMul, h4CustomAdd, h5CustomMulAdd,
-        q1, q2CustomNeutral, q3CustomMul, q4CustomAdd, q5CustomMulAdd);
-    r.close();
-  }
-
-  // verify results are as expected.
-  private void verifyResults(float boost, IndexSearcher s, 
-      Map<Integer,Float> h1, Map<Integer,Float> h2customNeutral, Map<Integer,Float> h3CustomMul, Map<Integer,Float> h4CustomAdd, Map<Integer,Float> h5CustomMulAdd,
-      Query q1, Query q2, Query q3, Query q4, Query q5) throws Exception {
-    
-    // verify numbers of matches
-    log("#hits = "+h1.size());
-    assertEquals("queries should have same #hits",h1.size(),h2customNeutral.size());
-    assertEquals("queries should have same #hits",h1.size(),h3CustomMul.size());
-    assertEquals("queries should have same #hits",h1.size(),h4CustomAdd.size());
-    assertEquals("queries should have same #hits",h1.size(),h5CustomMulAdd.size());
-
-    QueryUtils.check(random(), q1, s, rarely());
-    QueryUtils.check(random(), q2, s, rarely());
-    QueryUtils.check(random(), q3, s, rarely());
-    QueryUtils.check(random(), q4, s, rarely());
-    QueryUtils.check(random(), q5, s, rarely());
-
-    // verify scores ratios
-    for (final Integer doc : h1.keySet()) {
-
-      log("doc = "+doc);
-
-      float fieldScore = expectedFieldScore(s.getIndexReader().document(doc).get(ID_FIELD));
-      log("fieldScore = " + fieldScore);
-      assertTrue("fieldScore should not be 0", fieldScore > 0);
-
-      float score1 = h1.get(doc);
-      logResult("score1=", s, q1, doc, score1);
-      
-      float score2 = h2customNeutral.get(doc);
-      logResult("score2=", s, q2, doc, score2);
-      assertEquals("same score (just boosted) for neutral", boost * score1, score2, Math.scalb(score2, -15));
-
-      float score3 = h3CustomMul.get(doc);
-      logResult("score3=", s, q3, doc, score3);
-      assertEquals("new score for custom mul", boost * fieldScore * score1, score3, Math.scalb(score3, -15));
-      
-      float score4 = h4CustomAdd.get(doc);
-      logResult("score4=", s, q4, doc, score4);
-      assertEquals("new score for custom add", boost * (fieldScore + score1), score4, Math.scalb(score4, -15));
-      
-      float score5 = h5CustomMulAdd.get(doc);
-      logResult("score5=", s, q5, doc, score5);
-      assertEquals("new score for custom mul add", boost * fieldScore * (score1 + fieldScore), score5, Math.scalb(score5, -15));
-    }
-  }
-
-  private void logResult(String msg, IndexSearcher s, Query q, int doc, float score1) throws IOException {
-    log(msg+" "+score1);
-    log("Explain by: "+q);
-    log(s.explain(q,doc));
-  }
-
-  // since custom scoring modifies the order of docs, map results 
-  // by doc ids so that we can later compare/verify them 
-  private Map<Integer,Float> topDocsToMap(TopDocs td) {
-    Map<Integer,Float> h = new HashMap<>();
-    for (int i=0; i<td.totalHits; i++) {
-      h.put(td.scoreDocs[i].doc, td.scoreDocs[i].score);
-    }
-    return h;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java
deleted file mode 100644
index a870366..0000000
--- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestBoostedQuery.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queries.function;
-
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.queries.function.valuesource.ConstValueSource;
-import org.apache.lucene.search.CheckHits;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LuceneTestCase;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-/**
- * Basic tests for {@link BoostedQuery}
- */
-// TODO: more tests
-public class TestBoostedQuery extends LuceneTestCase {
-  static Directory dir;
-  static IndexReader ir;
-  static IndexSearcher is;
-  
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    dir = newDirectory();
-    IndexWriterConfig iwConfig = newIndexWriterConfig(new MockAnalyzer(random()));
-    iwConfig.setMergePolicy(newLogMergePolicy());
-    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConfig);
-    Document document = new Document();
-    Field idField = new SortedDocValuesField("id", new BytesRef());
-    document.add(idField);
-    iw.addDocument(document);
-    ir = iw.getReader();
-    is = newSearcher(ir);
-    iw.close();
-  }
-  
-  @AfterClass
-  public static void afterClass() throws Exception {
-    is = null;
-    ir.close();
-    ir = null;
-    dir.close();
-    dir = null;
-  }
-  
-  public void testBasic() throws Exception {
-    Query q = new MatchAllDocsQuery();
-    TopDocs docs = is.search(q, 10);
-    assertEquals(1, docs.totalHits);
-    float score = docs.scoreDocs[0].score;
-    
-    Query boostedQ = new BoostedQuery(q, new ConstValueSource(2.0f));
-    assertHits(boostedQ, new float[] { score*2 });
-  }
-  
-  void assertHits(Query q, float scores[]) throws Exception {
-    ScoreDoc expected[] = new ScoreDoc[scores.length];
-    int expectedDocs[] = new int[scores.length];
-    for (int i = 0; i < expected.length; i++) {
-      expectedDocs[i] = i;
-      expected[i] = new ScoreDoc(i, scores[i]);
-    }
-    TopDocs docs = is.search(q, 10, 
-        new Sort(new SortField("id", SortField.Type.STRING)));
-    CheckHits.checkHits(random(), q, "", is, expectedDocs);
-    CheckHits.checkHitsQuery(q, expected, docs.scoreDocs, expectedDocs);
-    CheckHits.checkExplanations(q, "", is);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CorePlusQueriesParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CorePlusQueriesParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CorePlusQueriesParser.java
index 87b5f8f..3243928 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CorePlusQueriesParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CorePlusQueriesParser.java
@@ -19,7 +19,6 @@ package org.apache.lucene.queryparser.xml;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.queryparser.xml.builders.LikeThisQueryBuilder;
-import org.apache.lucene.queryparser.xml.builders.BoostingQueryBuilder;
 
 /**
  * Assembles a QueryBuilder which uses Query objects from
@@ -50,7 +49,6 @@ public class CorePlusQueriesParser extends CoreParser {
     super(defaultField, analyzer, parser);
     String fields[] = {"contents"};
     queryFactory.addBuilder("LikeThisQuery", new LikeThisQueryBuilder(analyzer, fields));
-    queryFactory.addBuilder("BoostingQuery", new BoostingQueryBuilder(queryFactory));
 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c27099b4/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/BoostingQueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/BoostingQueryBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/BoostingQueryBuilder.java
deleted file mode 100644
index f58804b..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/BoostingQueryBuilder.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queryparser.xml.builders;
-
-import org.apache.lucene.queries.BoostingQuery;
-import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.queryparser.xml.DOMUtils;
-import org.apache.lucene.queryparser.xml.ParserException;
-import org.apache.lucene.queryparser.xml.QueryBuilder;
-import org.w3c.dom.Element;
-/**
- * Builder for {@link BoostingQuery}
- */
-public class BoostingQueryBuilder implements QueryBuilder {
-
-  private static float DEFAULT_BOOST = 0.01f;
-  
-  private final QueryBuilder factory;
-
-  public BoostingQueryBuilder(QueryBuilder factory) {
-    this.factory = factory;
-  }
-
-  @Override
-  public Query getQuery(Element e) throws ParserException {
-    Element mainQueryElem = DOMUtils.getChildByTagOrFail(e, "Query");
-    mainQueryElem = DOMUtils.getFirstChildOrFail(mainQueryElem);
-    Query mainQuery = factory.getQuery(mainQueryElem);
-
-    Element boostQueryElem = DOMUtils.getChildByTagOrFail(e, "BoostQuery");
-    float boost = DOMUtils.getAttribute(boostQueryElem, "boost", DEFAULT_BOOST);
-    boostQueryElem = DOMUtils.getFirstChildOrFail(boostQueryElem);
-    Query boostQuery = factory.getQuery(boostQueryElem);
-
-    Query bq = new BoostingQuery(mainQuery, boostQuery, boost);
-
-    boost = DOMUtils.getAttribute(e, "boost", 1.0f);
-    if (boost != 1f) {
-      return new BoostQuery(bq, boost);
-    }
-    return bq;
-  }
-}


[33/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-5803: Add a Solr test that we reuse analysis components across fields for the same field type

Posted by da...@apache.org.
LUCENE-5803: Add a Solr test that we reuse analysis components across fields for the same field type


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

Branch: refs/heads/jira/solr-11702
Commit: 9f7f76f267bd46b0069731ba1ae4990d31c33df8
Parents: 79073fa
Author: David Smiley <ds...@apache.org>
Authored: Tue Dec 19 10:28:22 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Tue Dec 19 10:28:22 2017 -0500

----------------------------------------------------------------------
 .../miscellaneous/TestPerFieldAnalyzerWrapper.java    |  2 ++
 .../test/org/apache/solr/schema/IndexSchemaTest.java  | 14 ++++++++++++++
 2 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9f7f76f2/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java
index 903076d..54a64ce 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestPerFieldAnalyzerWrapper.java
@@ -88,7 +88,9 @@ public class TestPerFieldAnalyzerWrapper extends BaseTokenStreamTestCase {
     // test that the PerFieldWrapper returns the same instance as original Analyzer:
     ts1 = defaultAnalyzer.tokenStream("something", text);
     ts2 = wrapper1.tokenStream("something", text);
+    ts3 = wrapper1.tokenStream("somethingElse", text);
     assertSame(ts1, ts2);
+    assertSame(ts2, ts3);
 
     ts1 = specialAnalyzer.tokenStream("special", text);
     ts2 = wrapper1.tokenStream("special", text);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9f7f76f2/solr/core/src/test/org/apache/solr/schema/IndexSchemaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/IndexSchemaTest.java b/solr/core/src/test/org/apache/solr/schema/IndexSchemaTest.java
index 4719f04..6bfa14f 100644
--- a/solr/core/src/test/org/apache/solr/schema/IndexSchemaTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/IndexSchemaTest.java
@@ -17,6 +17,8 @@
 package org.apache.solr.schema;
 
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
@@ -115,4 +117,16 @@ public class IndexSchemaTest extends SolrTestCaseJ4 {
                    6, ((TrieDateField)tdatedv).getPrecisionStep());
     }
   }
+
+  @Test // LUCENE-5803
+  public void testReuseAnalysisComponents() throws Exception {
+    IndexSchema schema = h.getCore().getLatestSchema();
+    Analyzer solrAnalyzer = schema.getIndexAnalyzer();
+    // Get the tokenStream for two fields that both have the same field type (name "text")
+    TokenStream ts1 = solrAnalyzer.tokenStream("text", "foo bar"); // a non-dynamic field
+    TokenStream ts2 = solrAnalyzer.tokenStream("t_text", "whatever"); // a dynamic field
+    assertSame(ts1, ts2);
+    ts1.close();
+    ts2.close();
+  }
 }


[17/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Add OpenNLP Analysis capabilities as a module

Posted by da...@apache.org.
LUCENE-2899: Add OpenNLP Analysis capabilities as a module


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

Branch: refs/heads/jira/solr-11702
Commit: 3e2f9e62d772218bf1fcae6d58542fad3ec43742
Parents: d02d1f1
Author: Steve Rowe <sa...@apache.org>
Authored: Fri Dec 15 11:24:18 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Dec 15 11:24:18 2017 -0500

----------------------------------------------------------------------
 dev-tools/idea/.idea/ant.xml                    |    1 +
 dev-tools/idea/.idea/modules.xml                |    1 +
 dev-tools/idea/.idea/workspace.xml              |   83 +-
 .../idea/lucene/analysis/opennlp/opennlp.iml    |   30 +
 .../contrib/analysis-extras/analysis-extras.iml |    1 +
 .../lucene/analysis/opennlp/pom.xml.template    |   78 +
 .../maven/lucene/analysis/pom.xml.template      |    1 +
 lucene/CHANGES.txt                              |    9 +
 lucene/analysis/README.txt                      |    5 +
 lucene/analysis/build.xml                       |    6 +-
 .../miscellaneous/TypeAsSynonymFilter.java      |   80 +
 .../TypeAsSynonymFilterFactory.java             |   55 +
 ...ache.lucene.analysis.util.TokenFilterFactory |    1 +
 .../analysis/minhash/MinHashFilterTest.java     |    6 +-
 .../TestTypeAsSynonymFilterFactory.java         |   50 +
 lucene/analysis/opennlp/build.xml               |  118 +
 lucene/analysis/opennlp/ivy.xml                 |   29 +
 .../analysis/opennlp/OpenNLPChunkerFilter.java  |  108 +
 .../opennlp/OpenNLPChunkerFilterFactory.java    |   81 +
 .../opennlp/OpenNLPLemmatizerFilter.java        |  123 +
 .../opennlp/OpenNLPLemmatizerFilterFactory.java |   89 +
 .../analysis/opennlp/OpenNLPPOSFilter.java      |   96 +
 .../opennlp/OpenNLPPOSFilterFactory.java        |   71 +
 .../opennlp/OpenNLPSentenceBreakIterator.java   |  224 ++
 .../analysis/opennlp/OpenNLPTokenizer.java      |   98 +
 .../opennlp/OpenNLPTokenizerFactory.java        |   79 +
 .../lucene/analysis/opennlp/package-info.java   |   21 +
 .../analysis/opennlp/tools/NLPChunkerOp.java    |   41 +
 .../analysis/opennlp/tools/NLPLemmatizerOp.java |   80 +
 .../analysis/opennlp/tools/NLPNERTaggerOp.java  |   56 +
 .../analysis/opennlp/tools/NLPPOSTaggerOp.java  |   41 +
 .../opennlp/tools/NLPSentenceDetectorOp.java    |   50 +
 .../analysis/opennlp/tools/NLPTokenizerOp.java  |   48 +
 .../opennlp/tools/OpenNLPOpsFactory.java        |  176 +
 .../analysis/opennlp/tools/package-info.java    |   21 +
 lucene/analysis/opennlp/src/java/overview.html  |   61 +
 ...ache.lucene.analysis.util.TokenFilterFactory |   18 +
 ...apache.lucene.analysis.util.TokenizerFactory |   16 +
 .../lucene/analysis/opennlp/en-test-chunker.bin |  Bin 0 -> 89915 bytes
 .../lucene/analysis/opennlp/en-test-lemmas.dict |   12 +
 .../analysis/opennlp/en-test-lemmatizer.bin     |  Bin 0 -> 7370 bytes
 .../analysis/opennlp/en-test-ner-person.bin     |  Bin 0 -> 1700 bytes
 .../analysis/opennlp/en-test-pos-maxent.bin     |  Bin 0 -> 18424 bytes
 .../lucene/analysis/opennlp/en-test-sent.bin    |  Bin 0 -> 1050 bytes
 .../analysis/opennlp/en-test-tokenizer.bin      |  Bin 0 -> 15096 bytes
 .../TestOpenNLPChunkerFilterFactory.java        |   74 +
 .../TestOpenNLPLemmatizerFilterFactory.java     |  169 +
 .../opennlp/TestOpenNLPPOSFilterFactory.java    |   95 +
 .../TestOpenNLPSentenceBreakIterator.java       |  201 +
 .../opennlp/TestOpenNLPTokenizerFactory.java    |   97 +
 .../src/tools/test-model-data/README.txt        |    6 +
 .../src/tools/test-model-data/chunks.txt        | 3566 ++++++++++++++++++
 .../src/tools/test-model-data/lemmas.txt        |  875 +++++
 .../tools/test-model-data/ner_TrainerParams.txt |   21 +
 .../src/tools/test-model-data/ner_flashman.txt  |  143 +
 .../opennlp/src/tools/test-model-data/pos.txt   |   30 +
 .../src/tools/test-model-data/sentences.txt     |  144 +
 .../src/tools/test-model-data/tokenizer.txt     |   69 +
 .../apache/lucene/analysis/TestStopFilter.java  |    9 +-
 lucene/ivy-versions.properties                  |    3 +
 lucene/licenses/opennlp-maxent-3.0.3.jar.sha1   |    1 +
 lucene/licenses/opennlp-maxent-LICENSE-ASL.txt  |  202 +
 lucene/licenses/opennlp-maxent-NOTICE.txt       |    6 +
 lucene/licenses/opennlp-tools-1.8.3.jar.sha1    |    1 +
 lucene/licenses/opennlp-tools-LICENSE-ASL.txt   |  202 +
 lucene/licenses/opennlp-tools-NOTICE.txt        |    6 +
 lucene/module-build.xml                         |   22 +
 .../analysis/BaseTokenStreamTestCase.java       |   32 +-
 solr/CHANGES.txt                                |    7 +
 solr/contrib/analysis-extras/README.txt         |   10 +-
 solr/contrib/analysis-extras/build.xml          |   20 +-
 solr/contrib/analysis-extras/ivy.xml            |    3 +
 ...ractNamedEntitiesUpdateProcessorFactory.java |  571 +++
 .../apache/solr/update/processor/package.html   |   24 +
 .../collection1/conf/en-test-ner-person.bin     |  Bin 0 -> 1700 bytes
 .../solr/collection1/conf/en-test-sent.bin      |  Bin 0 -> 1050 bytes
 .../solr/collection1/conf/en-test-tokenizer.bin |  Bin 0 -> 15096 bytes
 .../collection1/conf/schema-opennlp-extract.xml |   49 +
 .../conf/solrconfig-opennlp-extract.xml         |  206 +
 .../solrconfig.snippet.randomindexconfig.xml    |   48 +
 ...ractNamedEntitiesUpdateProcessorFactory.java |  192 +
 .../processor/UpdateProcessorTestBase.java      |  168 -
 solr/licenses/opennlp-maxent-3.0.3.jar.sha1     |    1 +
 solr/licenses/opennlp-maxent-LICENSE-ASL.txt    |  202 +
 solr/licenses/opennlp-maxent-NOTICE.txt         |    6 +
 solr/licenses/opennlp-tools-1.8.3.jar.sha1      |    1 +
 solr/licenses/opennlp-tools-LICENSE-ASL.txt     |  202 +
 solr/licenses/opennlp-tools-NOTICE.txt          |    6 +
 .../solr-ref-guide/src/filter-descriptions.adoc |   32 +
 solr/solr-ref-guide/src/language-analysis.adoc  |  208 +
 solr/solr-ref-guide/src/tokenizers.adoc         |    4 +
 .../src/update-request-processors.adoc          |    6 +
 .../processor/UpdateProcessorTestBase.java      |  168 +
 93 files changed, 10040 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/idea/.idea/ant.xml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/.idea/ant.xml b/dev-tools/idea/.idea/ant.xml
index 8723e63..6c7bc8c 100644
--- a/dev-tools/idea/.idea/ant.xml
+++ b/dev-tools/idea/.idea/ant.xml
@@ -11,6 +11,7 @@
     <buildFile url="file://$PROJECT_DIR$/lucene/analysis/icu/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/analysis/kuromoji/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/analysis/morfologik/build.xml" />
+    <buildFile url="file://$PROJECT_DIR$/lucene/analysis/opennlp/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/analysis/phonetic/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/analysis/smartcn/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/analysis/stempel/build.xml" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/idea/.idea/modules.xml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/.idea/modules.xml b/dev-tools/idea/.idea/modules.xml
index 7ad2a78..4df1000 100644
--- a/dev-tools/idea/.idea/modules.xml
+++ b/dev-tools/idea/.idea/modules.xml
@@ -15,6 +15,7 @@
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/icu/icu.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/kuromoji/kuromoji.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/morfologik/morfologik.iml" />
+      <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/opennlp/opennlp.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/phonetic/phonetic.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/smartcn/smartcn.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/stempel/stempel.iml" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/idea/.idea/workspace.xml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/.idea/workspace.xml b/dev-tools/idea/.idea/workspace.xml
index e22108f..11794af 100644
--- a/dev-tools/idea/.idea/workspace.xml
+++ b/dev-tools/idea/.idea/workspace.xml
@@ -44,6 +44,14 @@
       <option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
       <patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
     </configuration>
+    <configuration default="false" name="Module analyzers-opennlp" type="JUnit" factoryName="JUnit">
+      <module name="opennlp" />
+      <option name="TEST_OBJECT" value="pattern" />
+      <option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$/idea-build/lucene/analysis/opennlp" />
+      <option name="VM_PARAMETERS" value="-ea -DtempDir=temp" />
+      <option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
+      <patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
+    </configuration>
     <configuration default="false" name="Module analyzers-phonetic" type="JUnit" factoryName="JUnit">
       <module name="phonetic" />
       <option name="TEST_OBJECT" value="pattern" />
@@ -333,48 +341,49 @@
       <patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
     </configuration>
 
-    <list size="41">
+    <list size="42">
       <item index="0" class="java.lang.String" itemvalue="JUnit.Lucene core" />
       <item index="1" class="java.lang.String" itemvalue="JUnit.Module analyzers-common" />
       <item index="2" class="java.lang.String" itemvalue="JUnit.Module analyzers-icu" />
       <item index="3" class="java.lang.String" itemvalue="JUnit.Module analyzers-kuromoji" />
       <item index="4" class="java.lang.String" itemvalue="JUnit.Module analyzers-morfologik" />
-      <item index="5" class="java.lang.String" itemvalue="JUnit.Module analyzers-phonetic" />
-      <item index="6" class="java.lang.String" itemvalue="JUnit.Module analyzers-smartcn" />
-      <item index="7" class="java.lang.String" itemvalue="JUnit.Module analyzers-stempel" />
-      <item index="8" class="java.lang.String" itemvalue="JUnit.Module analyzers-uima" />
-      <item index="9" class="java.lang.String" itemvalue="JUnit.Module backward-codecs" />
-      <item index="10" class="java.lang.String" itemvalue="JUnit.Module benchmark" />
-      <item index="11" class="java.lang.String" itemvalue="JUnit.Module classification" />
-      <item index="12" class="java.lang.String" itemvalue="JUnit.Module codecs" />
-      <item index="13" class="java.lang.String" itemvalue="JUnit.Module expressions" />
-      <item index="14" class="java.lang.String" itemvalue="JUnit.Module facet" />
-      <item index="15" class="java.lang.String" itemvalue="JUnit.Module grouping" />
-      <item index="16" class="java.lang.String" itemvalue="JUnit.Module highlighter" />
-      <item index="17" class="java.lang.String" itemvalue="JUnit.Module join" />
-      <item index="18" class="java.lang.String" itemvalue="JUnit.Module memory" />
-      <item index="19" class="java.lang.String" itemvalue="JUnit.Module misc" />
-      <item index="20" class="java.lang.String" itemvalue="JUnit.Module queries" />
-      <item index="21" class="java.lang.String" itemvalue="JUnit.Module queryparser" />
-      <item index="22" class="java.lang.String" itemvalue="JUnit.Module replicator" />
-      <item index="23" class="java.lang.String" itemvalue="JUnit.Module sandbox" />
-      <item index="24" class="java.lang.String" itemvalue="JUnit.Module spatial" />
-      <item index="25" class="java.lang.String" itemvalue="JUnit.Module spatial-extras" />
-      <item index="26" class="java.lang.String" itemvalue="JUnit.Module spatial3d" />
-      <item index="27" class="java.lang.String" itemvalue="JUnit.Module suggest" />
-      <item index="28" class="java.lang.String" itemvalue="Application.solrcloud" />
-      <item index="29" class="java.lang.String" itemvalue="JUnit.Solr core" />
-      <item index="30" class="java.lang.String" itemvalue="JUnit.Solrj" />
-      <item index="31" class="java.lang.String" itemvalue="JUnit.Solr analysis-extras contrib" />
-      <item index="32" class="java.lang.String" itemvalue="JUnit.Solr analytics contrib" />
-      <item index="33" class="java.lang.String" itemvalue="JUnit.Solr clustering contrib" />
-      <item index="34" class="java.lang.String" itemvalue="JUnit.Solr dataimporthandler contrib" />
-      <item index="35" class="java.lang.String" itemvalue="JUnit.Solr dataimporthandler-extras contrib" />
-      <item index="36" class="java.lang.String" itemvalue="JUnit.Solr extraction contrib" />
-      <item index="37" class="java.lang.String" itemvalue="JUnit.Solr langid contrib" />
-      <item index="38" class="java.lang.String" itemvalue="JUnit.Solr ltr contrib" />
-      <item index="39" class="java.lang.String" itemvalue="JUnit.Solr uima contrib" />
-      <item index="40" class="java.lang.String" itemvalue="JUnit.Solr velocity contrib" />
+      <item index="5" class="java.lang.String" itemvalue="JUnit.Module analyzers-opennlp" />
+      <item index="6" class="java.lang.String" itemvalue="JUnit.Module analyzers-phonetic" />
+      <item index="7" class="java.lang.String" itemvalue="JUnit.Module analyzers-smartcn" />
+      <item index="8" class="java.lang.String" itemvalue="JUnit.Module analyzers-stempel" />
+      <item index="9" class="java.lang.String" itemvalue="JUnit.Module analyzers-uima" />
+      <item index="10" class="java.lang.String" itemvalue="JUnit.Module backward-codecs" />
+      <item index="11" class="java.lang.String" itemvalue="JUnit.Module benchmark" />
+      <item index="12" class="java.lang.String" itemvalue="JUnit.Module classification" />
+      <item index="13" class="java.lang.String" itemvalue="JUnit.Module codecs" />
+      <item index="14" class="java.lang.String" itemvalue="JUnit.Module expressions" />
+      <item index="15" class="java.lang.String" itemvalue="JUnit.Module facet" />
+      <item index="16" class="java.lang.String" itemvalue="JUnit.Module grouping" />
+      <item index="17" class="java.lang.String" itemvalue="JUnit.Module highlighter" />
+      <item index="18" class="java.lang.String" itemvalue="JUnit.Module join" />
+      <item index="19" class="java.lang.String" itemvalue="JUnit.Module memory" />
+      <item index="20" class="java.lang.String" itemvalue="JUnit.Module misc" />
+      <item index="21" class="java.lang.String" itemvalue="JUnit.Module queries" />
+      <item index="22" class="java.lang.String" itemvalue="JUnit.Module queryparser" />
+      <item index="23" class="java.lang.String" itemvalue="JUnit.Module replicator" />
+      <item index="24" class="java.lang.String" itemvalue="JUnit.Module sandbox" />
+      <item index="25" class="java.lang.String" itemvalue="JUnit.Module spatial" />
+      <item index="26" class="java.lang.String" itemvalue="JUnit.Module spatial-extras" />
+      <item index="27" class="java.lang.String" itemvalue="JUnit.Module spatial3d" />
+      <item index="28" class="java.lang.String" itemvalue="JUnit.Module suggest" />
+      <item index="29" class="java.lang.String" itemvalue="Application.solrcloud" />
+      <item index="30" class="java.lang.String" itemvalue="JUnit.Solr core" />
+      <item index="31" class="java.lang.String" itemvalue="JUnit.Solrj" />
+      <item index="32" class="java.lang.String" itemvalue="JUnit.Solr analysis-extras contrib" />
+      <item index="33" class="java.lang.String" itemvalue="JUnit.Solr analytics contrib" />
+      <item index="34" class="java.lang.String" itemvalue="JUnit.Solr clustering contrib" />
+      <item index="35" class="java.lang.String" itemvalue="JUnit.Solr dataimporthandler contrib" />
+      <item index="36" class="java.lang.String" itemvalue="JUnit.Solr dataimporthandler-extras contrib" />
+      <item index="37" class="java.lang.String" itemvalue="JUnit.Solr extraction contrib" />
+      <item index="38" class="java.lang.String" itemvalue="JUnit.Solr langid contrib" />
+      <item index="39" class="java.lang.String" itemvalue="JUnit.Solr ltr contrib" />
+      <item index="40" class="java.lang.String" itemvalue="JUnit.Solr uima contrib" />
+      <item index="41" class="java.lang.String" itemvalue="JUnit.Solr velocity contrib" />
     </list>
   </component>
 </project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/idea/lucene/analysis/opennlp/opennlp.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/analysis/opennlp/opennlp.iml b/dev-tools/idea/lucene/analysis/opennlp/opennlp.iml
new file mode 100644
index 0000000..7725065
--- /dev/null
+++ b/dev-tools/idea/lucene/analysis/opennlp/opennlp.iml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<module type="JAVA_MODULE" version="4">
+  <component name="NewModuleRootManager" inherit-compiler-output="false">
+    <output url="file://$MODULE_DIR$/../../../idea-build/lucene/analysis/opennlp/classes/java" />
+    <output-test url="file://$MODULE_DIR$/../../../idea-build/lucene/analysis/opennlp/classes/test" />
+    <exclude-output />
+    <content url="file://$MODULE_DIR$">
+      <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
+      <sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
+      <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
+      <sourceFolder url="file://$MODULE_DIR$/src/test-files" type="java-test-resource" />
+    </content>
+    <orderEntry type="inheritedJdk" />
+    <orderEntry type="sourceFolder" forTests="false" />
+    <orderEntry type="module-library">
+      <library>
+        <CLASSES>
+          <root url="file://$MODULE_DIR$/lib" />
+        </CLASSES>
+        <JAVADOC />
+        <SOURCES />
+        <jarDirectory url="file://$MODULE_DIR$/lib" recursive="false" />
+      </library>
+    </orderEntry>
+    <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
+    <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
+    <orderEntry type="module" module-name="analysis-common" />
+    <orderEntry type="module" module-name="lucene-core" />
+  </component>
+</module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml b/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml
index 287b46a..7c0c0c1 100644
--- a/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml
+++ b/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml
@@ -37,5 +37,6 @@
     <orderEntry type="module" module-name="lucene-core" />
     <orderEntry type="module" module-name="misc" />
     <orderEntry type="module" module-name="sandbox" />
+    <orderEntry type="module" module-name="opennlp" />
   </component>
 </module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/maven/lucene/analysis/opennlp/pom.xml.template
----------------------------------------------------------------------
diff --git a/dev-tools/maven/lucene/analysis/opennlp/pom.xml.template b/dev-tools/maven/lucene/analysis/opennlp/pom.xml.template
new file mode 100644
index 0000000..4109a0a
--- /dev/null
+++ b/dev-tools/maven/lucene/analysis/opennlp/pom.xml.template
@@ -0,0 +1,78 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.lucene</groupId>
+    <artifactId>lucene-parent</artifactId>
+    <version>@version@</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <groupId>org.apache.lucene</groupId>
+  <artifactId>lucene-analyzers-opennlp</artifactId>
+  <packaging>jar</packaging>
+  <name>Lucene OpenNLP integration</name>
+  <description>
+    Lucene OpenNLP integration
+  </description>
+  <properties>
+    <module-directory>lucene/analysis/opennlp</module-directory>
+    <relative-top-level>../../../..</relative-top-level>
+    <module-path>${relative-top-level}/${module-directory}</module-path>
+  </properties>
+  <scm>
+    <connection>scm:git:${vc-anonymous-base-url}</connection>
+    <developerConnection>scm:git:${vc-dev-base-url}</developerConnection>
+    <url>${vc-browse-base-url};f=${module-directory}</url>
+  </scm>
+  <dependencies>
+    <dependency>
+      <!-- lucene-test-framework dependency must be declared before lucene-core -->
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-test-framework</artifactId>
+      <scope>test</scope>
+    </dependency>
+    @lucene-analyzers-opennlp.internal.dependencies@
+    @lucene-analyzers-opennlp.external.dependencies@
+    @lucene-analyzers-opennlp.internal.test.dependencies@
+    @lucene-analyzers-opennlp.external.test.dependencies@
+  </dependencies>
+  <build>
+    <sourceDirectory>${module-path}/src/java</sourceDirectory>
+    <testSourceDirectory>${module-path}/src/test</testSourceDirectory>
+    <resources>
+      <resource>
+        <directory>${module-path}/src/resources</directory>
+      </resource>
+    </resources>
+    <testResources>
+      <testResource>
+        <directory>${project.build.testSourceDirectory}</directory>
+        <excludes>
+          <exclude>**/*.java</exclude>
+        </excludes>
+      </testResource>
+      <testResource>
+        <directory>${module-path}/src/test-files</directory>
+      </testResource>
+    </testResources>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/dev-tools/maven/lucene/analysis/pom.xml.template
----------------------------------------------------------------------
diff --git a/dev-tools/maven/lucene/analysis/pom.xml.template b/dev-tools/maven/lucene/analysis/pom.xml.template
index 9058abf..466ad30 100644
--- a/dev-tools/maven/lucene/analysis/pom.xml.template
+++ b/dev-tools/maven/lucene/analysis/pom.xml.template
@@ -35,6 +35,7 @@
     <module>icu</module>
     <module>kuromoji</module>
     <module>morfologik</module>
+    <module>opennlp</module>
     <module>phonetic</module>
     <module>smartcn</module>
     <module>stempel</module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 0fbf446..db8aaab 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,15 @@ API Changes
 * LUCENE-8051: LevensteinDistance renamed to LevenshteinDistance.
   (Pulak Ghosh via Adrien Grand)
 
+New Features
+
+* LUCENE-2899: Add new module analysis/opennlp, with analysis components
+  to perform tokenization, part-of-speech tagging, lemmatization and phrase
+  chunking by invoking the corresponding OpenNLP tools. Named entity
+  recognition is also provided as a Solr update request processor.
+  (Lance Norskog, Grant Ingersoll, Joern Kottmann, Em, Kai Gülzau,
+  Rene Nederhand, Robert Muir, Steven Bower, Steve Rowe)
+
 Improvements
 
 * LUCENE-8081: Allow IndexWriter to opt out of flushing on indexing threads

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/README.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/README.txt b/lucene/analysis/README.txt
index 7dc7f53..c68584e 100644
--- a/lucene/analysis/README.txt
+++ b/lucene/analysis/README.txt
@@ -28,6 +28,9 @@ lucene-analyzers-kuromoji-XX.jar
 lucene-analyzers-morfologik-XX.jar
   An analyzer using the Morfologik stemming library.
 
+lucene-analyzers-opennlp-XX.jar
+  An analyzer using the OpenNLP natural-language processing library.
+
 lucene-analyzers-phonetic-XX.jar
   An add-on analysis library that provides phonetic encoders via Apache
   Commons-Codec. Note: this module depends on the commons-codec jar 
@@ -49,6 +52,7 @@ common/src/java
 icu/src/java
 kuromoji/src/java
 morfologik/src/java
+opennlp/src/java
 phonetic/src/java
 smartcn/src/java
 stempel/src/java
@@ -59,6 +63,7 @@ common/src/test
 icu/src/test
 kuromoji/src/test
 morfologik/src/test
+opennlp/src/test
 phonetic/src/test
 smartcn/src/test
 stempel/src/test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/build.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/build.xml b/lucene/analysis/build.xml
index 844f5f3..ed1566c 100644
--- a/lucene/analysis/build.xml
+++ b/lucene/analysis/build.xml
@@ -65,6 +65,10 @@
     <ant dir="morfologik" />
   </target>
 
+  <target name="opennlp">
+    <ant dir="opennlp" />
+  </target>
+
   <target name="phonetic">
     <ant dir="phonetic" />
   </target>
@@ -82,7 +86,7 @@
   </target>
 
   <target name="default" depends="compile"/>
-  <target name="compile" depends="common,icu,kuromoji,morfologik,phonetic,smartcn,stempel,uima" />
+  <target name="compile" depends="common,icu,kuromoji,morfologik,opennlp,phonetic,smartcn,stempel,uima" />
 
   <target name="clean">
     <forall-analyzers target="clean"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java
new file mode 100644
index 0000000..8269d5d
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilter.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.miscellaneous;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * Adds the {@link TypeAttribute#type()} as a synonym,
+ * i.e. another token at the same position, optionally with a specified prefix prepended.
+ */
+public final class TypeAsSynonymFilter extends TokenFilter {
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private final String prefix;
+
+  AttributeSource.State savedToken = null;
+
+
+  public TypeAsSynonymFilter(TokenStream input) {
+    this(input, null);
+  }
+
+  /**
+   * @param input input tokenstream
+   * @param prefix Prepend this string to every token type emitted as token text.
+   *               If null, nothing will be prepended.
+   */
+  public TypeAsSynonymFilter(TokenStream input, String prefix) {
+    super(input);
+    this.prefix = prefix;
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (savedToken != null) {         // Emit last token's type at the same position
+      restoreState(savedToken);
+      savedToken = null;
+      termAtt.setEmpty();
+      if (prefix != null) {
+        termAtt.append(prefix);
+      }
+      termAtt.append(typeAtt.type());
+      posIncrAtt.setPositionIncrement(0);
+      return true;
+    } else if (input.incrementToken()) { // Ho pending token type to emit
+      savedToken = captureState();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    savedToken = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
new file mode 100644
index 0000000..69708b7
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/TypeAsSynonymFilterFactory.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.miscellaneous;
+
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link TypeAsSynonymFilter}.
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_type_as_synonym" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.UAX29URLEmailTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.TypeAsSynonymFilterFactory" prefix="_type_" /&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ *
+ * <p>
+ * If the optional {@code prefix} parameter is used, the specified value will be prepended
+ * to the type, e.g. with prefix="_type_", for a token "example.com" with type "&lt;URL&gt;",
+ * the emitted synonym will have text "_type_&lt;URL&gt;".
+ */
+public class TypeAsSynonymFilterFactory extends TokenFilterFactory {
+  private final String prefix;
+
+  public TypeAsSynonymFilterFactory(Map<String,String> args) {
+    super(args);
+    prefix = get(args, "prefix");  // default value is null
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new TypeAsSynonymFilter(input, prefix);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
index d871ad6..6dcc81c 100644
--- a/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
+++ b/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -80,6 +80,7 @@ org.apache.lucene.analysis.miscellaneous.RemoveDuplicatesTokenFilterFactory
 org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilterFactory
 org.apache.lucene.analysis.miscellaneous.TrimFilterFactory
 org.apache.lucene.analysis.miscellaneous.TruncateTokenFilterFactory
+org.apache.lucene.analysis.miscellaneous.TypeAsSynonymFilterFactory
 org.apache.lucene.analysis.miscellaneous.WordDelimiterFilterFactory
 org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilterFactory
 org.apache.lucene.analysis.miscellaneous.ScandinavianFoldingFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/common/src/test/org/apache/lucene/analysis/minhash/MinHashFilterTest.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/minhash/MinHashFilterTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/minhash/MinHashFilterTest.java
index a4080fe..1bc6ed7 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/minhash/MinHashFilterTest.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/minhash/MinHashFilterTest.java
@@ -183,14 +183,14 @@ public class MinHashFilterTest extends BaseTokenStreamTestCase {
     TokenStream ts = createTokenStream(5, "woof woof woof woof woof", 1, 1, 100, false);
     assertTokenStreamContents(ts, hashes, new int[]{0},
         new int[]{24}, new String[]{MinHashFilter.MIN_HASH_TYPE}, new int[]{1}, new int[]{1}, 24, 0, null,
-        true);
+        true, null);
 
     ts = createTokenStream(5, "woof woof woof woof woof", 2, 1, 1, false);
     assertTokenStreamContents(ts, new String[]{new String(new char[]{0, 0, 8449, 54077, 64133, 32857, 8605, 41409}),
             new String(new char[]{0, 1, 16887, 58164, 39536, 14926, 6529, 17276})}, new int[]{0, 0},
         new int[]{24, 24}, new String[]{MinHashFilter.MIN_HASH_TYPE, MinHashFilter.MIN_HASH_TYPE}, new int[]{1, 0},
         new int[]{1, 1}, 24, 0, null,
-        true);
+        true, null);
   }
 
   @Test
@@ -203,7 +203,7 @@ public class MinHashFilterTest extends BaseTokenStreamTestCase {
         false);
     assertTokenStreamContents(ts, hashes, new int[]{0, 0},
         new int[]{49, 49}, new String[]{MinHashFilter.MIN_HASH_TYPE, MinHashFilter.MIN_HASH_TYPE}, new int[]{1, 0},
-        new int[]{1, 1}, 49, 0, null, true);
+        new int[]{1, 1}, 49, 0, null, true, null);
   }
 
   private ArrayList<String> getTokens(TokenStream ts) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java
new file mode 100644
index 0000000..6beb139
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestTypeAsSynonymFilterFactory.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.miscellaneous;
+
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
+
+public class TestTypeAsSynonymFilterFactory extends BaseTokenStreamFactoryTestCase {
+
+  private static final Token[] TOKENS =  { token("Visit", "<ALPHANUM>"), token("example.com", "<URL>") };
+
+  public void testBasic() throws Exception {
+    TokenStream stream = new CannedTokenStream(TOKENS);
+    stream = tokenFilterFactory("TypeAsSynonym").create(stream);
+    assertTokenStreamContents(stream, new String[] { "Visit", "<ALPHANUM>", "example.com", "<URL>" },
+        null, null, new String[] { "<ALPHANUM>", "<ALPHANUM>", "<URL>", "<URL>" }, new int[] { 1, 0, 1, 0 });
+  }
+
+  public void testPrefix() throws Exception {
+    TokenStream stream = new CannedTokenStream(TOKENS);
+    stream = tokenFilterFactory("TypeAsSynonym", "prefix", "_type_").create(stream);
+    assertTokenStreamContents(stream, new String[] { "Visit", "_type_<ALPHANUM>", "example.com", "_type_<URL>" },
+        null, null, new String[] { "<ALPHANUM>", "<ALPHANUM>", "<URL>", "<URL>" }, new int[] { 1, 0, 1, 0 });
+  }
+
+  private static Token token(String term, String type) {
+    Token token = new Token();
+    token.setEmpty();
+    token.append(term);
+    token.setType(type);
+    return token;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/build.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/build.xml b/lucene/analysis/opennlp/build.xml
new file mode 100644
index 0000000..e2cd20a
--- /dev/null
+++ b/lucene/analysis/opennlp/build.xml
@@ -0,0 +1,118 @@
+<?xml version="1.0"?>
+
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+ -->
+
+<project name="analyzers-opennlp" default="default">
+
+  <description>
+    OpenNLP Library Integration
+  </description>
+
+  <path id="opennlpjars">
+    <fileset dir="lib"/>
+  </path>
+
+  <property name="test.model.data.dir" location="src/tools/test-model-data"/>
+  <property name="tests.userdir" location="src/test-files"/>
+  <property name="test.model.dir" location="${tests.userdir}/org/apache/lucene/analysis/opennlp"/>
+
+  <import file="../analysis-module-build.xml"/>
+
+  <property name="analysis-extras.conf.dir"
+            location="${common.dir}/../solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf"/>
+
+  <path id="classpath">
+    <pathelement path="${analyzers-common.jar}"/>
+    <path refid="opennlpjars"/>
+    <path refid="base.classpath"/>
+  </path>
+
+  <path id="test.classpath">
+    <path refid="test.base.classpath"/>
+    <pathelement path="${tests.userdir}"/>
+  </path>
+
+  <target name="compile-core" depends="jar-analyzers-common, common.compile-core" />
+
+  <!--
+    This does not create real NLP models, just small unencumbered ones for the unit tests.
+    All text taken from reuters corpus.
+    Tags applied with online demos at CCG Urbana-Champaign.
+    -->
+  <target name="train-test-models" description="Train all small test models for unit tests" depends="resolve">
+    <mkdir dir="${test.model.dir}"/>
+    <!-- https://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.sentdetect.training -->
+    <trainModel command="SentenceDetectorTrainer" lang="en" data="sentences.txt" model="en-test-sent.bin"/>
+    <copy file="${test.model.dir}/en-test-sent.bin" todir="${analysis-extras.conf.dir}"/>
+
+    <!-- https://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.tokenizer.training -->
+    <trainModel command="TokenizerTrainer" lang="en" data="tokenizer.txt" model="en-test-tokenizer.bin"/>
+    <copy file="${test.model.dir}/en-test-tokenizer.bin" todir="${analysis-extras.conf.dir}"/>
+
+    <!-- https://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.postagger.training -->
+    <trainModel command="POSTaggerTrainer" lang="en" data="pos.txt" model="en-test-pos-maxent.bin"/>
+
+    <!-- https://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.chunker.training -->
+    <trainModel command="ChunkerTrainerME" lang="en" data="chunks.txt" model="en-test-chunker.bin"/>
+
+    <!-- https://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.namefind.training -->
+    <trainModel command="TokenNameFinderTrainer" lang="en" data="ner_flashman.txt" model="en-test-ner-person.bin">
+      <extra-args>
+        <arg value="-params"/>
+        <arg value="ner_TrainerParams.txt"/>
+      </extra-args>
+    </trainModel>
+    <copy file="${test.model.dir}/en-test-ner-person.bin" todir="${analysis-extras.conf.dir}"/>
+
+    <!-- https://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.lemmatizer.training -->
+    <trainModel command="LemmatizerTrainerME" lang="en" data="lemmas.txt" model="en-test-lemmatizer.bin"/>
+  </target>
+
+  <macrodef name="trainModel">
+    <attribute name="command"/>
+    <attribute name="lang"/>
+    <attribute name="data"/>
+    <attribute name="model"/>
+    <element name="extra-args" optional="true"/>
+    <sequential>
+      <java classname="opennlp.tools.cmdline.CLI"
+            dir="${test.model.data.dir}"
+            fork="true"
+            failonerror="true">
+        <classpath>
+          <path refid="opennlpjars"/>
+        </classpath>
+
+        <arg value="@{command}"/>
+
+        <arg value="-lang"/>
+        <arg value="@{lang}"/>
+
+        <arg value="-data"/>
+        <arg value="@{data}"/>
+
+        <arg value="-model"/>
+        <arg value="${test.model.dir}/@{model}"/>
+
+        <extra-args/>
+      </java>
+    </sequential>
+  </macrodef>
+
+  <target name="regenerate" depends="train-test-models"/>
+</project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/ivy.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/ivy.xml b/lucene/analysis/opennlp/ivy.xml
new file mode 100644
index 0000000..c7b885f
--- /dev/null
+++ b/lucene/analysis/opennlp/ivy.xml
@@ -0,0 +1,29 @@
+<!--
+   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.
+-->
+<ivy-module version="2.0">
+  <info organisation="org.apache.lucene" module="analyzers-opennlp" />
+  <configurations defaultconfmapping="compile->master">
+    <conf name="compile" transitive="false"/>
+  </configurations>
+  <dependencies>
+    <dependency org="org.apache.opennlp" name="opennlp-tools" rev="${/org.apache.opennlp/opennlp-tools}" transitive="false" conf="compile" />
+    <dependency org="org.apache.opennlp" name="opennlp-maxent" rev="${/org.apache.opennlp/opennlp-maxent}" transitive="false" conf="compile" />
+    <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}" />
+  </dependencies>
+</ivy-module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilter.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilter.java
new file mode 100644
index 0000000..cfc47e6
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilter.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.tools.NLPChunkerOp;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * Run OpenNLP chunker.  Prerequisite: the OpenNLPTokenizer and OpenNLPPOSFilter must precede this filter.
+ * Tags terms in the TypeAttribute, replacing the POS tags previously put there by OpenNLPPOSFilter.
+ */
+public final class OpenNLPChunkerFilter extends TokenFilter {
+
+  private List<AttributeSource> sentenceTokenAttrs = new ArrayList<>();
+  private int tokenNum = 0;
+  private boolean moreTokensAvailable = true;
+  private String[] sentenceTerms = null;
+  private String[] sentenceTermPOSTags = null;
+
+  private final NLPChunkerOp chunkerOp;
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  private final FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.class);
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+  public OpenNLPChunkerFilter(TokenStream input, NLPChunkerOp chunkerOp) {
+    super(input);
+    this.chunkerOp = chunkerOp;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if ( ! moreTokensAvailable) {
+      clear();
+      return false;
+    }
+    if (tokenNum == sentenceTokenAttrs.size()) {
+      nextSentence();
+      if (sentenceTerms == null) {
+        clear();
+        return false;
+      }
+      assignTokenTypes(chunkerOp.getChunks(sentenceTerms, sentenceTermPOSTags, null));
+      tokenNum = 0;
+    }
+    clearAttributes();
+    sentenceTokenAttrs.get(tokenNum++).copyTo(this);
+    return true;
+  }
+
+  private void nextSentence() throws IOException {
+    List<String> termList = new ArrayList<>();
+    List<String> posTagList = new ArrayList<>();
+    sentenceTokenAttrs.clear();
+    boolean endOfSentence = false;
+    while ( ! endOfSentence && (moreTokensAvailable = input.incrementToken())) {
+      termList.add(termAtt.toString());
+      posTagList.add(typeAtt.type());
+      endOfSentence = 0 != (flagsAtt.getFlags() & OpenNLPTokenizer.EOS_FLAG_BIT);
+      sentenceTokenAttrs.add(input.cloneAttributes());
+    }
+    sentenceTerms = termList.size() > 0 ? termList.toArray(new String[termList.size()]) : null;
+    sentenceTermPOSTags = posTagList.size() > 0 ? posTagList.toArray(new String[posTagList.size()]) : null;
+  }
+
+  private void assignTokenTypes(String[] tags) {
+    for (int i = 0 ; i < tags.length ; ++i) {
+      sentenceTokenAttrs.get(i).getAttribute(TypeAttribute.class).setType(tags[i]);
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    moreTokensAvailable = true;
+    clear();
+  }
+
+  private void clear() {
+    sentenceTokenAttrs.clear();
+    sentenceTerms = null;
+    sentenceTermPOSTags = null;
+    tokenNum = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilterFactory.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilterFactory.java
new file mode 100644
index 0000000..96eb672
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPChunkerFilterFactory.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.tools.NLPChunkerOp;
+import org.apache.lucene.analysis.opennlp.tools.OpenNLPOpsFactory;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link OpenNLPChunkerFilter}.
+ *
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_opennlp_chunked" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.OpenNLPTokenizerFactory" sentenceModel="filename" tokenizerModel="filename"/&gt;
+ *     &lt;filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="filename"/&gt;
+ *     &lt;filter class="solr.OpenNLPChunkerFilterFactory" chunkerModel="filename"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ * @since 7.3.0
+ */
+public class OpenNLPChunkerFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+  public static final String CHUNKER_MODEL = "chunkerModel";
+
+  private final String chunkerModelFile;
+
+  public OpenNLPChunkerFilterFactory(Map<String,String> args) {
+    super(args);
+    chunkerModelFile = get(args, CHUNKER_MODEL);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public OpenNLPChunkerFilter create(TokenStream in) {
+    try {
+      NLPChunkerOp chunkerOp = null;
+
+      if (chunkerModelFile != null) {
+        chunkerOp = OpenNLPOpsFactory.getChunker(chunkerModelFile);
+      }
+      return new OpenNLPChunkerFilter(in, chunkerOp);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) {
+    try {
+      // load and register read-only models in cache with file/resource names
+      if (chunkerModelFile != null) {
+        OpenNLPOpsFactory.getChunkerModel(chunkerModelFile, loader);
+      }
+    } catch (IOException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilter.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilter.java
new file mode 100644
index 0000000..4c484b9
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilter.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.tools.NLPLemmatizerOp;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * <p>Runs OpenNLP dictionary-based and/or MaxEnt lemmatizers.</p>
+ * <p>
+ *   Both a dictionary-based lemmatizer and a MaxEnt lemmatizer are supported,
+ *   via the "dictionary" and "lemmatizerModel" params, respectively.
+ *   If both are configured, the dictionary-based lemmatizer is tried first,
+ *   and then the MaxEnt lemmatizer is consulted for out-of-vocabulary tokens.
+ * </p>
+ * <p>
+ *   The dictionary file must be encoded as UTF-8, with one entry per line,
+ *   in the form <tt>word[tab]lemma[tab]part-of-speech</tt>
+ * </p>
+ */
+public class OpenNLPLemmatizerFilter extends TokenFilter {
+  private final NLPLemmatizerOp lemmatizerOp;
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  private final KeywordAttribute keywordAtt = addAttribute(KeywordAttribute.class);
+  private final FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.class);
+  private List<AttributeSource> sentenceTokenAttrs = new ArrayList<>();
+  private Iterator<AttributeSource> sentenceTokenAttrsIter = null;
+  private boolean moreTokensAvailable = true;
+  private String[] sentenceTokens = null;     // non-keyword tokens
+  private String[] sentenceTokenTypes = null; // types for non-keyword tokens
+  private String[] lemmas = null;             // lemmas for non-keyword tokens
+  private int lemmaNum = 0;                   // lemma counter
+
+  public OpenNLPLemmatizerFilter(TokenStream input, NLPLemmatizerOp lemmatizerOp) {
+    super(input);
+    this.lemmatizerOp = lemmatizerOp;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if ( ! moreTokensAvailable) {
+      clear();
+      return false;
+    }
+    if (sentenceTokenAttrsIter == null || ! sentenceTokenAttrsIter.hasNext()) {
+      nextSentence();
+      if (sentenceTokens == null) { // zero non-keyword tokens
+        clear();
+        return false;
+      }
+      lemmas = lemmatizerOp.lemmatize(sentenceTokens, sentenceTokenTypes);
+      lemmaNum = 0;
+      sentenceTokenAttrsIter = sentenceTokenAttrs.iterator();
+    }
+    clearAttributes();
+    sentenceTokenAttrsIter.next().copyTo(this);
+    if ( ! keywordAtt.isKeyword()) {
+      termAtt.setEmpty().append(lemmas[lemmaNum++]);
+    }
+    return true;
+
+  }
+
+  private void nextSentence() throws IOException {
+    List<String> tokenList = new ArrayList<>();
+    List<String> typeList = new ArrayList<>();
+    sentenceTokenAttrs.clear();
+    boolean endOfSentence = false;
+    while ( ! endOfSentence && (moreTokensAvailable = input.incrementToken())) {
+      if ( ! keywordAtt.isKeyword()) {
+        tokenList.add(termAtt.toString());
+        typeList.add(typeAtt.type());
+      }
+      endOfSentence = 0 != (flagsAtt.getFlags() & OpenNLPTokenizer.EOS_FLAG_BIT);
+      sentenceTokenAttrs.add(input.cloneAttributes());
+    }
+    sentenceTokens = tokenList.size() > 0 ? tokenList.toArray(new String[tokenList.size()]) : null;
+    sentenceTokenTypes = typeList.size() > 0 ? typeList.toArray(new String[typeList.size()]) : null;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    moreTokensAvailable = true;
+    clear();
+  }
+
+  private void clear() {
+    sentenceTokenAttrs.clear();
+    sentenceTokenAttrsIter = null;
+    sentenceTokens = null;
+    sentenceTokenTypes = null;
+    lemmas = null;
+    lemmaNum = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilterFactory.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilterFactory.java
new file mode 100644
index 0000000..90a0e43
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPLemmatizerFilterFactory.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.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.tools.NLPLemmatizerOp;
+import org.apache.lucene.analysis.opennlp.tools.OpenNLPOpsFactory;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link OpenNLPLemmatizerFilter}.
+ *
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_opennlp_lemma" class="solr.TextField" positionIncrementGap="100"
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.OpenNLPTokenizerFactory"
+ *                sentenceModel="filename"
+ *                tokenizerModel="filename"/&gt;
+ *     /&gt;
+ *     &lt;filter class="solr.OpenNLPLemmatizerFilterFactory"
+ *             dictionary="filename"
+ *             lemmatizerModel="filename"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ * @since 7.3.0
+ */
+public class OpenNLPLemmatizerFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+  public static final String DICTIONARY = "dictionary";
+  public static final String LEMMATIZER_MODEL = "lemmatizerModel";
+
+  private final String dictionaryFile;
+  private final String lemmatizerModelFile;
+
+  public OpenNLPLemmatizerFilterFactory(Map<String,String> args) {
+    super(args);
+    dictionaryFile = get(args, DICTIONARY);
+    lemmatizerModelFile = get(args, LEMMATIZER_MODEL);
+
+    if (dictionaryFile == null && lemmatizerModelFile == null) {
+      throw new IllegalArgumentException("Configuration Error: missing parameter: at least one of '"
+          + DICTIONARY + "' and '" + LEMMATIZER_MODEL + "' must be provided.");
+    }
+
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public OpenNLPLemmatizerFilter create(TokenStream in) {
+    try {
+      NLPLemmatizerOp lemmatizerOp = OpenNLPOpsFactory.getLemmatizer(dictionaryFile, lemmatizerModelFile);
+      return new OpenNLPLemmatizerFilter(in, lemmatizerOp);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    // register models in cache with file/resource names
+    if (dictionaryFile != null) {
+      OpenNLPOpsFactory.getLemmatizerDictionary(dictionaryFile, loader);
+    }
+    if (lemmatizerModelFile != null) {
+      OpenNLPOpsFactory.getLemmatizerModel(lemmatizerModelFile, loader);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
new file mode 100644
index 0000000..a5bea28
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilter.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.tools.NLPPOSTaggerOp;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * Run OpenNLP POS tagger.  Tags all terms in the TypeAttribute.
+ */
+public final class OpenNLPPOSFilter extends TokenFilter {
+
+  private List<AttributeSource> sentenceTokenAttrs = new ArrayList<>();
+  String[] tags = null;
+  private int tokenNum = 0;
+  private boolean moreTokensAvailable = true;
+
+  private final NLPPOSTaggerOp posTaggerOp;
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  private final FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.class);
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+  public OpenNLPPOSFilter(TokenStream input, NLPPOSTaggerOp posTaggerOp) {
+    super(input);
+    this.posTaggerOp = posTaggerOp;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if ( ! moreTokensAvailable) {
+      clear();
+      return false;
+    }
+    if (tokenNum == sentenceTokenAttrs.size()) { // beginning of stream, or previous sentence exhausted
+      String[] sentenceTokens = nextSentence();
+      if (sentenceTokens == null) {
+        clear();
+        return false;
+      }
+      tags = posTaggerOp.getPOSTags(sentenceTokens);
+      tokenNum = 0;
+    }
+    clearAttributes();
+    sentenceTokenAttrs.get(tokenNum).copyTo(this);
+    typeAtt.setType(tags[tokenNum++]);
+    return true;
+  }
+
+  private String[] nextSentence() throws IOException {
+    List<String> termList = new ArrayList<>();
+    sentenceTokenAttrs.clear();
+    boolean endOfSentence = false;
+    while ( ! endOfSentence && (moreTokensAvailable = input.incrementToken())) {
+      termList.add(termAtt.toString());
+      endOfSentence = 0 != (flagsAtt.getFlags() & OpenNLPTokenizer.EOS_FLAG_BIT);
+      sentenceTokenAttrs.add(input.cloneAttributes());
+    }
+    return termList.size() > 0 ? termList.toArray(new String[termList.size()]) : null;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    moreTokensAvailable = true;
+  }
+
+  private void clear() {
+    sentenceTokenAttrs.clear();
+    tags = null;
+    tokenNum = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilterFactory.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilterFactory.java
new file mode 100644
index 0000000..952218f
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPPOSFilterFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.tools.OpenNLPOpsFactory;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link OpenNLPPOSFilter}.
+ *
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_opennlp_pos" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.OpenNLPTokenizerFactory" sentenceModel="filename" tokenizerModel="filename"/&gt;
+ *     &lt;filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="filename"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ * @since 7.3.0
+ */
+public class OpenNLPPOSFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+  public static final String POS_TAGGER_MODEL = "posTaggerModel";
+
+  private final String posTaggerModelFile;
+
+  public OpenNLPPOSFilterFactory(Map<String,String> args) {
+    super(args);
+    posTaggerModelFile = require(args, POS_TAGGER_MODEL);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public OpenNLPPOSFilter create(TokenStream in) {
+    try {
+      return new OpenNLPPOSFilter(in, OpenNLPOpsFactory.getPOSTagger(posTaggerModelFile));
+    } catch (IOException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) {
+    try { // load and register the read-only model in cache with file/resource name
+      OpenNLPOpsFactory.getPOSTaggerModel(posTaggerModelFile, loader);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPSentenceBreakIterator.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPSentenceBreakIterator.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPSentenceBreakIterator.java
new file mode 100644
index 0000000..f69fbc6
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPSentenceBreakIterator.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.text.BreakIterator;
+import java.text.CharacterIterator;
+
+import opennlp.tools.util.Span;
+import org.apache.lucene.analysis.opennlp.tools.NLPSentenceDetectorOp;
+import org.apache.lucene.analysis.util.CharArrayIterator;
+
+/**
+ * A {@link BreakIterator} that splits sentences using an OpenNLP sentence chunking model.
+ */
+public final class OpenNLPSentenceBreakIterator extends BreakIterator {
+
+  private CharacterIterator text;
+  private int currentSentence;
+  private int[] sentenceStarts;
+  private NLPSentenceDetectorOp sentenceOp;
+
+  public OpenNLPSentenceBreakIterator(NLPSentenceDetectorOp sentenceOp) {
+    this.sentenceOp = sentenceOp;
+  }
+
+  @Override
+  public int current() {
+    return text.getIndex();
+  }
+
+  @Override
+  public int first() {
+    currentSentence = 0;
+    text.setIndex(text.getBeginIndex());
+    return current();
+  }
+
+  @Override
+  public int last() {
+    if (sentenceStarts.length > 0) {
+      currentSentence = sentenceStarts.length - 1;
+      text.setIndex(text.getEndIndex());
+    } else { // there are no sentences; both the first and last positions are the begin index
+      currentSentence = 0;
+      text.setIndex(text.getBeginIndex());
+    }
+    return current();
+  }
+
+  @Override
+  public int next() {
+    if (text.getIndex() == text.getEndIndex() || 0 == sentenceStarts.length) {
+      return DONE;
+    } else if (currentSentence < sentenceStarts.length - 1) {
+      text.setIndex(sentenceStarts[++currentSentence]);
+      return current();
+    } else {
+      return last();
+    }
+  }
+
+  @Override
+  public int following(int pos) {
+    if (pos < text.getBeginIndex() || pos > text.getEndIndex()) {
+      throw new IllegalArgumentException("offset out of bounds");
+    } else if (0 == sentenceStarts.length) {
+      text.setIndex(text.getBeginIndex());
+      return DONE;
+    } else if (pos >= sentenceStarts[sentenceStarts.length - 1]) {
+      // this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something)
+      // https://bugs.openjdk.java.net/browse/JDK-8015110
+      text.setIndex(text.getEndIndex());
+      currentSentence = sentenceStarts.length - 1;
+      return DONE;
+    } else { // there are at least two sentences
+      currentSentence = (sentenceStarts.length - 1) / 2; // start search from the middle
+      moveToSentenceAt(pos, 0, sentenceStarts.length - 2);
+      text.setIndex(sentenceStarts[++currentSentence]);
+      return current();
+    }
+  }
+
+  /** Binary search over sentences */
+  private void moveToSentenceAt(int pos, int minSentence, int maxSentence) {
+    if (minSentence != maxSentence) {
+      if (pos < sentenceStarts[currentSentence]) {
+        int newMaxSentence = currentSentence - 1;
+        currentSentence = minSentence + (currentSentence - minSentence) / 2;
+        moveToSentenceAt(pos, minSentence, newMaxSentence);
+      } else if (pos >= sentenceStarts[currentSentence + 1]) {
+        int newMinSentence = currentSentence + 1;
+        currentSentence = maxSentence - (maxSentence - currentSentence) / 2;
+        moveToSentenceAt(pos, newMinSentence, maxSentence);
+      }
+    } else {
+      assert currentSentence == minSentence;
+      assert pos >= sentenceStarts[currentSentence];
+      assert (currentSentence == sentenceStarts.length - 1 && pos <= text.getEndIndex())
+          || pos < sentenceStarts[currentSentence + 1];
+    }
+    // we have arrived - nothing to do
+  }
+
+  @Override
+  public int previous() {
+    if (text.getIndex() == text.getBeginIndex()) {
+      return DONE;
+    } else {
+      if (0 == sentenceStarts.length) {
+        text.setIndex(text.getBeginIndex());
+        return DONE;
+      }
+      if (text.getIndex() == text.getEndIndex()) {
+        text.setIndex(sentenceStarts[currentSentence]);
+      } else {
+        text.setIndex(sentenceStarts[--currentSentence]);
+      }
+      return current();
+    }
+  }
+
+  @Override
+  public int preceding(int pos) {
+    if (pos < text.getBeginIndex() || pos > text.getEndIndex()) {
+      throw new IllegalArgumentException("offset out of bounds");
+    } else if (0 == sentenceStarts.length) {
+      text.setIndex(text.getBeginIndex());
+      currentSentence = 0;
+      return DONE;
+    } else if (pos < sentenceStarts[0]) {
+      // this conflicts with the javadocs, but matches actual behavior (Oracle has a bug in something)
+      // https://bugs.openjdk.java.net/browse/JDK-8015110
+      text.setIndex(text.getBeginIndex());
+      currentSentence = 0;
+      return DONE;
+    } else {
+      currentSentence = sentenceStarts.length / 2; // start search from the middle
+      moveToSentenceAt(pos, 0, sentenceStarts.length - 1);
+      if (0 == currentSentence) {
+        text.setIndex(text.getBeginIndex());
+        return DONE;
+      } else {
+        text.setIndex(sentenceStarts[--currentSentence]);
+        return current();
+      }
+    }
+  }
+
+  @Override
+  public int next(int n) {
+    currentSentence += n;
+    if (n < 0) {
+      if (text.getIndex() == text.getEndIndex()) {
+        ++currentSentence;
+      }
+      if (currentSentence < 0) {
+        currentSentence = 0;
+        text.setIndex(text.getBeginIndex());
+        return DONE;
+      } else {
+        text.setIndex(sentenceStarts[currentSentence]);
+      }
+    } else if (n > 0) {
+      if (currentSentence >= sentenceStarts.length) {
+        currentSentence = sentenceStarts.length - 1;
+        text.setIndex(text.getEndIndex());
+        return DONE;
+      } else {
+        text.setIndex(sentenceStarts[currentSentence]);
+      }
+    }
+    return current();
+  }
+
+  @Override
+  public CharacterIterator getText() {
+    return text;
+  }
+
+  @Override
+  public void setText(CharacterIterator newText) {
+    text = newText;
+    text.setIndex(text.getBeginIndex());
+    currentSentence = 0;
+    Span[] spans = sentenceOp.splitSentences(characterIteratorToString());
+    sentenceStarts = new int[spans.length];
+    for (int i = 0; i < spans.length; ++i) {
+      // Adjust start positions to match those of the passed-in CharacterIterator
+      sentenceStarts[i] = spans[i].getStart() + text.getBeginIndex();
+    }
+  }
+
+  private String characterIteratorToString() {
+    String fullText;
+    if (text instanceof CharArrayIterator) {
+      CharArrayIterator charArrayIterator = (CharArrayIterator)text;
+      fullText = new String(charArrayIterator.getText(), charArrayIterator.getStart(), charArrayIterator.getLength());
+    } else {
+      // TODO: is there a better way to extract full text from arbitrary CharacterIterators?
+      StringBuilder builder = new StringBuilder();
+      for (char ch = text.first(); ch != CharacterIterator.DONE; ch = text.next()) {
+        builder.append(ch);
+      }
+      fullText = builder.toString();
+      text.setIndex(text.getBeginIndex());
+    }
+    return fullText;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizer.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizer.java
new file mode 100644
index 0000000..75a3b81
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizer.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+
+import opennlp.tools.util.Span;
+
+import org.apache.lucene.analysis.opennlp.tools.NLPSentenceDetectorOp;
+import org.apache.lucene.analysis.opennlp.tools.NLPTokenizerOp;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
+import org.apache.lucene.util.AttributeFactory;
+
+/**
+ * Run OpenNLP SentenceDetector and Tokenizer.
+ * The last token in each sentence is marked by setting the {@link #EOS_FLAG_BIT} in the FlagsAttribute;
+ * following filters can use this information to apply operations to tokens one sentence at a time.
+ */
+public final class OpenNLPTokenizer extends SegmentingTokenizerBase {
+  public static int EOS_FLAG_BIT = 1;
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final FlagsAttribute flagsAtt = addAttribute(FlagsAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  private Span[] termSpans = null;
+  private int termNum = 0;
+  private int sentenceStart = 0;
+
+  private NLPSentenceDetectorOp sentenceOp = null;
+  private NLPTokenizerOp tokenizerOp = null;
+
+  public OpenNLPTokenizer(AttributeFactory factory, NLPSentenceDetectorOp sentenceOp, NLPTokenizerOp tokenizerOp) throws IOException {
+    super(factory, new OpenNLPSentenceBreakIterator(sentenceOp));
+    if (sentenceOp == null || tokenizerOp == null) {
+      throw new IllegalArgumentException("OpenNLPTokenizer: both a Sentence Detector and a Tokenizer are required");
+    }
+    this.sentenceOp = sentenceOp;
+    this.tokenizerOp = tokenizerOp;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    termSpans = null;
+    termNum = sentenceStart = 0;
+  };
+
+  @Override
+  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+    this.sentenceStart = sentenceStart;
+    String sentenceText = new String(buffer, sentenceStart, sentenceEnd - sentenceStart);
+    termSpans = tokenizerOp.getTerms(sentenceText);
+    termNum = 0;
+  }
+
+  @Override
+  protected boolean incrementWord() {
+    if (termSpans == null || termNum == termSpans.length) {
+      return false;
+    }
+    clearAttributes();
+    Span term = termSpans[termNum];
+    termAtt.copyBuffer(buffer, sentenceStart + term.getStart(), term.length());
+    offsetAtt.setOffset(correctOffset(offset + sentenceStart + term.getStart()),
+                        correctOffset(offset + sentenceStart + term.getEnd()));
+    if (termNum == termSpans.length - 1) {
+      flagsAtt.setFlags(flagsAtt.getFlags() | EOS_FLAG_BIT); // mark the last token in the sentence with EOS_FLAG_BIT
+    }
+    ++termNum;
+    return true;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    termSpans = null;
+    termNum = sentenceStart = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizerFactory.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizerFactory.java
new file mode 100644
index 0000000..a60f23f
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/OpenNLPTokenizerFactory.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.analysis.opennlp.tools.NLPSentenceDetectorOp;
+import org.apache.lucene.analysis.opennlp.tools.NLPTokenizerOp;
+import org.apache.lucene.analysis.opennlp.tools.OpenNLPOpsFactory;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.analysis.util.TokenizerFactory;
+import org.apache.lucene.util.AttributeFactory;
+
+/**
+ * Factory for {@link OpenNLPTokenizer}.
+ *
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_opennlp" class="solr.TextField" positionIncrementGap="100"
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.OpenNLPTokenizerFactory" sentenceModel="filename" tokenizerModel="filename"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ * @since 7.3.0
+ */
+public class OpenNLPTokenizerFactory extends TokenizerFactory implements ResourceLoaderAware {
+  public static final String SENTENCE_MODEL = "sentenceModel";
+  public static final String TOKENIZER_MODEL = "tokenizerModel";
+
+  private final String sentenceModelFile;
+  private final String tokenizerModelFile;
+
+  public OpenNLPTokenizerFactory(Map<String,String> args) {
+    super(args);
+    sentenceModelFile = require(args, SENTENCE_MODEL);
+    tokenizerModelFile = require(args, TOKENIZER_MODEL);
+    if ( ! args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public OpenNLPTokenizer create(AttributeFactory factory) {
+    try {
+      NLPSentenceDetectorOp sentenceOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+      NLPTokenizerOp tokenizerOp = OpenNLPOpsFactory.getTokenizer(tokenizerModelFile);
+      return new OpenNLPTokenizer(factory, sentenceOp, tokenizerOp);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    // register models in cache with file/resource names
+    if (sentenceModelFile != null) {
+      OpenNLPOpsFactory.getSentenceModel(sentenceModelFile, loader);
+    }
+    if (tokenizerModelFile != null) {
+      OpenNLPOpsFactory.getTokenizerModel(tokenizerModelFile, loader);
+    }
+  }
+}


[38/54] [abbrv] lucene-solr:jira/solr-11702: Add 7.2.0 backcompat indices.

Posted by da...@apache.org.
Add 7.2.0 backcompat indices.


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

Branch: refs/heads/jira/solr-11702
Commit: 69e9f1797664810afeb918531e1778a25e8145a8
Parents: ba2ef4a
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Dec 21 09:05:00 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Dec 21 09:05:00 2017 +0100

----------------------------------------------------------------------
 .../lucene/index/TestBackwardsCompatibility.java   |   7 +++++--
 .../org/apache/lucene/index/index.7.2.0-cfs.zip    | Bin 0 -> 15641 bytes
 .../org/apache/lucene/index/index.7.2.0-nocfs.zip  | Bin 0 -> 15629 bytes
 .../test/org/apache/lucene/index/sorted.7.2.0.zip  | Bin 0 -> 87935 bytes
 4 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69e9f179/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 0ba6fa3..98e7aaf 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -291,7 +291,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     "7.0.1-cfs",
     "7.0.1-nocfs",
     "7.1.0-cfs",
-    "7.1.0-nocfs"
+    "7.1.0-nocfs",
+    "7.2.0-cfs",
+    "7.2.0-nocfs"
   };
 
   public static String[] getOldNames() {
@@ -301,7 +303,8 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
   final static String[] oldSortedNames = {
     "sorted.7.0.0",
     "sorted.7.0.1",
-    "sorted.7.1.0"
+    "sorted.7.1.0",
+    "sorted.7.2.0"
   };
 
   public static String[] getOldSortedNames() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69e9f179/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-cfs.zip
new file mode 100644
index 0000000..5dc88f0
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69e9f179/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-nocfs.zip
new file mode 100644
index 0000000..da79b30
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.7.2.0-nocfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69e9f179/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.2.0.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.2.0.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.2.0.zip
new file mode 100644
index 0000000..30dcec7
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/sorted.7.2.0.zip differ


[32/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8099: Fix IntelliJ dependencies

Posted by da...@apache.org.
LUCENE-8099: Fix IntelliJ dependencies


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

Branch: refs/heads/jira/solr-11702
Commit: 79073fafd34341afb575b4a045f2511d35d30d90
Parents: f8fb139
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Dec 19 08:52:21 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Tue Dec 19 08:52:55 2017 +0000

----------------------------------------------------------------------
 dev-tools/idea/lucene/queries/queries.iml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/79073faf/dev-tools/idea/lucene/queries/queries.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/queries/queries.iml b/dev-tools/idea/lucene/queries/queries.iml
index 948b651..e995df2 100644
--- a/dev-tools/idea/lucene/queries/queries.iml
+++ b/dev-tools/idea/lucene/queries/queries.iml
@@ -13,6 +13,7 @@
     <orderEntry type="sourceFolder" forTests="false" />
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
+    <orderEntry type="module" scope="TEST" module-name="expressions" />
     <orderEntry type="module" module-name="lucene-core" />
   </component>
 </module>


[09/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
SOLR-11285: Simulation framework for autoscaling.


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

Branch: refs/heads/jira/solr-11702
Commit: d6d2e3b2e30b600b9f550abf9414743d628cbcf0
Parents: d66d954
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Dec 14 12:13:05 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Dec 14 12:56:23 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |    4 +
 .../org/apache/solr/cloud/ActionThrottle.java   |   14 +-
 .../org/apache/solr/cloud/AddReplicaCmd.java    |  149 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |   24 +-
 .../solr/cloud/CloudConfigSetService.java       |    2 +-
 .../java/org/apache/solr/cloud/CloudUtil.java   |    7 +
 .../apache/solr/cloud/CreateCollectionCmd.java  |  244 ++--
 .../org/apache/solr/cloud/CreateShardCmd.java   |  102 +-
 .../apache/solr/cloud/DeleteCollectionCmd.java  |    9 +-
 .../org/apache/solr/cloud/DeleteShardCmd.java   |    9 +-
 .../java/org/apache/solr/cloud/MigrateCmd.java  |    9 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   |    9 +-
 .../java/org/apache/solr/cloud/Overseer.java    |   16 +-
 .../cloud/OverseerCollectionMessageHandler.java |   43 +-
 .../java/org/apache/solr/cloud/RestoreCmd.java  |    2 +-
 .../org/apache/solr/cloud/SplitShardCmd.java    |  248 ++--
 .../solr/cloud/autoscaling/AutoScaling.java     |   14 +-
 .../cloud/autoscaling/AutoScalingHandler.java   |   70 +-
 .../cloud/autoscaling/ComputePlanAction.java    |   24 +-
 .../cloud/autoscaling/ExecutePlanAction.java    |   38 +-
 .../cloud/autoscaling/HttpTriggerListener.java  |    6 +-
 .../cloud/autoscaling/NodeAddedTrigger.java     |   15 +-
 .../solr/cloud/autoscaling/NodeLostTrigger.java |   14 +-
 .../autoscaling/OverseerTriggerThread.java      |   20 +-
 .../cloud/autoscaling/ScheduledTriggers.java    |   53 +-
 .../cloud/autoscaling/SearchRateTrigger.java    |   10 +-
 .../cloud/autoscaling/SystemLogListener.java    |    6 +-
 .../cloud/autoscaling/TriggerEventQueue.java    |    4 +-
 .../cloud/autoscaling/TriggerListenerBase.java  |    6 +-
 .../solr/cloud/overseer/CollectionMutator.java  |    8 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |    2 +-
 .../solr/cloud/overseer/SliceMutator.java       |    2 +-
 .../org/apache/solr/core/CoreContainer.java     |    6 +-
 .../solr/handler/CdcrReplicatorManager.java     |   11 +-
 .../solr/handler/admin/CollectionsHandler.java  |    2 +-
 .../org/apache/solr/schema/SchemaManager.java   |    3 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |    5 +-
 .../processor/DistributedUpdateProcessor.java   |    3 +-
 .../src/java/org/apache/solr/util/IdUtils.java  |    1 +
 .../src/java/org/apache/solr/util/TimeOut.java  |   18 +-
 .../java/org/apache/solr/util/TimeSource.java   |   57 -
 .../solr/util/xslt/TransformerProvider.java     |    3 +-
 .../apache/solr/cloud/ActionThrottleTest.java   |   12 +-
 .../org/apache/solr/cloud/AddReplicaTest.java   |    2 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   10 +-
 ...MonkeyNothingIsSafeWithPullReplicasTest.java |    3 +-
 ...aosMonkeySafeLeaderWithPullReplicasTest.java |    3 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |    5 +-
 ...ConcurrentDeleteAndCreateCollectionTest.java |    3 +-
 ...DistribDocExpirationUpdateProcessorTest.java |    3 +-
 .../cloud/LeaderFailureAfterFreshStartTest.java |    3 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |    3 +-
 ...verseerCollectionConfigSetProcessorTest.java |    9 +-
 .../apache/solr/cloud/OverseerRolesTest.java    |    3 +-
 .../solr/cloud/PeerSyncReplicationTest.java     |    3 +-
 .../cloud/SharedFSAutoReplicaFailoverTest.java  |    3 +-
 .../TestLeaderInitiatedRecoveryThread.java      |    7 +-
 .../org/apache/solr/cloud/TestPullReplica.java  |    7 +-
 .../cloud/TestPullReplicaErrorHandling.java     |    9 +-
 .../apache/solr/cloud/TestRebalanceLeaders.java |    5 +-
 .../org/apache/solr/cloud/TestTlogReplica.java  |   11 +-
 .../solr/cloud/UnloadDistributedZkTest.java     |    3 +-
 .../AutoAddReplicasIntegrationTest.java         |    3 +-
 .../AutoAddReplicasPlanActionTest.java          |    4 +-
 .../autoscaling/AutoScalingHandlerTest.java     |    3 +-
 .../solr/cloud/autoscaling/CapturedEvent.java   |   63 +
 .../autoscaling/ComputePlanActionTest.java      |   16 +
 .../autoscaling/ExecutePlanActionTest.java      |    2 +-
 .../cloud/autoscaling/NodeAddedTriggerTest.java |    2 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |    2 +-
 .../autoscaling/TriggerIntegrationTest.java     |  155 +--
 .../solr/cloud/autoscaling/sim/ActionError.java |   24 +
 .../sim/GenericDistributedQueue.java            |  599 ++++++++
 .../sim/GenericDistributedQueueFactory.java     |   45 +
 .../cloud/autoscaling/sim/LiveNodesSet.java     |   99 ++
 .../cloud/autoscaling/sim/SimCloudManager.java  |  607 +++++++++
 .../sim/SimClusterStateProvider.java            | 1275 ++++++++++++++++++
 .../autoscaling/sim/SimDistribStateManager.java |  580 ++++++++
 .../sim/SimDistributedQueueFactory.java         |  284 ++++
 .../autoscaling/sim/SimNodeStateProvider.java   |  267 ++++
 .../autoscaling/sim/SimSolrCloudTestCase.java   |  251 ++++
 .../sim/TestClusterStateProvider.java           |  221 +++
 .../autoscaling/sim/TestComputePlanAction.java  |  357 +++++
 .../sim/TestDistribStateManager.java            |  284 ++++
 .../autoscaling/sim/TestExecutePlanAction.java  |  216 +++
 .../sim/TestGenericDistributedQueue.java        |   32 +
 .../cloud/autoscaling/sim/TestLargeCluster.java |  266 ++++
 .../autoscaling/sim/TestNodeAddedTrigger.java   |  306 +++++
 .../autoscaling/sim/TestNodeLostTrigger.java    |  331 +++++
 .../cloud/autoscaling/sim/TestPolicyCloud.java  |  357 +++++
 .../sim/TestSimDistributedQueue.java            |  220 +++
 .../autoscaling/sim/TestTriggerIntegration.java | 1217 +++++++++++++++++
 .../cloud/autoscaling/sim/package-info.java     |   21 +
 .../cloud/cdcr/BaseCdcrDistributedZkTest.java   |    3 +-
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |    3 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java  |    3 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |    3 +
 .../solr/core/OpenCloseCoreStressTest.java      |    7 +-
 .../cloud/autoscaling/AutoScalingConfig.java    |    9 +
 .../autoscaling/DelegatingCloudManager.java     |   17 +
 .../DelegatingDistribStateManager.java          |   17 +-
 .../DelegatingNodeStateProvider.java            |   11 +
 .../cloud/autoscaling/DistribStateManager.java  |   23 +-
 .../cloud/autoscaling/NodeStateProvider.java    |    4 +-
 .../client/solrj/cloud/autoscaling/Policy.java  |   21 +-
 .../solrj/cloud/autoscaling/PolicyHelper.java   |   26 +-
 .../solrj/cloud/autoscaling/ReplicaInfo.java    |   33 +-
 .../cloud/autoscaling/SolrCloudManager.java     |   10 +-
 .../solrj/cloud/autoscaling/Suggestion.java     |    2 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |    5 +-
 .../client/solrj/impl/ClusterStateProvider.java |    4 +-
 .../solrj/impl/HttpClusterStateProvider.java    |   21 +-
 .../solrj/impl/SolrClientCloudManager.java      |    6 +
 .../solrj/impl/SolrClientNodeStateProvider.java |    6 +
 .../solrj/impl/ZkDistribStateManager.java       |   32 +-
 .../apache/solr/common/cloud/ClusterState.java  |   10 +
 .../org/apache/solr/common/cloud/Replica.java   |   11 +
 .../apache/solr/common/cloud/ZkStateReader.java |   19 +-
 .../org/apache/solr/common/util/TimeSource.java |  161 +++
 .../java/org/apache/solr/common/util/Utils.java |   23 +-
 .../solr/client/solrj/SolrExampleTestsBase.java |    3 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java |    3 +-
 .../solrj/cloud/autoscaling/TestPolicy.java     |   12 +
 .../cloud/AbstractFullDistribZkTestBase.java    |    7 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java |    3 +-
 .../org/apache/solr/cloud/ZkTestServer.java     |    3 +-
 126 files changed, 9186 insertions(+), 843 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b6e729f..a3f6f75 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -49,6 +49,10 @@ Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.10
 Jetty 9.3.20.v20170531
 
+New Features
+----------------------
+* SOLR-11285: Simulation framework for autoscaling. (ab)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java b/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
index f60332c..520a269 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
@@ -20,7 +20,7 @@ import java.lang.invoke.MethodHandles;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,9 +35,7 @@ public class ActionThrottle {
   private final TimeSource timeSource;
 
   public ActionThrottle(String name, long minMsBetweenActions) {
-    this.name = name;
-    this.minMsBetweenActions = minMsBetweenActions;
-    this.timeSource = TimeSource.NANO_TIME;
+    this(name, minMsBetweenActions, TimeSource.NANO_TIME);
   }
   
   public ActionThrottle(String name, long minMsBetweenActions, TimeSource timeSource) {
@@ -47,16 +45,20 @@ public class ActionThrottle {
   }
 
   public ActionThrottle(String name, long minMsBetweenActions, long lastActionStartedAt)  {
+    this(name, minMsBetweenActions, lastActionStartedAt, TimeSource.NANO_TIME);
+  }
+
+  public ActionThrottle(String name, long minMsBetweenActions, long lastActionStartedAt, TimeSource timeSource)  {
     this.name = name;
     this.minMsBetweenActions = minMsBetweenActions;
     this.lastActionStartedAt = lastActionStartedAt;
-    this.timeSource = TimeSource.NANO_TIME;
+    this.timeSource = timeSource;
   }
 
   public void reset() {
     lastActionStartedAt = null;
   }
-  
+
   public void markAttemptingAction() {
     lastActionStartedAt = timeSource.getTime();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
index c785f9f..71a54c14 100644
--- a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.common.SolrCloseableLatch;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -48,7 +49,6 @@ import org.apache.solr.handler.component.ShardHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.Assign.getNodesForNewReplicas;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_CONF;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
@@ -56,6 +56,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonAdminParams.TIMEOUT;
 import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
 
 public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@@ -69,81 +70,30 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
-    addReplica(ocmh.zkStateReader.getClusterState(), message, results, null);
+    addReplica(state, message, results, null);
   }
 
   ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
       throws IOException, InterruptedException {
     log.debug("addReplica() : {}", Utils.toJSONString(message));
+    boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
+    boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
+    final String asyncId = message.getStr(ASYNC);
+
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+    message = assignReplicaDetails(ocmh.cloudManager, clusterState, message, sessionWrapper);
+
     String collection = message.getStr(COLLECTION_PROP);
+    DocCollection coll = clusterState.getCollection(collection);
+
     String node = message.getStr(CoreAdminParams.NODE);
     String shard = message.getStr(SHARD_ID_PROP);
     String coreName = message.getStr(CoreAdminParams.NAME);
     String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
-    int timeout = message.getInt("timeout", 10 * 60); // 10 minutes
+    int timeout = message.getInt(TIMEOUT, 10 * 60); // 10 minutes
     Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
     boolean parallel = message.getBool("parallel", false);
-    boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
-    if (StringUtils.isBlank(coreName)) {
-      coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
-    }
-
-    final String asyncId = message.getStr(ASYNC);
-
-    DocCollection coll = clusterState.getCollection(collection);
-    if (coll == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
-    }
-    if (coll.getSlice(shard) == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Collection: " + collection + " shard: " + shard + " does not exist");
-    }
-    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
-    boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
-
 
-    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
-    // Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
-    if (!skipCreateReplicaInClusterState) {
-      if (CreateShardCmd.usePolicyFramework(coll, ocmh)) {
-        if (node == null) {
-          if(coll.getPolicyName() != null) message.getProperties().put(Policy.POLICY, coll.getPolicyName());
-          node = Assign.identifyNodes(ocmh,
-              clusterState,
-              Collections.emptyList(),
-              collection,
-              message,
-              Collections.singletonList(shard),
-              replicaType == Replica.Type.NRT ? 0 : 1,
-              replicaType == Replica.Type.TLOG ? 0 : 1,
-              replicaType == Replica.Type.PULL ? 0 : 1
-          ).get(0).node;
-          sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
-        }
-      } else {
-        node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
-            ocmh.overseer.getSolrCloudManager()).get(0).nodeName;// TODO: use replica type in this logic too
-      }
-    }
-    log.info("Node Identified {} for creating new replica", node);
-
-    if (!clusterState.liveNodesContain(node)) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
-    }
-    if (coreName == null) {
-      coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, shard, replicaType);
-    } else if (!skipCreateReplicaInClusterState) {
-      //Validate that the core name is unique in that collection
-      for (Slice slice : coll.getSlices()) {
-        for (Replica replica : slice.getReplicas()) {
-          String replicaCoreName = replica.getStr(CORE_NAME_PROP);
-          if (coreName.equals(replicaCoreName)) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Another replica with the same core name already exists" +
-                " for this collection");
-          }
-        }
-      }
-    }
     ModifiableSolrParams params = new ModifiableSolrParams();
 
     ZkStateReader zkStateReader = ocmh.zkStateReader;
@@ -210,6 +160,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
     // For tracking async calls.
     Map<String,String> requestMap = new HashMap<>();
+    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
+
     ocmh.sendShardRequest(node, params, shardHandler, asyncId, requestMap);
 
     final String fnode = node;
@@ -253,4 +205,75 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     );
   }
 
+  public static ZkNodeProps assignReplicaDetails(SolrCloudManager cloudManager, ClusterState clusterState,
+                                                 ZkNodeProps message, AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+    boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
+
+    String collection = message.getStr(COLLECTION_PROP);
+    String node = message.getStr(CoreAdminParams.NODE);
+    String shard = message.getStr(SHARD_ID_PROP);
+    String coreName = message.getStr(CoreAdminParams.NAME);
+    String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
+    Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
+    if (StringUtils.isBlank(coreName)) {
+      coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
+    }
+
+    DocCollection coll = clusterState.getCollection(collection);
+    if (coll == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
+    }
+    if (coll.getSlice(shard) == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Collection: " + collection + " shard: " + shard + " does not exist");
+    }
+
+    // Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
+    if (!skipCreateReplicaInClusterState) {
+      if (CloudUtil.usePolicyFramework(coll, cloudManager)) {
+        if (node == null) {
+          if(coll.getPolicyName() != null) message.getProperties().put(Policy.POLICY, coll.getPolicyName());
+          node = Assign.identifyNodes(cloudManager,
+              clusterState,
+              Collections.emptyList(),
+              collection,
+              message,
+              Collections.singletonList(shard),
+              replicaType == Replica.Type.NRT ? 0 : 1,
+              replicaType == Replica.Type.TLOG ? 0 : 1,
+              replicaType == Replica.Type.PULL ? 0 : 1
+          ).get(0).node;
+          sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
+        }
+      } else {
+        node = Assign.getNodesForNewReplicas(clusterState, collection, shard, 1, node,
+            cloudManager).get(0).nodeName;// TODO: use replica type in this logic too
+      }
+    }
+    log.info("Node Identified {} for creating new replica", node);
+
+    if (!clusterState.liveNodesContain(node)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
+    }
+    if (coreName == null) {
+      coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
+    } else if (!skipCreateReplicaInClusterState) {
+      //Validate that the core name is unique in that collection
+      for (Slice slice : coll.getSlices()) {
+        for (Replica replica : slice.getReplicas()) {
+          String replicaCoreName = replica.getStr(CORE_NAME_PROP);
+          if (coreName.equals(replicaCoreName)) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Another replica with the same core name already exists" +
+                " for this collection");
+          }
+        }
+      }
+    }
+    if (coreNodeName != null) {
+      message = message.plus(CoreAdminParams.CORE_NODE_NAME, coreNodeName);
+    }
+    message = message.plus(CoreAdminParams.NAME, coreName);
+    message = message.plus(CoreAdminParams.NODE, node);
+    return message;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/Assign.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java
index fd0738f..c746c94 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java
@@ -114,7 +114,7 @@ public class Assign {
     }
   }
 
-  public static String assignNode(DistribStateManager stateManager, DocCollection collection) {
+  public static String assignCoreNodeName(DistribStateManager stateManager, DocCollection collection) {
     // for backward compatibility;
     int defaultValue = defaultCounterValue(collection, false);
     String coreNodeName = "core_node" + incAndGetId(stateManager, collection.getName(), defaultValue);
@@ -170,7 +170,7 @@ public class Assign {
     return returnShardId;
   }
 
-  private static String buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
+  private static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
     // TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
     return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
   }
@@ -187,20 +187,20 @@ public class Assign {
     return defaultValue * 20;
   }
 
-  public static String buildCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
+  public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
     Slice slice = collection.getSlice(shard);
     int defaultValue = defaultCounterValue(collection, newCollection);
     int replicaNum = incAndGetId(stateManager, collection.getName(), defaultValue);
-    String coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
+    String coreName = buildSolrCoreName(collection.getName(), shard, type, replicaNum);
     while (existCoreName(coreName, slice)) {
       replicaNum = incAndGetId(stateManager, collection.getName(), defaultValue);
-      coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
+      coreName = buildSolrCoreName(collection.getName(), shard, type, replicaNum);
     }
     return coreName;
   }
 
-  public static String buildCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
-    return buildCoreName(stateManager, collection, shard, type, false);
+  public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
+    return buildSolrCoreName(stateManager, collection, shard, type, false);
   }
 
   private static boolean existCoreName(String coreName, Slice slice) {
@@ -237,7 +237,7 @@ public class Assign {
     return nodeList;
   }
 
-  public static List<ReplicaPosition> identifyNodes(OverseerCollectionMessageHandler ocmh,
+  public static List<ReplicaPosition> identifyNodes(SolrCloudManager cloudManager,
                                                     ClusterState clusterState,
                                                     List<String> nodeList,
                                                     String collectionName,
@@ -248,7 +248,7 @@ public class Assign {
                                                     int numPullReplicas) throws IOException, InterruptedException {
     List<Map> rulesMap = (List) message.get("rule");
     String policyName = message.getStr(POLICY);
-    AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
+    AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
 
     if (rulesMap == null && policyName == null && autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
       log.debug("Identify nodes using default");
@@ -283,7 +283,7 @@ public class Assign {
           (List<Map>) message.get(SNITCH),
           new HashMap<>(),//this is a new collection. So, there are no nodes in any shard
           nodeList,
-          ocmh.overseer.getSolrCloudManager(),
+          cloudManager,
           clusterState);
 
       Map<ReplicaPosition, String> nodeMappings = replicaAssigner.getNodeMappings();
@@ -294,7 +294,7 @@ public class Assign {
       if (message.getStr(CREATE_NODE_SET) == null)
         nodeList = Collections.emptyList();// unless explicitly specified do not pass node list to Policy
       return getPositionsUsingPolicy(collectionName,
-          shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, ocmh.overseer.getSolrCloudManager(), nodeList);
+          shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, cloudManager, nodeList);
     }
   }
 
@@ -397,7 +397,7 @@ public class Assign {
           nodesList);
       return replicaPositions;
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error getting replica locations", e);
     } finally {
       if (log.isTraceEnabled()) {
         if (replicaPositions != null)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
index 6e0583f..3cdc903 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
@@ -42,7 +42,7 @@ public class CloudConfigSetService extends ConfigSetService {
     try {
       // for back compat with cores that can create collections without the collections API
       if (!zkController.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cd.getCollectionName(), true)) {
-        CreateCollectionCmd.createCollectionZkNode(zkController.getZkClient(), cd.getCollectionName(), cd.getCloudDescriptor().getParams());
+        CreateCollectionCmd.createCollectionZkNode(zkController.getSolrCloudManager().getDistribStateManager(), cd.getCollectionName(), cd.getCloudDescriptor().getParams());
       }
     } catch (KeeperException e) {
       SolrException.log(log, null, e);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
index 62cde7c..30de3d4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
@@ -25,6 +25,8 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection;
@@ -130,4 +132,9 @@ public class CloudUtil {
 
   }
 
+  static boolean usePolicyFramework(DocCollection collection, SolrCloudManager cloudManager)
+      throws IOException, InterruptedException {
+    AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
+    return !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || collection.getPolicyName() != null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index 2c4f01e..2171c60 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -18,6 +18,7 @@
 package org.apache.solr.cloud;
 
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -26,11 +27,18 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.cloud.overseer.ClusterStateMutator;
 import org.apache.solr.common.SolrException;
@@ -40,7 +48,6 @@ import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
-import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -51,6 +58,7 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.admin.ConfigSetsHandlerApi;
 import org.apache.solr.handler.component.ShardHandler;
@@ -70,7 +78,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
-import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@@ -81,11 +88,13 @@ import static org.apache.solr.common.util.StrUtils.formatString;
 public class CreateCollectionCmd implements Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
-  private SolrZkClient zkClient;
+  private final TimeSource timeSource;
+  private final DistribStateManager stateManager;
 
   public CreateCollectionCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
-    this.zkClient = ocmh.zkStateReader.getZkClient();
+    this.stateManager = ocmh.cloudManager.getDistribStateManager();
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -103,95 +112,20 @@ public class CreateCollectionCmd implements Cmd {
     }
 
     ocmh.validateConfigOrThrowSolrException(configName);
-    PolicyHelper.SessionWrapper sessionWrapper = null;
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
 
     try {
-      // look at the replication factor and see if it matches reality
-      // if it does not, find best nodes to create more cores
 
-      int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
-      int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
-      int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
-      Map autoScalingJson = Utils.getJson(ocmh.zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
-      String policy = message.getStr(Policy.POLICY);
-      boolean usePolicyFramework = autoScalingJson.get(Policy.CLUSTER_POLICY) != null || policy != null;
-
-      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
       final String async = message.getStr(ASYNC);
 
-      Integer numSlices = message.getInt(NUM_SLICES, null);
-      String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
+      List<String> nodeList = new ArrayList<>();
       List<String> shardNames = new ArrayList<>();
-      if(ImplicitDocRouter.NAME.equals(router)){
-        ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
-        numSlices = shardNames.size();
-      } else {
-        if (numSlices == null ) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param (when using CompositeId router).");
-        }
-        ClusterStateMutator.getShardNames(numSlices, shardNames);
-      }
-
-      int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
-      if (usePolicyFramework && message.getStr(MAX_SHARDS_PER_NODE) != null && maxShardsPerNode > 0) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "'maxShardsPerNode>0' is not supported when autoScaling policies are used");
-      }
-      if (maxShardsPerNode == -1 || usePolicyFramework) maxShardsPerNode = Integer.MAX_VALUE;
-      if (numNrtReplicas + numTlogReplicas <= 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
-      }
-
-      if (numSlices <= 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " must be > 0");
-      }
-
-      // we need to look at every node and see how many cores it serves
-      // add our new cores to existing nodes serving the least number of cores
-      // but (for now) require that each core goes on a distinct node.
-
-      final List<String> nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM);
-      List<ReplicaPosition> replicaPositions;
-      if (nodeList.isEmpty()) {
-        log.warn("It is unusual to create a collection ("+collectionName+") without cores.");
-
-        replicaPositions = new ArrayList<>();
-      } else {
-        int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
-        if (totalNumReplicas > nodeList.size()) {
-          log.warn("Specified number of replicas of "
-              + totalNumReplicas
-              + " on collection "
-              + collectionName
-              + " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
-              + nodeList.size()
-              + "). It's unusual to run two replica of the same slice on the same Solr-instance.");
-        }
-
-        int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
-            Integer.MAX_VALUE :
-            maxShardsPerNode * nodeList.size();
-        int requestedShardsToCreate = numSlices * totalNumReplicas;
-        if (maxShardsAllowedToCreate < requestedShardsToCreate) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
-              + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
-              + ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
-              + ". This allows a maximum of " + maxShardsAllowedToCreate
-              + " to be created. Value of " + NUM_SLICES + " is " + numSlices
-              + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
-              + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
-              + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
-              + ". This requires " + requestedShardsToCreate
-              + " shards to be created (higher than the allowed number)");
-        }
-        replicaPositions = Assign.identifyNodes(ocmh
-            , clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
-        sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
-      }
-
+      List<ReplicaPosition> replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, message,
+          nodeList, shardNames, sessionWrapper);
       ZkStateReader zkStateReader = ocmh.zkStateReader;
       boolean isLegacyCloud = Overseer.isLegacy(zkStateReader);
 
-      ocmh.createConfNode(configName, collectionName, isLegacyCloud);
+      ocmh.createConfNode(stateManager, configName, collectionName, isLegacyCloud);
 
       Map<String,String> collectionParams = new HashMap<>();
       Map<String,Object> collectionProps = message.getProperties();
@@ -201,16 +135,16 @@ public class CreateCollectionCmd implements Cmd {
         }
       }
       
-      createCollectionZkNode(zkClient, collectionName, collectionParams);
+      createCollectionZkNode(stateManager, collectionName, collectionParams);
       
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
 
       // wait for a while until we don't see the collection
-      TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS);
+      TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, timeSource);
       boolean created = false;
       while (! waitUntil.hasTimedOut()) {
-        Thread.sleep(100);
-        created = zkStateReader.getClusterState().hasCollection(collectionName);
+        waitUntil.sleep(100);
+        created = ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
         if(created) break;
       }
       if (!created)
@@ -225,12 +159,14 @@ public class CreateCollectionCmd implements Cmd {
       Map<String, String> requestMap = new HashMap<>();
 
 
-      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , nrtReplicas : {2}, tlogReplicas: {3}, pullReplicas: {4}",
-          collectionName, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas));
+      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , message : {2}",
+          collectionName, shardNames, message));
       Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
+      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
       for (ReplicaPosition replicaPosition : replicaPositions) {
         String nodeName = replicaPosition.node;
-        String coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), zkStateReader.getClusterState().getCollection(collectionName),
+        String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(),
+            ocmh.cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName),
             replicaPosition.shard, replicaPosition.type, true);
         log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
             , coreName, replicaPosition.shard, collectionName, nodeName));
@@ -260,7 +196,7 @@ public class CreateCollectionCmd implements Cmd {
         params.set(COLL_CONF, configName);
         params.set(CoreAdminParams.COLLECTION, collectionName);
         params.set(CoreAdminParams.SHARD, replicaPosition.shard);
-        params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices);
+        params.set(ZkStateReader.NUM_SHARDS_PROP, shardNames.size());
         params.set(CoreAdminParams.NEW_COLLECTION, "true");
         params.set(CoreAdminParams.REPLICA_TYPE, replicaPosition.type.name());
 
@@ -320,10 +256,93 @@ public class CreateCollectionCmd implements Cmd {
     } catch (Exception ex) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
     } finally {
-      if(sessionWrapper != null) sessionWrapper.release();
+      if (sessionWrapper.get() != null) sessionWrapper.get().release();
     }
   }
 
+  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
+                                                            ZkNodeProps message,
+                                                            List<String> nodeList, List<String> shardNames,
+                                                            AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+    final String collectionName = message.getStr(NAME);
+    // look at the replication factor and see if it matches reality
+    // if it does not, find best nodes to create more cores
+    int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
+    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
+    int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
+    AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
+    String policy = message.getStr(Policy.POLICY);
+    boolean usePolicyFramework = !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || policy != null;
+
+    Integer numSlices = message.getInt(NUM_SLICES, null);
+    String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
+    if(ImplicitDocRouter.NAME.equals(router)){
+      ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
+      numSlices = shardNames.size();
+    } else {
+      if (numSlices == null ) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param (when using CompositeId router).");
+      }
+      if (numSlices <= 0) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " must be > 0");
+      }
+      ClusterStateMutator.getShardNames(numSlices, shardNames);
+    }
+
+    int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
+    if (usePolicyFramework && message.getStr(MAX_SHARDS_PER_NODE) != null && maxShardsPerNode > 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'maxShardsPerNode>0' is not supported when autoScaling policies are used");
+    }
+    if (maxShardsPerNode == -1 || usePolicyFramework) maxShardsPerNode = Integer.MAX_VALUE;
+    if (numNrtReplicas + numTlogReplicas <= 0) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
+    }
+
+    // we need to look at every node and see how many cores it serves
+    // add our new cores to existing nodes serving the least number of cores
+    // but (for now) require that each core goes on a distinct node.
+
+    List<ReplicaPosition> replicaPositions;
+    nodeList.addAll(Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM));
+    if (nodeList.isEmpty()) {
+      log.warn("It is unusual to create a collection ("+collectionName+") without cores.");
+
+      replicaPositions = new ArrayList<>();
+    } else {
+      int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
+      if (totalNumReplicas > nodeList.size()) {
+        log.warn("Specified number of replicas of "
+            + totalNumReplicas
+            + " on collection "
+            + collectionName
+            + " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
+            + nodeList.size()
+            + "). It's unusual to run two replica of the same slice on the same Solr-instance.");
+      }
+
+      int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
+          Integer.MAX_VALUE :
+          maxShardsPerNode * nodeList.size();
+      int requestedShardsToCreate = numSlices * totalNumReplicas;
+      if (maxShardsAllowedToCreate < requestedShardsToCreate) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
+            + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
+            + ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
+            + ". This allows a maximum of " + maxShardsAllowedToCreate
+            + " to be created. Value of " + NUM_SLICES + " is " + numSlices
+            + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
+            + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
+            + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
+            + ". This requires " + requestedShardsToCreate
+            + " shards to be created (higher than the allowed number)");
+      }
+      replicaPositions = Assign.identifyNodes(cloudManager
+          , clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
+      sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
+    }
+    return replicaPositions;
+  }
+
   String getConfigName(String coll, ZkNodeProps message) throws KeeperException, InterruptedException {
     String configName = message.getStr(COLL_CONF);
 
@@ -375,12 +394,12 @@ public class CreateCollectionCmd implements Cmd {
     }
   }
 
-  public static void createCollectionZkNode(SolrZkClient zkClient, String collection, Map<String,String> params) {
+  public static void createCollectionZkNode(DistribStateManager stateManager, String collection, Map<String,String> params) {
     log.debug("Check for collection zkNode:" + collection);
     String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
 
     try {
-      if (!zkClient.exists(collectionPath, true)) {
+      if (!stateManager.hasData(collectionPath)) {
         log.debug("Creating collection in ZooKeeper:" + collection);
 
         try {
@@ -394,7 +413,7 @@ public class CreateCollectionCmd implements Cmd {
             // if the config name wasn't passed in, use the default
             if (!collectionProps.containsKey(ZkController.CONFIGNAME_PROP)) {
               // users can create the collection node and conf link ahead of time, or this may return another option
-              getConfName(zkClient, collection, collectionPath, collectionProps);
+              getConfName(stateManager, collection, collectionPath, collectionProps);
             }
 
           } else if (System.getProperty("bootstrap_confdir") != null) {
@@ -417,19 +436,21 @@ public class CreateCollectionCmd implements Cmd {
             // the conf name should should be the collection name of this core
             collectionProps.put(ZkController.CONFIGNAME_PROP, collection);
           } else {
-            getConfName(zkClient, collection, collectionPath, collectionProps);
+            getConfName(stateManager, collection, collectionPath, collectionProps);
           }
 
           collectionProps.remove(ZkStateReader.NUM_SHARDS_PROP);  // we don't put numShards in the collections properties
 
           ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
-          zkClient.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, null, true);
+          stateManager.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, false);
 
         } catch (KeeperException e) {
           // it's okay if the node already exists
           if (e.code() != KeeperException.Code.NODEEXISTS) {
             throw e;
           }
+        } catch (AlreadyExistsException e) {
+          // it's okay if the node already exists
         }
       } else {
         log.debug("Collection zkNode exists");
@@ -441,6 +462,8 @@ public class CreateCollectionCmd implements Cmd {
         return;
       }
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
+    } catch (IOException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
     } catch (InterruptedException e) {
       Thread.interrupted();
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
@@ -448,8 +471,8 @@ public class CreateCollectionCmd implements Cmd {
 
   }
   
-  private static void getConfName(SolrZkClient zkClient, String collection, String collectionPath, Map<String,Object> collectionProps) throws KeeperException,
-  InterruptedException {
+  private static void getConfName(DistribStateManager stateManager, String collection, String collectionPath, Map<String,Object> collectionProps) throws IOException,
+      KeeperException, InterruptedException {
     // check for configName
     log.debug("Looking for collection configName");
     if (collectionProps.containsKey("configName")) {
@@ -461,17 +484,17 @@ public class CreateCollectionCmd implements Cmd {
     int retry = 1;
     int retryLimt = 6;
     for (; retry < retryLimt; retry++) {
-      if (zkClient.exists(collectionPath, true)) {
-        ZkNodeProps cProps = ZkNodeProps.load(zkClient.getData(collectionPath, null, null, true));
+      if (stateManager.hasData(collectionPath)) {
+        VersionedData data = stateManager.getData(collectionPath);
+        ZkNodeProps cProps = ZkNodeProps.load(data.getData());
         if (cProps.containsKey(ZkController.CONFIGNAME_PROP)) {
           break;
         }
       }
 
       try {
-        configNames = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE, null,
-            true);
-      } catch (NoNodeException e) {
+        configNames = stateManager.listData(ZkConfigManager.CONFIGS_ZKNODE);
+      } catch (NoSuchElementException | NoNodeException e) {
         // just keep trying
       }
 
@@ -507,15 +530,4 @@ public class CreateCollectionCmd implements Cmd {
           "Could not find configName for collection " + collection + " found:" + configNames);
     }
   }
-
-  public static boolean usePolicyFramework(ZkStateReader zkStateReader, ZkNodeProps message) {
-    Map autoScalingJson = Collections.emptyMap();
-    try {
-      autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
-    } catch (Exception e) {
-      return false;
-    }
-    return autoScalingJson.get(Policy.CLUSTER_POLICY) != null || message.getStr(Policy.POLICY) != null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
index 18b0b63..c6afdcc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
@@ -25,11 +25,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.common.SolrCloseableLatch;
 import org.apache.solr.common.SolrException;
@@ -76,59 +77,21 @@ public class CreateShardCmd implements Cmd {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'collection' and 'shard' are required parameters");
 
     DocCollection collection = clusterState.getCollection(collectionName);
-    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
-    int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
-    int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
-    int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
-    
-    if (numNrtReplicas + numTlogReplicas <= 0) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
-    }
-    
-    Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
 
     ZkStateReader zkStateReader = ocmh.zkStateReader;
-    PolicyHelper.SessionWrapper sessionWrapper = null;
-    boolean usePolicyFramework = usePolicyFramework(collection,ocmh);
-    List<ReplicaPosition> positions = null;
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
     SolrCloseableLatch countDownLatch;
     try {
-      if (usePolicyFramework) {
-        if (collection.getPolicyName() != null) message.getProperties().put(Policy.POLICY, collection.getPolicyName());
-        positions = Assign.identifyNodes(ocmh,
-            clusterState,
-            Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM),
-            collectionName,
-            message,
-            Collections.singletonList(sliceName),
-            numNrtReplicas,
-            numTlogReplicas,
-            numPullReplicas);
-        sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
-      } else {
-        List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
-            createNodeSetStr, ocmh.overseer.getSolrCloudManager());
-        int i = 0;
-        positions = new ArrayList<>();
-        for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
-            Replica.Type.TLOG, numTlogReplicas,
-            Replica.Type.PULL, numPullReplicas
-        ).entrySet()) {
-          for (int j = 0; j < e.getValue(); j++) {
-            positions.add(new ReplicaPosition(sliceName, j + 1, e.getKey(), sortedNodeList.get(i % sortedNodeList.size()).nodeName));
-            i++;
-          }
-        }
-      }
+      List<ReplicaPosition> positions = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, sessionWrapper);
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
       // wait for a while until we see the shard
       ocmh.waitForNewShard(collectionName, sliceName);
 
       String async = message.getStr(ASYNC);
-      countDownLatch = new SolrCloseableLatch(totalReplicas, ocmh);
+      countDownLatch = new SolrCloseableLatch(positions.size(), ocmh);
       for (ReplicaPosition position : positions) {
         String nodeName = position.node;
-        String coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), collection, sliceName, position.type);
+        String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(), collection, sliceName, position.type);
         log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
             + " on " + nodeName);
 
@@ -166,7 +129,7 @@ public class CreateShardCmd implements Cmd {
         });
       }
     } finally {
-      if(sessionWrapper != null) sessionWrapper.release();
+      if (sessionWrapper.get() != null) sessionWrapper.get().release();
     }
 
     log.debug("Waiting for create shard action to complete");
@@ -177,9 +140,52 @@ public class CreateShardCmd implements Cmd {
 
   }
 
-  static boolean usePolicyFramework(DocCollection collection, OverseerCollectionMessageHandler ocmh)
-      throws IOException, InterruptedException {
-    AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
-    return !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || collection.getPolicyName() != null;
+  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
+         String collectionName, ZkNodeProps message, AtomicReference< PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
+    String sliceName = message.getStr(SHARD_ID_PROP);
+    DocCollection collection = clusterState.getCollection(collectionName);
+
+    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
+    int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
+    int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
+    int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
+
+    if (numNrtReplicas + numTlogReplicas <= 0) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
+    }
+
+    Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
+
+    boolean usePolicyFramework = CloudUtil.usePolicyFramework(collection, cloudManager);
+    List<ReplicaPosition> positions;
+    if (usePolicyFramework) {
+      if (collection.getPolicyName() != null) message.getProperties().put(Policy.POLICY, collection.getPolicyName());
+      positions = Assign.identifyNodes(cloudManager,
+          clusterState,
+          Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM),
+          collection.getName(),
+          message,
+          Collections.singletonList(sliceName),
+          numNrtReplicas,
+          numTlogReplicas,
+          numPullReplicas);
+      sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
+    } else {
+      List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collection.getName(), sliceName, totalReplicas,
+          createNodeSetStr, cloudManager);
+      int i = 0;
+      positions = new ArrayList<>();
+      for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
+          Replica.Type.TLOG, numTlogReplicas,
+          Replica.Type.PULL, numPullReplicas
+      ).entrySet()) {
+        for (int j = 0; j < e.getValue(); j++) {
+          positions.add(new ReplicaPosition(sliceName, j + 1, e.getKey(), sortedNodeList.get(i % sortedNodeList.size()).nodeName));
+          i++;
+        }
+      }
+    }
+    return positions;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
index d2e40f7..dc91905 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DeleteCollectionCmd.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.util.TimeOut;
@@ -49,9 +50,11 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public DeleteCollectionCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -94,13 +97,13 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
 
       // wait for a while until we don't see the collection
-      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
       boolean removed = false;
       while (! timeout.hasTimedOut()) {
-        Thread.sleep(100);
+        timeout.sleep(100);
         removed = !zkStateReader.getClusterState().hasCollection(collection);
         if (removed) {
-          Thread.sleep(500); // just a bit of time so it's more likely other
+          timeout.sleep(500); // just a bit of time so it's more likely other
           // readers see on return
           break;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
index f13fed5..58c4e63 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DeleteShardCmd.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -55,9 +56,11 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 public class DeleteShardCmd implements Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public DeleteShardCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -134,14 +137,14 @@ public class DeleteShardCmd implements Cmd {
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
 
       // wait for a while until we don't see the shard
-      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
       boolean removed = false;
       while (!timeout.hasTimedOut()) {
-        Thread.sleep(100);
+        timeout.sleep(100);
         DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
         removed = collection.getSlice(sliceId) == null;
         if (removed) {
-          Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
+          timeout.sleep(100); // just a bit of time so it's more likely other readers see on return
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java b/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
index cacccb0..02fdb5c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
@@ -38,6 +38,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -63,9 +64,11 @@ import static org.apache.solr.common.util.Utils.makeMap;
 public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public MigrateCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
 
@@ -179,10 +182,10 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     // wait for a while until we see the new rule
     log.info("Waiting to see routing rule updated in clusterstate");
-    TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS, timeSource);
     boolean added = false;
     while (!waitUntil.hasTimedOut()) {
-      Thread.sleep(100);
+      waitUntil.sleep(100);
       sourceCollection = zkStateReader.getClusterState().getCollection(sourceCollection.getName());
       sourceSlice = sourceCollection.getSlice(sourceSlice.getName());
       Map<String, RoutingRule> rules = sourceSlice.getRoutingRules();
@@ -257,7 +260,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
         tempSourceCollectionName, targetLeader.getNodeName());
-    String tempCollectionReplica2 = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
+    String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
         zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
     props = new HashMap<>();
     props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
index 71d5c82..44493ec 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.util.TimeOut;
@@ -54,9 +55,11 @@ public class MoveReplicaCmd implements Cmd{
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
 
   public MoveReplicaCmd(OverseerCollectionMessageHandler ocmh) {
     this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
   }
 
   @Override
@@ -158,11 +161,11 @@ public class MoveReplicaCmd implements Cmd{
         return;
       }
 
-      TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS, timeSource);
       while (!timeOut.hasTimedOut()) {
         coll = ocmh.zkStateReader.getClusterState().getCollection(coll.getName());
         if (coll.getReplica(replica.getName()) != null) {
-          Thread.sleep(100);
+          timeOut.sleep(100);
         } else {
           break;
         }
@@ -233,7 +236,7 @@ public class MoveReplicaCmd implements Cmd{
 
   private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
                                  DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
-    String newCoreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
+    String newCoreName = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
     ZkNodeProps addReplicasProps = new ZkNodeProps(
         COLLECTION_PROP, coll.getName(),
         SHARD_ID_PROP, slice.getName(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 3b65d6f..d1bb13a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -80,7 +80,7 @@ public class Overseer implements SolrCloseable {
   enum LeaderStatus {DONT_KNOW, NO, YES}
 
   private class ClusterStateUpdater implements Runnable, Closeable {
-    
+
     private final ZkStateReader reader;
     private final SolrZkClient zkClient;
     private final String myId;
@@ -88,7 +88,7 @@ public class Overseer implements SolrCloseable {
     private final ZkDistributedQueue stateUpdateQueue;
     //TODO remove in 9.0, we do not push message into this queue anymore
     //Internal queue where overseer stores events that have not yet been published into cloudstate
-    //If Overseer dies while extracting the main queue a new overseer will start from this queue 
+    //If Overseer dies while extracting the main queue a new overseer will start from this queue
     private final ZkDistributedQueue workQueue;
     // Internal map which holds the information about running tasks.
     private final DistributedMap runningMap;
@@ -120,7 +120,7 @@ public class Overseer implements SolrCloseable {
     public Stats getWorkQueueStats()  {
       return workQueue.getZkStats();
     }
-    
+
     @Override
     public void run() {
 
@@ -442,7 +442,7 @@ public class Overseer implements SolrCloseable {
 
   }
 
-  static class OverseerThread extends Thread implements Closeable {
+  public static class OverseerThread extends Thread implements Closeable {
 
     protected volatile boolean isClosed;
     private Closeable thread;
@@ -466,9 +466,9 @@ public class Overseer implements SolrCloseable {
     public boolean isClosed() {
       return this.isClosed;
     }
-    
+
   }
-  
+
   private OverseerThread ccThread;
 
   private OverseerThread updaterThread;
@@ -478,7 +478,7 @@ public class Overseer implements SolrCloseable {
   private final ZkStateReader reader;
 
   private final ShardHandler shardHandler;
-  
+
   private final UpdateShardHandler updateShardHandler;
 
   private final String adminPath;
@@ -505,7 +505,7 @@ public class Overseer implements SolrCloseable {
     this.stats = new Stats();
     this.config = config;
   }
-  
+
   public synchronized void start(String id) {
     this.id = id;
     closed = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 26f1709..abfecab 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -36,6 +36,10 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@@ -65,6 +69,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -73,6 +78,7 @@ import org.apache.solr.handler.component.ShardResponse;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.TimeOut;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -143,8 +149,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   ShardHandlerFactory shardHandlerFactory;
   String adminPath;
   ZkStateReader zkStateReader;
+  SolrCloudManager cloudManager;
   String myId;
   Stats stats;
+  TimeSource timeSource;
 
   // Set that tracks collections that are currently being processed by a running task.
   // This is used for handling mutual exclusion of the tasks.
@@ -182,6 +190,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     this.myId = myId;
     this.stats = stats;
     this.overseer = overseer;
+    this.cloudManager = overseer.getSolrCloudManager();
+    this.timeSource = cloudManager.getTimeSource();
     this.isClosed = false;
     commandMap = new ImmutableMap.Builder<CollectionAction, Cmd>()
         .put(REPLACENODE, new ReplaceNodeCmd(this))
@@ -229,7 +239,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
       CollectionAction action = getCollectionAction(operation);
       Cmd command = commandMap.get(action);
       if (command != null) {
-        command.call(zkStateReader.getClusterState(), message, results);
+        command.call(cloudManager.getClusterStateProvider().getClusterState(), message, results);
       } else {
         throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
             + operation);
@@ -423,9 +433,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
   boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
-    TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS, timeSource);
     while (! timeout.hasTimedOut()) {
-      Thread.sleep(100);
+      timeout.sleep(100);
       DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName);
       if (docCollection == null) { // someone already deleted the collection
         return true;
@@ -465,7 +475,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
 
     boolean firstLoop = true;
     // wait for a while until the state format changes
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
     while (! timeout.hasTimedOut()) {
       DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
       if (collection == null) {
@@ -483,7 +493,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, MIGRATESTATEFORMAT.toLower(), COLLECTION_PROP, collectionName);
         Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
       }
-      Thread.sleep(100);
+      timeout.sleep(100);
     }
     throw new SolrException(ErrorCode.SERVER_ERROR, "Could not migrate state format for collection: " + collectionName);
   }
@@ -642,16 +652,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
       validateConfigOrThrowSolrException(configName);
       
       boolean isLegacyCloud =  Overseer.isLegacy(zkStateReader);
-      createConfNode(configName, collectionName, isLegacyCloud);
+      createConfNode(cloudManager.getDistribStateManager(), configName, collectionName, isLegacyCloud);
       reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
     }
     
     overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
 
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
     boolean areChangesVisible = true;
     while (!timeout.hasTimedOut()) {
-      DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
+      DocCollection collection = cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName);
       areChangesVisible = true;
       for (Map.Entry<String,Object> updateEntry : message.getProperties().entrySet()) {
         String updateKey = updateEntry.getKey();
@@ -663,7 +673,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         }
       }
       if (areChangesVisible) break;
-      Thread.sleep(100);
+      timeout.sleep(100);
     }
 
     if (!areChangesVisible)
@@ -680,7 +690,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
 
   Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreNames) throws InterruptedException {
     Map<String, Replica> result = new HashMap<>();
-    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
     while (true) {
       DocCollection coll = zkStateReader.getClusterState().getCollection(collectionName);
       for (String coreName : coreNames) {
@@ -746,8 +756,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
 
-  void validateConfigOrThrowSolrException(String configName) throws KeeperException, InterruptedException {
-    boolean isValid = zkStateReader.getZkClient().exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true);
+  void validateConfigOrThrowSolrException(String configName) throws IOException, KeeperException, InterruptedException {
+    boolean isValid = cloudManager.getDistribStateManager().hasData(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName);
     if(!isValid) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "Can not find the specified config set: " + configName);
     }
@@ -757,16 +767,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
    * This doesn't validate the config (path) itself and is just responsible for creating the confNode.
    * That check should be done before the config node is created.
    */
-  void createConfNode(String configName, String coll, boolean isLegacyCloud) throws KeeperException, InterruptedException {
+  public static void createConfNode(DistribStateManager stateManager, String configName, String coll, boolean isLegacyCloud) throws IOException, AlreadyExistsException, BadVersionException, KeeperException, InterruptedException {
     
     if (configName != null) {
       String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
       log.debug("creating collections conf node {} ", collDir);
       byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
-      if (zkStateReader.getZkClient().exists(collDir, true)) {
-        zkStateReader.getZkClient().setData(collDir, data, true);
+      if (stateManager.hasData(collDir)) {
+        stateManager.setData(collDir, data, -1);
       } else {
-        zkStateReader.getZkClient().makePath(collDir, data, true);
+        stateManager.makePath(collDir, data, CreateMode.PERSISTENT, false);
       }
     } else {
       if(isLegacyCloud){
@@ -775,7 +785,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         throw new SolrException(ErrorCode.BAD_REQUEST,"Unable to get config name");
       }
     }
-
   }
   
   private void collectionCmd(ZkNodeProps message, ModifiableSolrParams params,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
index 039ab5c..9c9a5c9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
@@ -223,7 +223,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
 
     try {
       List<ReplicaPosition> replicaPositions = Assign.identifyNodes(
-          ocmh, clusterState,
+          ocmh.cloudManager, clusterState,
           nodeList, restoreCollectionName,
           message, sliceNames,
           numNrtReplicas, numTlogReplicas, numPullReplicas);


[07/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/TimeOut.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeOut.java b/solr/core/src/java/org/apache/solr/util/TimeOut.java
index fd91045..bcc29961 100644
--- a/solr/core/src/java/org/apache/solr/util/TimeOut.java
+++ b/solr/core/src/java/org/apache/solr/util/TimeOut.java
@@ -18,26 +18,34 @@ package org.apache.solr.util;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.common.util.TimeSource;
+
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 
 public class TimeOut {
 
   private final long timeoutAt, startTime;
+  private final TimeSource timeSource;
 
-  public TimeOut(long interval, TimeUnit unit) {
-    startTime = System.nanoTime();
+  public TimeOut(long interval, TimeUnit unit, TimeSource timeSource) {
+    this.timeSource = timeSource;
+    startTime = timeSource.getTime();
     this.timeoutAt = startTime + NANOSECONDS.convert(interval, unit);
   }
 
   public boolean hasTimedOut() {
-    return System.nanoTime() > timeoutAt;
+    return timeSource.getTime() > timeoutAt;
+  }
+
+  public void sleep(long ms) throws InterruptedException {
+    timeSource.sleep(ms);
   }
 
   public long timeLeft(TimeUnit unit) {
-    return unit.convert(timeoutAt - System.nanoTime(), NANOSECONDS);
+    return unit.convert(timeoutAt - timeSource.getTime(), NANOSECONDS);
   }
 
   public long timeElapsed(TimeUnit unit) {
-    return unit.convert(System.nanoTime() - startTime, NANOSECONDS);
+    return unit.convert(timeSource.getTime() - startTime, NANOSECONDS);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/TimeSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeSource.java b/solr/core/src/java/org/apache/solr/util/TimeSource.java
deleted file mode 100644
index a0c7bc0..0000000
--- a/solr/core/src/java/org/apache/solr/util/TimeSource.java
+++ /dev/null
@@ -1,57 +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.util;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.solr.common.util.SuppressForbidden;
-
-/**
- * Source of timestamps.
- */
-public abstract class TimeSource {
-
-  /** Implementation that uses {@link System#currentTimeMillis()}. */
-  public static final class CurrentTimeSource extends TimeSource {
-
-    @Override
-    @SuppressForbidden(reason = "Needed to provide timestamps based on currentTimeMillis.")
-    public long getTime() {
-      return TimeUnit.NANOSECONDS.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
-    }
-  }
-
-  /** Implementation that uses {@link System#nanoTime()}. */
-  public static final class NanoTimeSource extends TimeSource {
-
-    @Override
-    public long getTime() {
-      return System.nanoTime();
-    }
-  }
-
-  /** This instance uses {@link CurrentTimeSource} for generating timestamps. */
-  public static final TimeSource CURRENT_TIME = new CurrentTimeSource();
-
-  /** This instance uses {@link NanoTimeSource} for generating timestamps. */
-  public static final TimeSource NANO_TIME = new NanoTimeSource();
-
-  /**
-   * Return a timestamp, in nanosecond unit.
-   */
-  public abstract long getTime();
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
index 76b7285..05c6c9f 100644
--- a/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
+++ b/solr/core/src/java/org/apache/solr/util/xslt/TransformerProvider.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -120,7 +121,7 @@ public class TransformerProvider {
     
     lastFilename = filename;
     lastTemplates = result;
-    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS);
+    cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java b/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
index 5463f5b..21a53d8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ActionThrottleTest.java
@@ -21,7 +21,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Test;
 
 public class ActionThrottleTest extends SolrTestCaseJ4 {
@@ -40,6 +40,16 @@ public class ActionThrottleTest extends SolrTestCaseJ4 {
       return returnValues.get(index++);
     }
 
+    @Override
+    public void sleep(long ms) throws InterruptedException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   // use the same time source as ActionThrottle

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
index 99d785a..07b59aa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AddReplicaTest.java
@@ -73,7 +73,7 @@ public class AddReplicaTest extends SolrCloudTestCase {
         success = true;
         break;
       }
-      assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
+      assertFalse(rsp.toString(), rsp.getRequestStatus() == RequestStatusState.FAILED);
       Thread.sleep(500);
     }
     assertTrue(success);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
index 21c001c..cf26de4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
@@ -75,11 +75,11 @@ public class AssignTest extends SolrTestCaseJ4 {
         zkClientData.get(invocation.getArgument(0)));
     // TODO: fix this to be independent of ZK
     ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
-    String nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    String nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node1", nodeName);
-    nodeName = Assign.assignNode(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node1", nodeName);
-    nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
+    nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
     assertEquals("core_node2", nodeName);
   }
 
@@ -145,8 +145,8 @@ public class AssignTest extends SolrTestCaseJ4 {
       slices.put("shard2", new Slice("shard2", new HashMap<>(), null));
 
       DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
-      assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
-      assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
+      assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildSolrCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
+      assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildSolrCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
     } finally {
       server.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
index c032f6c..67668c9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
@@ -34,6 +34,7 @@ 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.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
@@ -264,7 +265,7 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
         }
       }
       
-      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS));
+      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
 //      waitForAllWarmingSearchers();
       
       Set<String> addFails = getAddFails(indexTreads);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
index ce77996..cee7a35 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
@@ -32,6 +32,7 @@ import org.apache.solr.common.SolrInputDocument;
 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.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.junit.AfterClass;
@@ -206,7 +207,7 @@ public class ChaosMonkeySafeLeaderWithPullReplicasTest extends AbstractFullDistr
     
     log.info("collection state: " + printClusterStateInfo(DEFAULT_COLLECTION));
     
-    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS));
+    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
 //    waitForAllWarmingSearchers();
 
     checkShardConsistency(batchSize == 1, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
index 4ae2b70..5615918 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
@@ -66,6 +66,7 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean.Category;
@@ -458,7 +459,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
         .add("id", "7")
         .add("id", "8")
         .commit(cluster.getSolrClient(), collectionName);
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       try {
         long numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@@ -514,7 +515,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
   private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
 
 
-    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     boolean allTimesAreCorrect = false;
     while (! timeout.hasTimedOut()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
index bc3b9c5..57d38cd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ConcurrentDeleteAndCreateCollectionTest.java
@@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -140,7 +141,7 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
     
     @Override
     public void run() {
-      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS);
+      final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       while (! timeout.hasTimedOut() && failure.get() == null) {
         doWork();
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
index 31f8418..8847cec 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactory;
 import org.apache.solr.util.TimeOut;
@@ -175,7 +176,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
                                 SolrParams params)
       throws SolrServerException, InterruptedException, IOException {
 
-    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     long numFound = cluster.getSolrClient().query(COLLECTION, params).getResults().getNumFound();
     while (0L < numFound && ! timeout.hasTimedOut()) {
       Thread.sleep(Math.max(1, Math.min(5000, timeout.timeLeft(TimeUnit.MILLISECONDS))));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
index 8136d3e..0423428 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 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.util.TimeOut;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -153,7 +154,7 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
       // shutdown the original leader
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
-      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS));
+      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
       waitTillNodesActive();
       log.info("Updating mappings from zk");
       updateMappingsFromZk(jettys, clients, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
index b57d06f..57ff0b6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.RoutingRule;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
@@ -177,7 +178,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
 
     @Override
     public void run() {
-      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+      TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       for (int id = 26*3; id < 500 && ! timeout.hasTimedOut(); id++) {
         String shardKey = "" + (char) ('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution
         SolrInputDocument doc = new SolrInputDocument();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index b6e7415..8a46808 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -44,7 +44,9 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -87,6 +89,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   private static ClusterState clusterStateMock;
   private static SolrZkClient solrZkClientMock;
   private static DistribStateManager stateManagerMock;
+  private static ObjectCache objectCache;
   private static AutoScalingConfig autoScalingConfig = new AutoScalingConfig(Collections.emptyMap());
   private final Map zkMap = new HashMap();
   private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
@@ -136,6 +139,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     overseerMock = mock(Overseer.class);
     zkControllerMock = mock(ZkController.class);
     cloudDataProviderMock = mock(SolrCloudManager.class);
+    objectCache = new ObjectCache();
     clusterStateProviderMock = mock(ClusterStateProvider.class);
     stateManagerMock = mock(DistribStateManager.class);
   }
@@ -173,6 +177,9 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     reset(overseerMock);
     reset(zkControllerMock);
     reset(cloudDataProviderMock);
+    objectCache.clear();
+    when(cloudDataProviderMock.getObjectCache()).thenReturn(objectCache);
+    when(cloudDataProviderMock.getTimeSource()).thenReturn(TimeSource.NANO_TIME);
     reset(clusterStateProviderMock);
     reset(stateManagerMock);
 
@@ -537,7 +544,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   }
   
   protected void waitForEmptyQueue(long maxWait) throws Exception {
-    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS);
+    final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (queue.peek() != null) {
       if (timeout.hasTimedOut())
         fail("Queue not empty within " + maxWait + " ms");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
index 72b5142..edc6695 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
@@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -51,7 +52,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
   }
 
   private void waitForNewOverseer(int seconds, Predicate<String> state) throws Exception {
-    TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
+    TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     String current = null;
     while (timeout.hasTimedOut() == false) {
       current = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index 7ff5334..3dac890 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -48,6 +48,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 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.core.CoreContainer;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.TimeOut;
@@ -162,7 +163,7 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
       log.info("Updating mappings from zk");
-      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS));
+      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
       updateMappingsFromZk(jettys, clients, true);
       assertEquals("PeerSynced node did not become leader", nodePeerSynced, shardToLeaderJetty.get("shard1"));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
index 41c7c40..457af81 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
@@ -54,6 +54,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -389,7 +390,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
   }
 
   private void assertSliceAndReplicaCount(String collection, int numSlices, int numReplicas, int timeOutInMs) throws InterruptedException {
-    TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS);
+    TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
       Collection<Slice> slices = clusterState.getCollection(collection).getActiveSlices();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java b/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
index b6efa53..0fbc0a1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
@@ -24,6 +24,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.util.MockCoreContainer.MockCoreDescriptor;
 import org.apache.solr.util.TimeOut;
@@ -98,7 +99,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     // kill the replica
     int children = cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size();
     ChaosMonkey.stop(notLeader.jetty);
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       if (children > cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size()) {
         break;
@@ -173,7 +174,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     // this should have published a down state so assert that cversion has incremented
     assertTrue(getOverseerCversion() > cversion);
 
-    timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica r = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(replica.getName());
       if (r.getState() == Replica.State.DOWN) {
@@ -217,7 +218,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
     thread = new LeaderInitiatedRecoveryThread(zkController, coreContainer,
         DEFAULT_COLLECTION, SHARD1, replicaCoreNodeProps, 1, coreContainer.getCores().iterator().next().getCoreDescriptor());
     thread.publishDownState(replicaCoreNodeProps.getCoreName(), replica.getName(), replica.getNodeName(), replicaCoreNodeProps.getCoreUrl(), false);
-    timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       Replica.State state = zkController.getLeaderInitiatedRecoveryState(DEFAULT_COLLECTION, SHARD1, replica.getName());
       if (state == Replica.State.DOWN) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index 04e22f7..7d652b8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -52,6 +52,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
@@ -229,7 +230,7 @@ public class TestPullReplica extends SolrCloudTestCase {
       assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.PULL))) {
       //TODO: assert replication < REPLICATION_TIMEOUT_SECS
       try (HttpSolrClient readOnlyReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@@ -500,7 +501,7 @@ public class TestPullReplica extends SolrCloudTestCase {
   }
   
   private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query) throws IOException, SolrServerException, InterruptedException {
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:replicas) {
       try (HttpSolrClient replicaClient = getHttpSolrClient(r.getCoreUrl())) {
         while (true) {
@@ -521,7 +522,7 @@ public class TestPullReplica extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       LOG.info("Collection not yet deleted");
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
index 12b3ef0..dae0b4c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
@@ -43,6 +43,7 @@ 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.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -86,7 +87,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
       proxies.put(proxy.getUrl(), proxy);
       jettys.put(proxy.getUrl(), jetty);
     }
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (true) {
       try {
         CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false");
@@ -159,7 +160,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
       assertNumberOfReplicas(numShards, 0, numShards, true, true);// Replica should still be active, since it doesn't disconnect from ZooKeeper
       {
         long numFound = 0;
-        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
         while (numFound < 20 && !t.hasTimedOut()) {
           Thread.sleep(200);
           numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@@ -233,7 +234,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
   }
   
   private void assertNumDocs(int numDocs, SolrClient client, int timeoutSecs) throws InterruptedException, SolrServerException, IOException {
-    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     long numFound = -1;
     while (!t.hasTimedOut()) {
       Thread.sleep(200);
@@ -299,7 +300,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       LOG.info("Collection not yet deleted");
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
index 798a8a8..b47424f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -118,7 +119,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
   // 2> All the replicas we _think_ are leaders are in the 0th position in the leader election queue.
   // 3> The node that ZooKeeper thinks is the leader is the one we think should be the leader.
   void checkConsistency() throws InterruptedException, KeeperException {
-    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     boolean checkAppearOnce = false;
     boolean checkElectionZero = false;
     boolean checkZkLeadersAgree = false;
@@ -323,7 +324,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
 
   boolean waitForAllPreferreds() throws KeeperException, InterruptedException {
     boolean goAgain = true;
-    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     while (! timeout.hasTimedOut()) {
       goAgain = false;
       Map<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index ff9fbb8..4ac9f02 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -60,6 +60,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.SolrIndexWriter;
@@ -232,7 +233,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
       assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
-    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.TLOG))) {
       //TODO: assert replication < REPLICATION_TIMEOUT_SECS
       try (HttpSolrClient tlogReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@@ -400,7 +401,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
     docCollection = assertNumberOfReplicas(0, 1, 0, true, true);
     
     // Wait until a new leader is elected
-    TimeOut t = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!t.hasTimedOut()) {
       docCollection = getCollectionState(collectionName);
       Replica leader = docCollection.getSlice("shard1").getLeader();
@@ -488,7 +489,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
 
     waitForNumDocsInAllActiveReplicas(2);
     // There are a small delay between new searcher and copy over old updates operation
-    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       if (assertCopyOverOldUpdates(1, timeCopyOverPerCores)) {
         break;
@@ -748,7 +749,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
   }
   
   private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query, int timeout) throws IOException, SolrServerException, InterruptedException {
-    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     for (Replica r:replicas) {
       if (!r.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
         continue;
@@ -772,7 +773,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
   }
   
   private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
-    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
       try {
         Thread.sleep(100);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
index 28a0a4e..16673da 100644
--- a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -73,7 +74,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
 
   private void checkCoreNamePresenceAndSliceCount(String collectionName, String coreName,
       boolean shouldBePresent, int expectedSliceCount) throws Exception {
-    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     Boolean isPresent = null; // null meaning "don't know"
     while (null == isPresent || shouldBePresent != isPresent.booleanValue()) {
       final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
index 600cd65..e3df94b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
@@ -36,6 +36,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
@@ -177,7 +178,7 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
 
   private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
     ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
-    TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
       Thread.sleep(100);
       if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
index b3147a4..d31bba5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanActionTest.java
@@ -37,6 +37,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,6 +45,7 @@ import org.junit.Test;
 import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
 
 public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(3)
@@ -141,7 +143,7 @@ public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
 
   private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
     ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
       Thread.sleep(100);
       if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
index 985d3aa..cf3bff3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
@@ -69,7 +70,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
   }
 
   private static void testAutoAddReplicas() throws Exception {
-    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut()) {
       byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
       ZkNodeProps loaded = ZkNodeProps.load(data);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
new file mode 100644
index 0000000..e08d37b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
@@ -0,0 +1,63 @@
+/*
+ * 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.autoscaling;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+
+/**
+ *
+ */
+public class CapturedEvent {
+  public final AutoScalingConfig.TriggerListenerConfig config;
+  public final TriggerEventProcessorStage stage;
+  public final String actionName;
+  public final TriggerEvent event;
+  public final String message;
+  public final Map<String, Object> context = new HashMap<>();
+  public final long timestamp;
+
+  public CapturedEvent(long timestamp, ActionContext context, AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName,
+                       TriggerEvent event, String message) {
+    if (context != null) {
+      context.toMap(this.context);
+    }
+    this.config = config;
+    this.stage = stage;
+    this.actionName = actionName;
+    this.event = event;
+    this.message = message;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String toString() {
+    return "CapturedEvent{" +
+        "timestamp=" + timestamp +
+        ", stage=" + stage +
+        ", actionName='" + actionName + '\'' +
+        ", event=" + event +
+        ", context=" + context +
+        ", config=" + config +
+        ", message='" + message + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
index 837c683..2b80ec3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -38,11 +39,13 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -139,6 +142,19 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
   }
 
+  @After
+  public void printState() throws Exception {
+    log.debug("-------------_ FINAL STATE --------------");
+    SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    for (String node: cloudManager.getClusterStateProvider().getLiveNodes()) {
+      Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, ImplicitSnitch.tags);
+      log.debug("* Node values: " + node + "\n" + Utils.toJSONString(values));
+    }
+    log.debug("* Live nodes: " + cloudManager.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
+    state.forEachCollection(coll -> log.debug("* Collection " + coll.getName() + " state: " + coll));
+  }
+
   @Test
   public void testNodeLost() throws Exception  {
     // let's start a node so that we have at least two

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
index 74a1a82..fa826b4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
@@ -41,7 +41,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.data.Stat;
 import org.junit.Before;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index f2f3f74..d477594 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index 8bca296..dad81e0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index cacf39c..c104a99 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -56,7 +56,7 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZKUtil;
 import org.apache.zookeeper.data.Stat;
@@ -87,6 +87,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   private static AtomicBoolean triggerFired;
   private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
   private static ZkStateReader zkStateReader;
+  private static SolrCloudManager cloudManager;
 
   // use the same time source as triggers use
   private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
@@ -146,6 +147,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       // lets start a node
       cluster.startJettySolrRunner();
     }
+    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
   }
 
   private void deleteChildrenRecursively(String path) throws Exception {
@@ -322,7 +324,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
     while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
@@ -338,7 +340,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     cluster.stopJettySolrRunner(index);
 
     // ensure that the old trigger sees the stopped node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
+    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
 
     waitForSeconds = 0;
     setTriggerCommand = "{" +
@@ -386,7 +388,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
     while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
@@ -396,7 +398,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     JettySolrRunner newNode = cluster.startJettySolrRunner();
 
     // ensure that the old trigger sees the new node, todo find a better way to do this
-    Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
+    Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
 
     waitForSeconds = 0;
     setTriggerCommand = "{" +
@@ -954,52 +956,22 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(TriggerEventType.NODELOST, ev.getEventType());
   }
 
-  private static class TestEvent {
-    final AutoScalingConfig.TriggerListenerConfig config;
-    final TriggerEventProcessorStage stage;
-    final String actionName;
-    final TriggerEvent event;
-    final String message;
-    final long timestamp;
-
-    TestEvent(AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName, TriggerEvent event, String message) {
-      this.config = config;
-      this.stage = stage;
-      this.actionName = actionName;
-      this.event = event;
-      this.message = message;
-      this.timestamp = timeSource.getTime();
-    }
-
-    @Override
-    public String toString() {
-      return "TestEvent{" +
-          "timestamp=" + timestamp +
-          ", config=" + config +
-          ", stage=" + stage +
-          ", actionName='" + actionName + '\'' +
-          ", event=" + event +
-          ", message='" + message + '\'' +
-          '}';
-    }
-  }
-
-  static Map<String, List<TestEvent>> listenerEvents = new HashMap<>();
+  static Map<String, List<CapturedEvent>> listenerEvents = new HashMap<>();
   static CountDownLatch listenerCreated = new CountDownLatch(1);
   static boolean failDummyAction = false;
 
   public static class TestTriggerListener extends TriggerListenerBase {
     @Override
-    public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-      super.init(dataProvider, config);
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
       listenerCreated.countDown();
     }
 
     @Override
     public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
                                      ActionContext context, Throwable error, String message) {
-      List<TestEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
-      lst.add(new TestEvent(config, stage, actionName, event, message));
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(timeSource.getTime(), context, config, stage, actionName, event, message));
     }
   }
 
@@ -1079,38 +1051,38 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     Thread.sleep(2000);
 
     // check foo events
-    List<TestEvent> testEvents = listenerEvents.get("foo");
-    assertNotNull("foo events: " + testEvents, testEvents);
-    assertEquals("foo events: " + testEvents, 5, testEvents.size());
+    List<CapturedEvent> capturedEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + capturedEvents, capturedEvents);
+    assertEquals("foo events: " + capturedEvents, 5, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+    assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
-    assertEquals("test", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
 
-    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(4).stage);
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(4).stage);
 
     // check bar events
-    testEvents = listenerEvents.get("bar");
-    assertNotNull("bar events", testEvents);
-    assertEquals("bar events", 4, testEvents.size());
+    capturedEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", capturedEvents);
+    assertEquals("bar events", 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
-    assertEquals("test", testEvents.get(0).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
+    assertEquals("test", capturedEvents.get(0).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
-    assertEquals("test1", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test1", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(3).stage);
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(3).stage);
 
     // reset
     triggerFired.set(false);
@@ -1125,37 +1097,37 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     Thread.sleep(2000);
 
     // check foo events
-    testEvents = listenerEvents.get("foo");
-    assertNotNull("foo events: " + testEvents, testEvents);
-    assertEquals("foo events: " + testEvents, 4, testEvents.size());
+    capturedEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + capturedEvents, capturedEvents);
+    assertEquals("foo events: " + capturedEvents, 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+    assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
-    assertEquals("test", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
 
     // check bar events
-    testEvents = listenerEvents.get("bar");
-    assertNotNull("bar events", testEvents);
-    assertEquals("bar events", 4, testEvents.size());
+    capturedEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", capturedEvents);
+    assertEquals("bar events", 4, capturedEvents.size());
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
-    assertEquals("test", testEvents.get(0).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
+    assertEquals("test", capturedEvents.get(0).actionName);
 
-    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
-    assertEquals("test", testEvents.get(1).actionName);
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
+    assertEquals("test", capturedEvents.get(1).actionName);
 
-    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
-    assertEquals("test1", testEvents.get(2).actionName);
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
+    assertEquals("test1", capturedEvents.get(2).actionName);
 
-    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
-    assertEquals("test1", testEvents.get(3).actionName);
+    assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
+    assertEquals("test1", capturedEvents.get(3).actionName);
   }
 
   @Test
@@ -1200,7 +1172,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     Thread.sleep(1000);
 
-    List<TestEvent> capturedEvents = listenerEvents.get("bar");
+    List<CapturedEvent> capturedEvents = listenerEvents.get("bar");
     // we may get a few IGNORED events if other tests caused events within cooldown period
     assertTrue(capturedEvents.toString(), capturedEvents.size() > 0);
     long prevTimestamp = capturedEvents.get(capturedEvents.size() - 1).timestamp;
@@ -1220,11 +1192,11 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     capturedEvents = listenerEvents.get("bar");
     assertTrue(capturedEvents.toString(), capturedEvents.size() > 1);
     for (int i = 0; i < capturedEvents.size() - 1; i++) {
-      TestEvent ev = capturedEvents.get(i);
+      CapturedEvent ev = capturedEvents.get(i);
       assertEquals(ev.toString(), TriggerEventProcessorStage.IGNORED, ev.stage);
       assertTrue(ev.toString(), ev.message.contains("cooldown"));
     }
-    TestEvent ev = capturedEvents.get(capturedEvents.size() - 1);
+    CapturedEvent ev = capturedEvents.get(capturedEvents.size() - 1);
     assertEquals(ev.toString(), TriggerEventProcessorStage.SUCCEEDED, ev.stage);
     // the difference between timestamps of the first SUCCEEDED and the last SUCCEEDED
     // must be larger than cooldown period
@@ -1380,8 +1352,9 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
       Map<String, Object> props = map("waitFor", 0L, "actions", Collections.singletonList(map("name","throttler", "class", ThrottlingTesterAction.class.getName())));
       scheduledTriggers.add(new NodeAddedTrigger("y1", props, resourceLoader, solrCloudManager));
       scheduledTriggers.add(new NodeAddedTrigger("y2", props, resourceLoader, solrCloudManager));
+      scheduledTriggers.resetActionThrottle();
       JettySolrRunner newNode = cluster.startJettySolrRunner();
-      assertTrue(getTriggerFiredLatch().await(10, TimeUnit.SECONDS));
+      assertTrue(getTriggerFiredLatch().await(20, TimeUnit.SECONDS));
       for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
         if (cluster.getJettySolrRunner(i) == newNode) {
           cluster.stopJettySolrRunner(i);
@@ -1439,7 +1412,12 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
   public void testSearchRate() throws Exception {
+    // start a few more jetty-s
+    for (int i = 0; i < 3; i++) {
+      cluster.startJettySolrRunner();
+    }
     CloudSolrClient solrClient = cluster.getSolrClient();
     String COLL1 = "collection1";
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
@@ -1453,6 +1431,8 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         "'enabled' : true," +
         "'rate' : 1.0," +
         "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
         "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
         "]" +
         "}}";
@@ -1466,6 +1446,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
         "'name' : 'srt'," +
         "'trigger' : 'search_rate_trigger'," +
         "'stage' : ['FAILED','SUCCEEDED']," +
+        "'afterAction': ['compute', 'execute', 'test']," +
         "'class' : '" + TestTriggerListener.class.getName() + "'" +
         "}" +
         "}";
@@ -1481,7 +1462,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     // wait for listener to capture the SUCCEEDED stage
     Thread.sleep(2000);
     assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
-    TestEvent ev = listenerEvents.get("srt").get(0);
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
     long now = timeSource.getTime();
     // verify waitFor
     assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
new file mode 100644
index 0000000..c1c070d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/ActionError.java
@@ -0,0 +1,24 @@
+/*
+ * 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.autoscaling.sim;
+
+/**
+ * Interface that helps simulating action errors.
+ */
+public interface ActionError {
+  boolean shouldFail(String... args);
+}


[37/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11426: Make sure commitOnClose is enabled before running testNoCommit

Posted by da...@apache.org.
SOLR-11426: Make sure commitOnClose is enabled before running testNoCommit


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

Branch: refs/heads/jira/solr-11702
Commit: ba2ef4a894ca857ffe4a5ecb310363d315eef05f
Parents: 960a5fd
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Wed Dec 20 16:00:02 2017 -0800
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Wed Dec 20 16:00:02 2017 -0800

----------------------------------------------------------------------
 solr/core/src/test/org/apache/solr/core/TestLazyCores.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ba2ef4a8/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestLazyCores.java b/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
index 4c787d6..afc6b5a 100644
--- a/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
+++ b/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
@@ -42,6 +42,7 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.util.ReadOnlyCoresLocator;
 import org.junit.BeforeClass;
@@ -788,6 +789,7 @@ public class TestLazyCores extends SolrTestCaseJ4 {
   public void testNoCommit() throws Exception {
     String infoStream = System.getProperty("solr.tests.infostream");
     System.setProperty("solr.tests.infostream","true");
+    DirectUpdateHandler2.commitOnClose = true;
 
     CoreContainer cc = init();
     String[] coreList = new String[]{


[53/54] [abbrv] lucene-solr:jira/solr-11702: 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/23524499
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/23524499
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/23524499

Branch: refs/heads/jira/solr-11702
Commit: 23524499a7576ff87458193ec827a8d34515e153
Parents: 81efe52 137675a
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed Dec 27 20:16:28 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed Dec 27 20:16:28 2017 +0700

----------------------------------------------------------------------
 dev-tools/doap/lucene.rdf                       |    7 +
 dev-tools/doap/solr.rdf                         |    7 +
 dev-tools/idea/.idea/ant.xml                    |    1 +
 dev-tools/idea/.idea/modules.xml                |    1 +
 dev-tools/idea/.idea/workspace.xml              |   83 +-
 .../idea/lucene/analysis/opennlp/opennlp.iml    |   30 +
 dev-tools/idea/lucene/facet/facet.iml           |    2 +-
 dev-tools/idea/lucene/queries/queries.iml       |    1 +
 .../contrib/analysis-extras/analysis-extras.iml |    1 +
 .../dataimporthandler-extras.iml                |    1 +
 .../idea/solr/contrib/extraction/extraction.iml |    1 +
 .../lucene/analysis/opennlp/pom.xml.template    |   78 +
 .../maven/lucene/analysis/pom.xml.template      |    1 +
 lucene/CHANGES.txt                              |   28 +
 lucene/MIGRATE.txt                              |   13 +
 lucene/analysis/README.txt                      |    5 +
 lucene/analysis/build.xml                       |    6 +-
 .../miscellaneous/TrimFilterFactory.java        |   12 +-
 .../miscellaneous/TypeAsSynonymFilter.java      |   80 +
 .../TypeAsSynonymFilterFactory.java             |   55 +
 .../analysis/util/MultiTermAwareComponent.java  |    5 +
 ...ache.lucene.analysis.util.TokenFilterFactory |    1 +
 .../analysis/minhash/MinHashFilterTest.java     |    6 +-
 .../TestPerFieldAnalyzerWrapper.java            |    2 +
 .../TestTypeAsSynonymFilterFactory.java         |   50 +
 lucene/analysis/opennlp/build.xml               |  118 +
 lucene/analysis/opennlp/ivy.xml                 |   29 +
 .../analysis/opennlp/OpenNLPChunkerFilter.java  |  108 +
 .../opennlp/OpenNLPChunkerFilterFactory.java    |   81 +
 .../opennlp/OpenNLPLemmatizerFilter.java        |  123 +
 .../opennlp/OpenNLPLemmatizerFilterFactory.java |   89 +
 .../analysis/opennlp/OpenNLPPOSFilter.java      |   97 +
 .../opennlp/OpenNLPPOSFilterFactory.java        |   71 +
 .../opennlp/OpenNLPSentenceBreakIterator.java   |  224 ++
 .../analysis/opennlp/OpenNLPTokenizer.java      |   98 +
 .../opennlp/OpenNLPTokenizerFactory.java        |   79 +
 .../lucene/analysis/opennlp/package-info.java   |   21 +
 .../analysis/opennlp/tools/NLPChunkerOp.java    |   41 +
 .../analysis/opennlp/tools/NLPLemmatizerOp.java |   80 +
 .../analysis/opennlp/tools/NLPNERTaggerOp.java  |   56 +
 .../analysis/opennlp/tools/NLPPOSTaggerOp.java  |   41 +
 .../opennlp/tools/NLPSentenceDetectorOp.java    |   50 +
 .../analysis/opennlp/tools/NLPTokenizerOp.java  |   48 +
 .../opennlp/tools/OpenNLPOpsFactory.java        |  176 +
 .../analysis/opennlp/tools/package-info.java    |   21 +
 lucene/analysis/opennlp/src/java/overview.html  |   61 +
 ...ache.lucene.analysis.util.TokenFilterFactory |   18 +
 ...apache.lucene.analysis.util.TokenizerFactory |   16 +
 .../lucene/analysis/opennlp/en-test-chunker.bin |  Bin 0 -> 89915 bytes
 .../lucene/analysis/opennlp/en-test-lemmas.dict |   12 +
 .../analysis/opennlp/en-test-lemmatizer.bin     |  Bin 0 -> 7370 bytes
 .../analysis/opennlp/en-test-ner-person.bin     |  Bin 0 -> 1700 bytes
 .../analysis/opennlp/en-test-pos-maxent.bin     |  Bin 0 -> 18424 bytes
 .../lucene/analysis/opennlp/en-test-sent.bin    |  Bin 0 -> 1050 bytes
 .../analysis/opennlp/en-test-tokenizer.bin      |  Bin 0 -> 15096 bytes
 .../TestOpenNLPChunkerFilterFactory.java        |   74 +
 .../TestOpenNLPLemmatizerFilterFactory.java     |  169 +
 .../opennlp/TestOpenNLPPOSFilterFactory.java    |   92 +
 .../TestOpenNLPSentenceBreakIterator.java       |  201 +
 .../opennlp/TestOpenNLPTokenizerFactory.java    |   96 +
 .../src/tools/test-model-data/README.txt        |    6 +
 .../src/tools/test-model-data/chunks.txt        | 3566 ++++++++++++++++++
 .../src/tools/test-model-data/lemmas.txt        |  875 +++++
 .../tools/test-model-data/ner_TrainerParams.txt |   21 +
 .../src/tools/test-model-data/ner_flashman.txt  |  143 +
 .../opennlp/src/tools/test-model-data/pos.txt   |   30 +
 .../src/tools/test-model-data/sentences.txt     |  144 +
 .../src/tools/test-model-data/tokenizer.txt     |   69 +
 .../index/TestBackwardsCompatibility.java       |    7 +-
 .../org/apache/lucene/index/index.7.2.0-cfs.zip |  Bin 0 -> 15641 bytes
 .../apache/lucene/index/index.7.2.0-nocfs.zip   |  Bin 0 -> 15629 bytes
 .../org/apache/lucene/index/sorted.7.2.0.zip    |  Bin 0 -> 87935 bytes
 .../org/apache/lucene/index/CheckIndex.java     |    2 +-
 .../org/apache/lucene/search/BoostQuery.java    |    3 +
 .../lucene/search/DisjunctionMaxQuery.java      |   18 +-
 .../lucene/search/DisjunctionMaxScorer.java     |   46 +-
 .../lucene/search/DisjunctionSumScorer.java     |   20 +-
 .../apache/lucene/search/ReqOptSumScorer.java   |  159 +-
 .../apache/lucene/search/TermInSetQuery.java    |    8 +-
 .../org/apache/lucene/search/WANDScorer.java    |   14 +-
 .../java/org/apache/lucene/util/MathUtil.java   |   15 +
 .../apache/lucene/analysis/TestStopFilter.java  |    9 +-
 .../lucene/search/TermInSetQueryTest.java       |    4 +-
 .../lucene/search/TestReqOptSumScorer.java      |  190 +
 .../expressions/ExpressionValueSource.java      |    6 +-
 lucene/facet/build.xml                          |   15 +-
 .../lucene/facet/LongValueFacetCounts.java      |   13 +-
 .../facet/range/LongRangeFacetCounts.java       |    5 +-
 .../taxonomy/TaxonomyFacetSumValueSource.java   |    3 +-
 .../highlight/WeightedSpanTermExtractor.java    |    6 +-
 .../uhighlight/MultiTermHighlighting.java       |    6 +-
 .../search/vectorhighlight/FieldQuery.java      |    8 +-
 .../search/highlight/HighlighterTest.java       |    7 +-
 .../FastVectorHighlighterTest.java              |    7 +-
 lucene/ivy-versions.properties                  |   31 +-
 lucene/licenses/opennlp-maxent-3.0.3.jar.sha1   |    1 +
 lucene/licenses/opennlp-maxent-LICENSE-ASL.txt  |  202 +
 lucene/licenses/opennlp-maxent-NOTICE.txt       |    6 +
 lucene/licenses/opennlp-tools-1.8.3.jar.sha1    |    1 +
 lucene/licenses/opennlp-tools-LICENSE-ASL.txt   |  202 +
 lucene/licenses/opennlp-tools-NOTICE.txt        |    6 +
 lucene/licenses/slf4j-api-1.7.24.jar.sha1       |    1 +
 lucene/licenses/slf4j-api-1.7.7.jar.sha1        |    1 -
 lucene/module-build.xml                         |   22 +
 lucene/queries/build.xml                        |    8 +
 .../apache/lucene/queries/BoostingQuery.java    |  186 -
 .../lucene/queries/CustomScoreProvider.java     |  162 -
 .../apache/lucene/queries/CustomScoreQuery.java |  322 --
 .../lucene/queries/function/BoostedQuery.java   |  184 -
 .../queries/function/FunctionScoreQuery.java    |    7 +
 .../lucene/queries/BoostingQueryTest.java       |   56 -
 .../queries/TestCustomScoreExplanations.java    |   91 -
 .../lucene/queries/TestCustomScoreQuery.java    |  361 --
 .../queries/function/TestBoostedQuery.java      |   98 -
 .../function/TestFunctionScoreQuery.java        |  145 +-
 .../queryparser/xml/CorePlusQueriesParser.java  |    2 -
 .../xml/builders/BoostingQueryBuilder.java      |   58 -
 .../queryparser/xml/DisjunctionMaxQuery.xml     |    4 +-
 .../lucene/queryparser/xml/TestCoreParser.java  |    2 +-
 .../xml/TestCorePlusQueriesParser.java          |    5 -
 .../analysis/BaseTokenStreamTestCase.java       |   32 +-
 solr/CHANGES.txt                                |   29 +
 solr/bin/solr                                   |   34 +
 solr/bin/solr.in.sh                             |   11 +
 solr/contrib/analysis-extras/README.txt         |   10 +-
 solr/contrib/analysis-extras/build.xml          |   20 +-
 solr/contrib/analysis-extras/ivy.xml            |    3 +
 ...ractNamedEntitiesUpdateProcessorFactory.java |  571 +++
 .../apache/solr/update/processor/package.html   |   24 +
 .../collection1/conf/en-test-ner-person.bin     |  Bin 0 -> 1700 bytes
 .../solr/collection1/conf/en-test-sent.bin      |  Bin 0 -> 1050 bytes
 .../solr/collection1/conf/en-test-tokenizer.bin |  Bin 0 -> 15096 bytes
 .../collection1/conf/schema-opennlp-extract.xml |   49 +
 .../conf/solrconfig-opennlp-extract.xml         |  206 +
 .../solrconfig.snippet.randomindexconfig.xml    |   48 +
 ...ractNamedEntitiesUpdateProcessorFactory.java |  192 +
 .../handler/dataimport/TikaEntityProcessor.java |    4 +-
 .../src/resources/solr-default-tika-config.xml  |   20 +
 solr/contrib/extraction/ivy.xml                 |    9 +-
 .../extraction/ExtractingRequestHandler.java    |   14 +-
 .../handler/extraction/XLSXResponseWriter.java  |    5 +-
 .../src/resources/solr-default-tika-config.xml  |   20 +
 solr/core/ivy.xml                               |    2 +-
 .../org/apache/solr/cloud/ActionThrottle.java   |   16 +-
 .../org/apache/solr/cloud/AddReplicaCmd.java    |  149 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |   24 +-
 .../solr/cloud/CloudConfigSetService.java       |    2 +-
 .../java/org/apache/solr/cloud/CloudUtil.java   |    7 +
 .../apache/solr/cloud/CreateCollectionCmd.java  |  244 +-
 .../org/apache/solr/cloud/CreateShardCmd.java   |  102 +-
 .../apache/solr/cloud/DeleteCollectionCmd.java  |    9 +-
 .../org/apache/solr/cloud/DeleteShardCmd.java   |    9 +-
 .../java/org/apache/solr/cloud/MigrateCmd.java  |    9 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   |    9 +-
 .../java/org/apache/solr/cloud/Overseer.java    |   16 +-
 .../cloud/OverseerCollectionMessageHandler.java |   43 +-
 .../java/org/apache/solr/cloud/RestoreCmd.java  |    2 +-
 .../org/apache/solr/cloud/SplitShardCmd.java    |  248 +-
 .../solr/cloud/autoscaling/AutoScaling.java     |   16 +-
 .../cloud/autoscaling/AutoScalingHandler.java   |   70 +-
 .../cloud/autoscaling/ComputePlanAction.java    |   50 +-
 .../cloud/autoscaling/ExecutePlanAction.java    |   42 +-
 .../cloud/autoscaling/HttpTriggerListener.java  |    6 +-
 .../solr/cloud/autoscaling/MetricTrigger.java   |  190 +
 .../cloud/autoscaling/NodeAddedTrigger.java     |   15 +-
 .../solr/cloud/autoscaling/NodeLostTrigger.java |   16 +-
 .../autoscaling/OverseerTriggerThread.java      |   20 +-
 .../cloud/autoscaling/ScheduledTriggers.java    |   53 +-
 .../cloud/autoscaling/SearchRateTrigger.java    |   10 +-
 .../cloud/autoscaling/SystemLogListener.java    |    6 +-
 .../cloud/autoscaling/TriggerEventQueue.java    |    4 +-
 .../cloud/autoscaling/TriggerListenerBase.java  |    6 +-
 .../solr/cloud/overseer/CollectionMutator.java  |    8 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |    2 +-
 .../solr/cloud/overseer/SliceMutator.java       |    2 +-
 .../org/apache/solr/core/CoreContainer.java     |    6 +-
 .../apache/solr/core/CorePropertiesLocator.java |    6 +
 .../solr/handler/CdcrReplicatorManager.java     |   11 +-
 .../org/apache/solr/handler/StreamHandler.java  |    8 +-
 .../solr/handler/admin/CollectionsHandler.java  |    2 +-
 .../org/apache/solr/schema/SchemaManager.java   |    3 +-
 .../apache/solr/search/BoostQParserPlugin.java  |   23 +-
 .../solr/search/ExtendedDismaxQParser.java      |    5 +-
 .../apache/solr/search/ValueSourceParser.java   |    4 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |    5 +-
 .../processor/DistributedUpdateProcessor.java   |    3 +-
 .../src/java/org/apache/solr/util/IdUtils.java  |    1 +
 .../src/java/org/apache/solr/util/TimeOut.java  |   18 +-
 .../java/org/apache/solr/util/TimeSource.java   |   57 -
 .../solr/util/xslt/TransformerProvider.java     |    3 +-
 .../org/apache/solr/BasicFunctionalityTest.java |    2 +-
 .../apache/solr/cloud/ActionThrottleTest.java   |   12 +-
 .../org/apache/solr/cloud/AddReplicaTest.java   |    2 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   10 +-
 ...MonkeyNothingIsSafeWithPullReplicasTest.java |    3 +-
 ...aosMonkeySafeLeaderWithPullReplicasTest.java |    3 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |    5 +-
 ...ConcurrentDeleteAndCreateCollectionTest.java |    3 +-
 ...DistribDocExpirationUpdateProcessorTest.java |    3 +-
 .../cloud/LeaderFailureAfterFreshStartTest.java |    3 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |    3 +-
 ...verseerCollectionConfigSetProcessorTest.java |    9 +-
 .../apache/solr/cloud/OverseerRolesTest.java    |    3 +-
 .../solr/cloud/PeerSyncReplicationTest.java     |    3 +-
 .../cloud/SharedFSAutoReplicaFailoverTest.java  |    3 +-
 .../org/apache/solr/cloud/TestPullReplica.java  |    7 +-
 .../cloud/TestPullReplicaErrorHandling.java     |    9 +-
 .../apache/solr/cloud/TestRebalanceLeaders.java |    5 +-
 .../org/apache/solr/cloud/TestTlogReplica.java  |   11 +-
 .../solr/cloud/UnloadDistributedZkTest.java     |    3 +-
 .../org/apache/solr/cloud/ZkSolrClientTest.java |    4 +-
 .../AutoAddReplicasIntegrationTest.java         |    3 +-
 .../AutoAddReplicasPlanActionTest.java          |    4 +-
 .../autoscaling/AutoScalingHandlerTest.java     |    3 +-
 .../solr/cloud/autoscaling/CapturedEvent.java   |   63 +
 .../autoscaling/ComputePlanActionTest.java      |   16 +
 .../autoscaling/ExecutePlanActionTest.java      |    2 +-
 .../cloud/autoscaling/MetricTriggerTest.java    |  133 +
 .../cloud/autoscaling/NodeAddedTriggerTest.java |    2 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |    2 +-
 .../autoscaling/TriggerIntegrationTest.java     |  285 +-
 .../solr/cloud/autoscaling/sim/ActionError.java |   24 +
 .../sim/GenericDistributedQueue.java            |  599 +++
 .../sim/GenericDistributedQueueFactory.java     |   45 +
 .../cloud/autoscaling/sim/LiveNodesSet.java     |  103 +
 .../cloud/autoscaling/sim/SimCloudManager.java  |  620 +++
 .../sim/SimClusterStateProvider.java            | 1278 +++++++
 .../autoscaling/sim/SimDistribStateManager.java |  579 +++
 .../sim/SimDistributedQueueFactory.java         |  284 ++
 .../autoscaling/sim/SimNodeStateProvider.java   |  298 ++
 .../autoscaling/sim/SimSolrCloudTestCase.java   |  343 ++
 .../sim/TestClusterStateProvider.java           |  221 ++
 .../autoscaling/sim/TestComputePlanAction.java  |  348 ++
 .../sim/TestDistribStateManager.java            |  284 ++
 .../autoscaling/sim/TestExecutePlanAction.java  |  202 +
 .../sim/TestGenericDistributedQueue.java        |   32 +
 .../cloud/autoscaling/sim/TestLargeCluster.java |  463 +++
 .../autoscaling/sim/TestNodeAddedTrigger.java   |  306 ++
 .../autoscaling/sim/TestNodeLostTrigger.java    |  331 ++
 .../cloud/autoscaling/sim/TestPolicyCloud.java  |  357 ++
 .../sim/TestSimDistributedQueue.java            |  220 ++
 .../autoscaling/sim/TestTriggerIntegration.java | 1217 ++++++
 .../cloud/autoscaling/sim/package-info.java     |   21 +
 .../cloud/cdcr/BaseCdcrDistributedZkTest.java   |    3 +-
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |    3 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java  |    3 +-
 .../solr/cloud/rule/ImplicitSnitchTest.java     |    2 +
 .../apache/solr/cloud/rule/RuleEngineTest.java  |    3 +
 .../solr/core/OpenCloseCoreStressTest.java      |    7 +-
 .../org/apache/solr/core/PluginInfoTest.java    |   10 +-
 .../apache/solr/core/TestJmxIntegration.java    |    4 +-
 .../org/apache/solr/core/TestLazyCores.java     |    2 +
 .../apache/solr/core/TestXIncludeConfig.java    |    4 +-
 .../apache/solr/handler/SearchHandlerTest.java  |    4 +-
 .../handler/admin/CoreAdminHandlerTest.java     |   35 +
 .../handler/admin/LukeRequestHandlerTest.java   |    4 +-
 .../admin/SegmentsInfoRequestHandlerTest.java   |    4 +-
 .../handler/component/StatsComponentTest.java   |    4 +-
 .../solr/highlight/HighlighterConfigTest.java   |    4 +-
 .../org/apache/solr/request/TestWriterPerf.java |    4 +-
 .../solr/response/TestBinaryResponseWriter.java |    4 +-
 .../org/apache/solr/schema/IndexSchemaTest.java |   14 +
 .../search/TestComplexPhraseQParserPlugin.java  |    4 +-
 .../solr/search/TestMaxScoreQueryParser.java    |    4 +-
 .../org/apache/solr/search/TestQueryTypes.java  |    4 +-
 .../org/apache/solr/search/TestQueryUtils.java  |    4 +-
 .../org/apache/solr/search/TestSearchPerf.java  |    4 +-
 .../solr/search/TestSurroundQueryParser.java    |    4 +-
 .../search/function/SortByFunctionTest.java     |    4 +-
 .../solr/search/function/TestFunctionQuery.java |    2 +-
 .../solr/security/BasicAuthStandaloneTest.java  |    6 +-
 .../solr/servlet/DirectSolrConnectionTest.java  |    4 +-
 .../update/DirectUpdateHandlerOptimizeTest.java |    4 +-
 .../apache/solr/update/SoftAutoCommitTest.java  |   14 +-
 .../solr/update/TestIndexingPerformance.java    |    4 +-
 .../apache/solr/update/UpdateParamsTest.java    |    4 +-
 .../processor/UpdateProcessorTestBase.java      |  168 -
 .../UpdateRequestProcessorFactoryTest.java      |    4 +-
 solr/licenses/apache-mime4j-core-0.7.2.jar.sha1 |    1 -
 solr/licenses/apache-mime4j-core-0.8.1.jar.sha1 |    1 +
 solr/licenses/apache-mime4j-dom-0.7.2.jar.sha1  |    1 -
 solr/licenses/apache-mime4j-dom-0.8.1.jar.sha1  |    1 +
 solr/licenses/bcmail-jdk15-1.45.jar.sha1        |    1 -
 solr/licenses/bcmail-jdk15on-1.54.jar.sha1      |    1 +
 solr/licenses/bcpkix-jdk15on-1.54.jar.sha1      |    1 +
 .../bcpkix-jdk15on-LICENSE-BSD_LIKE.txt         |   15 +
 solr/licenses/bcpkix-jdk15on-NOTICE.txt         |    2 +
 solr/licenses/bcprov-jdk15-1.45.jar.sha1        |    1 -
 solr/licenses/bcprov-jdk15on-1.54.jar.sha1      |    1 +
 solr/licenses/fontbox-2.0.6.jar.sha1            |    1 -
 solr/licenses/fontbox-2.0.8.jar.sha1            |    1 +
 solr/licenses/jackcess-encrypt-2.1.4.jar.sha1   |    1 +
 solr/licenses/jackcess-encrypt-LICENSE-ASL.txt  |  507 +++
 solr/licenses/jackcess-encrypt-NOTICE.txt       |    2 +
 solr/licenses/jcl-over-slf4j-1.7.24.jar.sha1    |    1 +
 solr/licenses/jcl-over-slf4j-1.7.7.jar.sha1     |    1 -
 solr/licenses/jdom-1.0.jar.sha1                 |    1 -
 solr/licenses/jdom-2.0.2.jar.sha1               |    1 +
 solr/licenses/jul-to-slf4j-1.7.24.jar.sha1      |    1 +
 solr/licenses/jul-to-slf4j-1.7.7.jar.sha1       |    1 -
 .../licenses/metadata-extractor-2.10.1.jar.sha1 |    1 +
 solr/licenses/metadata-extractor-2.9.1.jar.sha1 |    1 -
 solr/licenses/opennlp-maxent-3.0.3.jar.sha1     |    1 +
 solr/licenses/opennlp-maxent-LICENSE-ASL.txt    |  202 +
 solr/licenses/opennlp-maxent-NOTICE.txt         |    6 +
 solr/licenses/opennlp-tools-1.8.3.jar.sha1      |    1 +
 solr/licenses/opennlp-tools-LICENSE-ASL.txt     |  202 +
 solr/licenses/opennlp-tools-NOTICE.txt          |    6 +
 solr/licenses/pdfbox-2.0.6.jar.sha1             |    1 -
 solr/licenses/pdfbox-2.0.8.jar.sha1             |    1 +
 solr/licenses/pdfbox-tools-2.0.6.jar.sha1       |    1 -
 solr/licenses/pdfbox-tools-2.0.8.jar.sha1       |    1 +
 solr/licenses/poi-3.17-beta1.jar.sha1           |    1 -
 solr/licenses/poi-3.17.jar.sha1                 |    1 +
 solr/licenses/poi-ooxml-3.17-beta1.jar.sha1     |    1 -
 solr/licenses/poi-ooxml-3.17.jar.sha1           |    1 +
 .../poi-ooxml-schemas-3.17-beta1.jar.sha1       |    1 -
 solr/licenses/poi-ooxml-schemas-3.17.jar.sha1   |    1 +
 .../licenses/poi-scratchpad-3.17-beta1.jar.sha1 |    1 -
 solr/licenses/poi-scratchpad-3.17.jar.sha1      |    1 +
 solr/licenses/rome-utils-1.5.1.jar.sha1         |    1 +
 solr/licenses/rome-utils-LICENSE-ASL.txt        |   14 +
 solr/licenses/rome-utils-NOTICE.txt             |    1 +
 solr/licenses/slf4j-api-1.7.24.jar.sha1         |    1 +
 solr/licenses/slf4j-api-1.7.7.jar.sha1          |    1 -
 solr/licenses/slf4j-log4j12-1.7.24.jar.sha1     |    1 +
 solr/licenses/slf4j-log4j12-1.7.7.jar.sha1      |    1 -
 solr/licenses/slf4j-simple-1.7.24.jar.sha1      |    1 +
 solr/licenses/slf4j-simple-1.7.7.jar.sha1       |    1 -
 solr/licenses/tika-core-1.16.jar.sha1           |    1 -
 solr/licenses/tika-core-1.17.jar.sha1           |    1 +
 solr/licenses/tika-java7-1.16.jar.sha1          |    1 -
 solr/licenses/tika-java7-1.17.jar.sha1          |    1 +
 solr/licenses/tika-parsers-1.16.jar.sha1        |    1 -
 solr/licenses/tika-parsers-1.17.jar.sha1        |    1 +
 solr/licenses/tika-xmp-1.16.jar.sha1            |    1 -
 solr/licenses/tika-xmp-1.17.jar.sha1            |    1 +
 solr/licenses/xmpcore-5.1.2.jar.sha1            |    1 -
 solr/licenses/xmpcore-5.1.3.jar.sha1            |    1 +
 .../solr-ref-guide/src/filter-descriptions.adoc |   32 +
 solr/solr-ref-guide/src/language-analysis.adoc  |  208 +
 .../src/solrcloud-autoscaling-triggers.adoc     |   32 +-
 .../src/statistical-programming.adoc            |    6 +-
 .../src/stream-evaluator-reference.adoc         |  689 +++-
 solr/solr-ref-guide/src/tokenizers.adoc         |    4 +
 .../src/update-request-processors.adoc          |    6 +
 .../cloud/autoscaling/AutoScalingConfig.java    |    9 +
 .../autoscaling/DelegatingCloudManager.java     |   17 +
 .../DelegatingDistribStateManager.java          |   17 +-
 .../DelegatingNodeStateProvider.java            |   11 +
 .../cloud/autoscaling/DistribStateManager.java  |   23 +-
 .../cloud/autoscaling/NodeStateProvider.java    |    4 +-
 .../client/solrj/cloud/autoscaling/Policy.java  |   21 +-
 .../solrj/cloud/autoscaling/PolicyHelper.java   |   35 +-
 .../solrj/cloud/autoscaling/ReplicaInfo.java    |   33 +-
 .../cloud/autoscaling/SolrCloudManager.java     |   10 +-
 .../solrj/cloud/autoscaling/Suggestion.java     |    2 +-
 .../cloud/autoscaling/TriggerEventType.java     |    3 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |    5 +-
 .../client/solrj/impl/ClusterStateProvider.java |    4 +-
 .../solrj/impl/HttpClusterStateProvider.java    |   21 +-
 .../solrj/impl/SolrClientCloudManager.java      |    6 +
 .../solrj/impl/SolrClientNodeStateProvider.java |    6 +
 .../solrj/impl/ZkDistribStateManager.java       |   32 +-
 .../client/solrj/io/eval/DensityEvaluator.java  |   55 +
 .../solrj/io/eval/IntegrateEvaluator.java       |   70 +
 ...MultiVariateNormalDistributionEvaluator.java |   54 +
 .../solrj/io/eval/PairedTTestEvaluator.java     |   75 +
 .../client/solrj/io/eval/SampleEvaluator.java   |   27 +-
 .../client/solrj/io/eval/TTestEvaluator.java    |   95 +
 .../org/apache/solr/common/SolrException.java   |    8 +-
 .../apache/solr/common/cloud/ClusterState.java  |   10 +
 .../org/apache/solr/common/cloud/Replica.java   |   11 +
 .../apache/solr/common/cloud/ZkStateReader.java |   19 +-
 .../solr/common/params/AutoScalingParams.java   |    4 +
 .../org/apache/solr/common/util/TimeSource.java |  161 +
 .../java/org/apache/solr/common/util/Utils.java |   23 +-
 .../solrj/MergeIndexesExampleTestBase.java      |   21 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |   64 -
 .../solr/client/solrj/SolrExampleTestsBase.java |    3 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java |    3 +-
 .../solrj/cloud/autoscaling/TestPolicy.java     |   12 +
 .../solrj/io/stream/StreamExpressionTest.java   |  120 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   31 +-
 .../cloud/AbstractFullDistribZkTestBase.java    |    7 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java |    3 +-
 .../org/apache/solr/cloud/ZkTestServer.java     |    3 +-
 .../processor/UpdateProcessorTestBase.java      |  168 +
 .../apache/solr/util/AbstractSolrTestCase.java  |  135 -
 389 files changed, 22555 insertions(+), 3340 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23524499/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index 60ad61a,2171c60..543f1e2
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@@ -375,19 -394,12 +394,19 @@@ public class CreateCollectionCmd implem
      }
    }
  
-   public static void createCollectionZkNode(SolrZkClient zkClient, String collection, Map<String,String> params) {
+   public static void createCollectionZkNode(DistribStateManager stateManager, String collection, Map<String,String> params) {
      log.debug("Check for collection zkNode:" + collection);
      String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
 -
 +    String termsPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/terms";
 +    try {
 +      if (zkClient.exists(termsPath, true)) {
 +        zkClient.clean(termsPath);
 +      }
 +    } catch (KeeperException | InterruptedException e) {
 +      throw new SolrException(ErrorCode.SERVER_ERROR, "Error deleting old term nodes for collection from Zookeeper", e);
 +    }
      try {
-       if (!zkClient.exists(collectionPath, true)) {
+       if (!stateManager.hasData(collectionPath)) {
          log.debug("Creating collection in ZooKeeper:" + collection);
  
          try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23524499/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23524499/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23524499/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23524499/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23524499/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------


[42/54] [abbrv] lucene-solr:jira/solr-11702: Remove extra tag

Posted by da...@apache.org.
Remove extra <release> tag


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

Branch: refs/heads/jira/solr-11702
Commit: 67e1b4a19be5a4e8c0aedb45393e64d3b3d73783
Parents: 1b201d0
Author: Steve Rowe <sa...@apache.org>
Authored: Thu Dec 21 17:36:06 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Thu Dec 21 17:36:06 2017 -0500

----------------------------------------------------------------------
 dev-tools/doap/lucene.rdf | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67e1b4a1/dev-tools/doap/lucene.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/lucene.rdf b/dev-tools/doap/lucene.rdf
index 101e094..ade660d 100644
--- a/dev-tools/doap/lucene.rdf
+++ b/dev-tools/doap/lucene.rdf
@@ -75,7 +75,6 @@
       </Version>
     </release>
     <release>
-    <release>
       <Version>
         <name>lucene-7.1.0</name>
         <created>2017-10-17</created>


[48/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11701: Upgrade to Tika 1.17 when available

Posted by da...@apache.org.
SOLR-11701: Upgrade to Tika 1.17 when available


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

Branch: refs/heads/jira/solr-11702
Commit: 7e321d70df302738358266dfcee892dac79d1c0d
Parents: 43f17f7
Author: Erick Erickson <er...@apache.org>
Authored: Sun Dec 24 17:03:00 2017 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Sun Dec 24 17:03:00 2017 -0800

----------------------------------------------------------------------
 .../dataimporthandler-extras.iml                |   1 +
 .../idea/solr/contrib/extraction/extraction.iml |   1 +
 lucene/ivy-versions.properties                  |  28 +-
 lucene/licenses/slf4j-api-1.7.24.jar.sha1       |   1 +
 lucene/licenses/slf4j-api-1.7.7.jar.sha1        |   1 -
 solr/CHANGES.txt                                |   2 +
 .../handler/dataimport/TikaEntityProcessor.java |   4 +-
 .../src/resources/solr-default-tika-config.xml  |  20 +
 solr/contrib/extraction/ivy.xml                 |   9 +-
 .../extraction/ExtractingRequestHandler.java    |  14 +-
 .../handler/extraction/XLSXResponseWriter.java  |   5 +-
 .../src/resources/solr-default-tika-config.xml  |  20 +
 solr/core/ivy.xml                               |   2 +-
 .../solr/cloud/rule/ImplicitSnitchTest.java     |   2 +
 solr/licenses/apache-mime4j-core-0.7.2.jar.sha1 |   1 -
 solr/licenses/apache-mime4j-core-0.8.1.jar.sha1 |   1 +
 solr/licenses/apache-mime4j-dom-0.7.2.jar.sha1  |   1 -
 solr/licenses/apache-mime4j-dom-0.8.1.jar.sha1  |   1 +
 solr/licenses/bcmail-jdk15-1.45.jar.sha1        |   1 -
 solr/licenses/bcmail-jdk15on-1.54.jar.sha1      |   1 +
 solr/licenses/bcpkix-jdk15on-1.54.jar.sha1      |   1 +
 .../bcpkix-jdk15on-LICENSE-BSD_LIKE.txt         |  15 +
 solr/licenses/bcpkix-jdk15on-NOTICE.txt         |   2 +
 solr/licenses/bcprov-jdk15-1.45.jar.sha1        |   1 -
 solr/licenses/bcprov-jdk15on-1.54.jar.sha1      |   1 +
 solr/licenses/fontbox-2.0.6.jar.sha1            |   1 -
 solr/licenses/fontbox-2.0.8.jar.sha1            |   1 +
 solr/licenses/jackcess-encrypt-2.1.4.jar.sha1   |   1 +
 solr/licenses/jackcess-encrypt-LICENSE-ASL.txt  | 507 +++++++++++++++++++
 solr/licenses/jackcess-encrypt-NOTICE.txt       |   2 +
 solr/licenses/jcl-over-slf4j-1.7.24.jar.sha1    |   1 +
 solr/licenses/jcl-over-slf4j-1.7.7.jar.sha1     |   1 -
 solr/licenses/jdom-1.0.jar.sha1                 |   1 -
 solr/licenses/jdom-2.0.2.jar.sha1               |   1 +
 solr/licenses/jul-to-slf4j-1.7.24.jar.sha1      |   1 +
 solr/licenses/jul-to-slf4j-1.7.7.jar.sha1       |   1 -
 .../licenses/metadata-extractor-2.10.1.jar.sha1 |   1 +
 solr/licenses/metadata-extractor-2.9.1.jar.sha1 |   1 -
 solr/licenses/pdfbox-2.0.6.jar.sha1             |   1 -
 solr/licenses/pdfbox-2.0.8.jar.sha1             |   1 +
 solr/licenses/pdfbox-tools-2.0.6.jar.sha1       |   1 -
 solr/licenses/pdfbox-tools-2.0.8.jar.sha1       |   1 +
 solr/licenses/poi-3.17-beta1.jar.sha1           |   1 -
 solr/licenses/poi-3.17.jar.sha1                 |   1 +
 solr/licenses/poi-ooxml-3.17-beta1.jar.sha1     |   1 -
 solr/licenses/poi-ooxml-3.17.jar.sha1           |   1 +
 .../poi-ooxml-schemas-3.17-beta1.jar.sha1       |   1 -
 solr/licenses/poi-ooxml-schemas-3.17.jar.sha1   |   1 +
 .../licenses/poi-scratchpad-3.17-beta1.jar.sha1 |   1 -
 solr/licenses/poi-scratchpad-3.17.jar.sha1      |   1 +
 solr/licenses/rome-utils-1.5.1.jar.sha1         |   1 +
 solr/licenses/rome-utils-LICENSE-ASL.txt        |  14 +
 solr/licenses/rome-utils-NOTICE.txt             |   1 +
 solr/licenses/slf4j-api-1.7.24.jar.sha1         |   1 +
 solr/licenses/slf4j-api-1.7.7.jar.sha1          |   1 -
 solr/licenses/slf4j-log4j12-1.7.24.jar.sha1     |   1 +
 solr/licenses/slf4j-log4j12-1.7.7.jar.sha1      |   1 -
 solr/licenses/slf4j-simple-1.7.24.jar.sha1      |   1 +
 solr/licenses/slf4j-simple-1.7.7.jar.sha1       |   1 -
 solr/licenses/tika-core-1.16.jar.sha1           |   1 -
 solr/licenses/tika-core-1.17.jar.sha1           |   1 +
 solr/licenses/tika-java7-1.16.jar.sha1          |   1 -
 solr/licenses/tika-java7-1.17.jar.sha1          |   1 +
 solr/licenses/tika-parsers-1.16.jar.sha1        |   1 -
 solr/licenses/tika-parsers-1.17.jar.sha1        |   1 +
 solr/licenses/tika-xmp-1.16.jar.sha1            |   1 -
 solr/licenses/tika-xmp-1.17.jar.sha1            |   1 +
 solr/licenses/xmpcore-5.1.2.jar.sha1            |   1 -
 solr/licenses/xmpcore-5.1.3.jar.sha1            |   1 +
 69 files changed, 649 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/dev-tools/idea/solr/contrib/dataimporthandler-extras/dataimporthandler-extras.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/solr/contrib/dataimporthandler-extras/dataimporthandler-extras.iml b/dev-tools/idea/solr/contrib/dataimporthandler-extras/dataimporthandler-extras.iml
index 1166d14..8bc21aa 100644
--- a/dev-tools/idea/solr/contrib/dataimporthandler-extras/dataimporthandler-extras.iml
+++ b/dev-tools/idea/solr/contrib/dataimporthandler-extras/dataimporthandler-extras.iml
@@ -8,6 +8,7 @@
       <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
       <sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
       <sourceFolder url="file://$MODULE_DIR$/src/test-files" type="java-test-resource" />
+      <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
     </content>
     <orderEntry type="inheritedJdk" />
     <orderEntry type="sourceFolder" forTests="false" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/dev-tools/idea/solr/contrib/extraction/extraction.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/solr/contrib/extraction/extraction.iml b/dev-tools/idea/solr/contrib/extraction/extraction.iml
index 7ca96fc..15dad16 100644
--- a/dev-tools/idea/solr/contrib/extraction/extraction.iml
+++ b/dev-tools/idea/solr/contrib/extraction/extraction.iml
@@ -8,6 +8,7 @@
       <sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
       <sourceFolder url="file://$MODULE_DIR$/src/test-files" type="java-test-resource" />
       <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
+      <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
     </content>
     <orderEntry type="inheritedJdk" />
     <orderEntry type="sourceFolder" forTests="false" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 35df7ae..72265a7 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -3,7 +3,7 @@
 # when the lexical sort check is performed by the ant check-lib-versions target.
 
 /antlr/antlr = 2.7.7
-/com.adobe.xmp/xmpcore = 5.1.2
+/com.adobe.xmp/xmpcore = 5.1.3
 
 com.carrotsearch.randomizedtesting.version = 2.5.3
 /com.carrotsearch.randomizedtesting/junit4-ant = ${com.carrotsearch.randomizedtesting.version}
@@ -13,7 +13,7 @@ com.carrotsearch.randomizedtesting.version = 2.5.3
 
 
 /com.cybozu.labs/langdetect = 1.1-20120112
-/com.drewnoakes/metadata-extractor = 2.9.1
+/com.drewnoakes/metadata-extractor = 2.10.1
 
 com.fasterxml.jackson.core.version = 2.5.4
 /com.fasterxml.jackson.core/jackson-annotations = ${com.fasterxml.jackson.core.version}
@@ -30,10 +30,13 @@ com.fasterxml.jackson.core.version = 2.5.4
 /com.googlecode.juniversalchardet/juniversalchardet = 1.0.3
 /com.googlecode.mp4parser/isoparser = 1.1.18
 /com.healthmarketscience.jackcess/jackcess = 2.1.8
+/com.healthmarketscience.jackcess/jackcess-encrypt = 2.1.4
 /com.ibm.icu/icu4j = 59.1
 /com.pff/java-libpst = 0.8.1
 
-/com.rometools/rome = 1.5.1
+com.rometools.version = 1.5.1
+/com.rometools/rome = ${com.rometools.version}
+/com.rometools/rome-utils = ${com.rometools.version}
 com.sun.jersey.version = 1.9
 /com.sun.jersey/jersey-core = ${com.sun.jersey.version}
 /com.sun.jersey/jersey-server = ${com.sun.jersey.version}
@@ -70,7 +73,6 @@ io.netty.netty-all.version = 4.0.36.Final
 /javax.activation/activation = 1.1.1
 /javax.servlet/javax.servlet-api = 3.1.0
 /javax.servlet/servlet-api = 2.4
-/jdom/jdom = 1.0
 /joda-time/joda-time = 2.2
 /junit/junit = 4.10
 /log4j/log4j = 1.2.17
@@ -155,7 +157,7 @@ org.apache.hadoop.version = 2.7.4
 
 /org.apache.ivy/ivy = 2.4.0
 
-org.apache.james.apache.mime4j.version = 0.7.2
+org.apache.james.apache.mime4j.version = 0.8.1
 /org.apache.james/apache-mime4j-core = ${org.apache.james.apache.mime4j.version}
 /org.apache.james/apache-mime4j-dom = ${org.apache.james.apache.mime4j.version}
 
@@ -164,19 +166,19 @@ org.apache.james.apache.mime4j.version = 0.7.2
 /org.apache.opennlp/opennlp-maxent = 3.0.3
 /org.apache.opennlp/opennlp-tools = 1.8.3
 
-org.apache.pdfbox.version = 2.0.6
+org.apache.pdfbox.version = 2.0.8
 /org.apache.pdfbox/fontbox = ${org.apache.pdfbox.version}
 /org.apache.pdfbox/jempbox = 1.8.13
 /org.apache.pdfbox/pdfbox = ${org.apache.pdfbox.version}
 /org.apache.pdfbox/pdfbox-tools = ${org.apache.pdfbox.version}
 
-org.apache.poi.version = 3.17-beta1
+org.apache.poi.version = 3.17
 /org.apache.poi/poi = ${org.apache.poi.version}
 /org.apache.poi/poi-ooxml = ${org.apache.poi.version}
 /org.apache.poi/poi-ooxml-schemas = ${org.apache.poi.version}
 /org.apache.poi/poi-scratchpad = ${org.apache.poi.version}
 
-org.apache.tika.version = 1.16
+org.apache.tika.version = 1.17
 /org.apache.tika/tika-core = ${org.apache.tika.version}
 /org.apache.tika/tika-java7 = ${org.apache.tika.version}
 /org.apache.tika/tika-parsers = ${org.apache.tika.version}
@@ -200,9 +202,10 @@ org.apache.uima.version = 2.3.1
 
 /org.aspectj/aspectjrt = 1.8.0
 
-org.bouncycastle.version = 1.45
-/org.bouncycastle/bcmail-jdk15 = ${org.bouncycastle.version}
-/org.bouncycastle/bcprov-jdk15 = ${org.bouncycastle.version}
+org.bouncycastle.version = 1.54
+/org.bouncycastle/bcmail-jdk15on = ${org.bouncycastle.version}
+/org.bouncycastle/bcpkix-jdk15on = ${org.bouncycastle.version}
+/org.bouncycastle/bcprov-jdk15on = ${org.bouncycastle.version}
 
 /org.carrot2.attributes/attributes-binder = 1.3.1
 /org.carrot2.shaded/carrot2-guava = 18.0
@@ -248,6 +251,7 @@ org.gagravarr.vorbis.java.version = 0.8
 /org.gagravarr/vorbis-java-tika = ${org.gagravarr.vorbis.java.version}
 
 /org.hsqldb/hsqldb = 2.4.0
+/org.jdom/jdom = 2.0.2
 
 /org.jsoup/jsoup = 1.8.2
 
@@ -273,7 +277,7 @@ org.restlet.jee.version = 2.3.0
 
 /org.simpleframework/simple-xml = 2.7.1
 
-org.slf4j.version = 1.7.7
+org.slf4j.version = 1.7.24
 /org.slf4j/jcl-over-slf4j = ${org.slf4j.version}
 /org.slf4j/jul-to-slf4j = ${org.slf4j.version}
 /org.slf4j/slf4j-api = ${org.slf4j.version}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/lucene/licenses/slf4j-api-1.7.24.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/slf4j-api-1.7.24.jar.sha1 b/lucene/licenses/slf4j-api-1.7.24.jar.sha1
new file mode 100644
index 0000000..e2722e7
--- /dev/null
+++ b/lucene/licenses/slf4j-api-1.7.24.jar.sha1
@@ -0,0 +1 @@
+3f6b4bd4f8dbe8d4bea06d107a3826469b85c3e9

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/lucene/licenses/slf4j-api-1.7.7.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/slf4j-api-1.7.7.jar.sha1 b/lucene/licenses/slf4j-api-1.7.7.jar.sha1
deleted file mode 100644
index 1a82218..0000000
--- a/lucene/licenses/slf4j-api-1.7.7.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-2b8019b6249bb05d81d3a3094e468753e2b21311

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d182e36..f98d71d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -79,6 +79,8 @@ Other Changes
 
 * SOLR-11754: Remove AbstractSolrTestCase which has long been supplanted by SolrTestCaseJ4. (David Smiley)
 
+* SOLR-11701: Upgrade to Tika 1.17 when available (Tim Allison, Karthik Ramachandran via Erick Erickson)
+
 ==================  7.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java b/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java
index 95f0073..875032b 100644
--- a/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java
+++ b/solr/contrib/dataimporthandler-extras/src/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java
@@ -86,7 +86,9 @@ public class TikaEntityProcessor extends EntityProcessorBase {
       String tikaConfigFile = context.getResolvedEntityAttribute("tikaConfig");
       if (tikaConfigFile == null) {
         ClassLoader classLoader = context.getSolrCore().getResourceLoader().getClassLoader();
-        tikaConfig = new TikaConfig(classLoader);
+        try (InputStream is = classLoader.getResourceAsStream("solr-default-tika-config.xml")) {
+          tikaConfig = new TikaConfig(is);
+        }
       } else {
         File configFile = new File(tikaConfigFile);
         if (!configFile.isAbsolute()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/contrib/dataimporthandler-extras/src/resources/solr-default-tika-config.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler-extras/src/resources/solr-default-tika-config.xml b/solr/contrib/dataimporthandler-extras/src/resources/solr-default-tika-config.xml
new file mode 100644
index 0000000..b598d9e
--- /dev/null
+++ b/solr/contrib/dataimporthandler-extras/src/resources/solr-default-tika-config.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no" ?>
+<!--
+  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.
+  -->
+<properties>
+  <service-loader initializableProblemHandler="ignore"/>
+</properties>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/contrib/extraction/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/extraction/ivy.xml b/solr/contrib/extraction/ivy.xml
index 7143d81..62ba750 100644
--- a/solr/contrib/extraction/ivy.xml
+++ b/solr/contrib/extraction/ivy.xml
@@ -33,6 +33,7 @@
          (except slf4j-api, commons-codec, commons-logging, commons-httpclient, geronimo-stax-api_1.0_spec, jcip-annotations, xml-apis, asm)
          WARNING: Don't add netcdf / unidataCommon (partially LGPL code) -->
     <dependency org="com.healthmarketscience.jackcess" name="jackcess" rev="${/com.healthmarketscience.jackcess/jackcess}" conf="compile"/>
+    <dependency org="com.healthmarketscience.jackcess" name="jackcess-encrypt" rev="${/com.healthmarketscience.jackcess/jackcess-encrypt}" conf="compile"/>
     <dependency org="org.gagravarr" name="vorbis-java-tika" rev="${/org.gagravarr/vorbis-java-tika}" conf="compile"/>
     <dependency org="org.gagravarr" name="vorbis-java-core" rev="${/org.gagravarr/vorbis-java-core}" conf="compile"/>
     <dependency org="org.apache.james" name="apache-mime4j-core" rev="${/org.apache.james/apache-mime4j-core}" conf="compile"/>
@@ -43,8 +44,9 @@
     <dependency org="org.apache.pdfbox" name="fontbox" rev="${/org.apache.pdfbox/fontbox}" conf="compile"/>
 
     <dependency org="org.apache.pdfbox" name="jempbox" rev="${/org.apache.pdfbox/jempbox}" conf="compile"/>
-    <dependency org="org.bouncycastle" name="bcmail-jdk15" rev="${/org.bouncycastle/bcmail-jdk15}" conf="compile"/>
-    <dependency org="org.bouncycastle" name="bcprov-jdk15" rev="${/org.bouncycastle/bcprov-jdk15}" conf="compile"/>
+    <dependency org="org.bouncycastle" name="bcmail-jdk15on" rev="${/org.bouncycastle/bcmail-jdk15on}" conf="compile"/>
+    <dependency org="org.bouncycastle" name="bcpkix-jdk15on" rev="${/org.bouncycastle/bcpkix-jdk15on}" conf="compile"/>
+    <dependency org="org.bouncycastle" name="bcprov-jdk15on" rev="${/org.bouncycastle/bcprov-jdk15on}" conf="compile"/>
     <dependency org="org.apache.poi" name="poi" rev="${/org.apache.poi/poi}" conf="compile"/>
     <dependency org="org.apache.poi" name="poi-scratchpad" rev="${/org.apache.poi/poi-scratchpad}" conf="compile"/>
     <dependency org="org.apache.poi" name="poi-ooxml" rev="${/org.apache.poi/poi-ooxml}" conf="compile"/>
@@ -58,7 +60,8 @@
     <dependency org="com.drewnoakes" name="metadata-extractor" rev="${/com.drewnoakes/metadata-extractor}" conf="compile"/>
     <dependency org="de.l3s.boilerpipe" name="boilerpipe" rev="${/de.l3s.boilerpipe/boilerpipe}" conf="compile"/>
     <dependency org="com.rometools" name="rome" rev="${/com.rometools/rome}" conf="compile"/>
-    <dependency org="jdom" name="jdom" rev="${/jdom/jdom}" conf="compile"/>
+    <dependency org="com.rometools" name="rome-utils" rev="${/com.rometools/rome-utils}" conf="compile"/>
+    <dependency org="org.jdom" name="jdom" rev="${/org.jdom/jdom}" conf="compile"/>
     <dependency org="com.googlecode.juniversalchardet" name="juniversalchardet" rev="${/com.googlecode.juniversalchardet/juniversalchardet}" conf="compile"/>
     <dependency org="org.tukaani" name="xz" rev="${/org.tukaani/xz}" conf="compile"/>
     <dependency org="com.adobe.xmp" name="xmpcore" rev="${/com.adobe.xmp/xmpcore}" conf="compile"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingRequestHandler.java b/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingRequestHandler.java
index 82fe633..44f7ac4 100644
--- a/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingRequestHandler.java
+++ b/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/ExtractingRequestHandler.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler.extraction;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 import java.util.HashSet;
@@ -37,9 +38,10 @@ import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.apache.tika.config.TikaConfig;
-import org.apache.tika.mime.MimeTypeException;
+import org.apache.tika.exception.TikaException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
 
 
 /**
@@ -110,9 +112,9 @@ public class ExtractingRequestHandler extends ContentStreamHandlerBase implement
       }
     }
     if (config == null) {
-      try {
-        config = getDefaultConfig(core.getResourceLoader().getClassLoader());
-      } catch (MimeTypeException | IOException e) {
+      try (InputStream is = core.getResourceLoader().getClassLoader().getResourceAsStream("solr-default-tika-config.xml")){
+        config = new TikaConfig(is);
+      } catch (IOException | SAXException | TikaException e) {
         throw new SolrException(ErrorCode.SERVER_ERROR, e);
       }
     }
@@ -122,10 +124,6 @@ public class ExtractingRequestHandler extends ContentStreamHandlerBase implement
     factory = createFactory();
   }
 
-  private TikaConfig getDefaultConfig(ClassLoader classLoader) throws MimeTypeException, IOException {
-    return new TikaConfig(classLoader);
-  }
-
   protected SolrContentHandlerFactory createFactory() {
     return new SolrContentHandlerFactory(dateFormats);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java b/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java
index 92bd01f..baa1ddb 100644
--- a/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java
+++ b/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java
@@ -35,6 +35,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.apache.lucene.index.IndexableField;
 import org.apache.poi.ss.usermodel.Cell;
+import org.apache.poi.ss.usermodel.FillPatternType;
 import org.apache.poi.ss.usermodel.Font;
 import org.apache.poi.ss.usermodel.IndexedColors;
 import org.apache.poi.ss.usermodel.Row;
@@ -116,10 +117,10 @@ class XLSXWriter extends TextResponseWriter {
       this.headerStyle = (XSSFCellStyle)swb.createCellStyle();
       this.headerStyle.setFillBackgroundColor(IndexedColors.BLACK.getIndex());
       //solid fill
-      this.headerStyle.setFillPattern((short)1);
+      this.headerStyle.setFillPattern(FillPatternType.SOLID_FOREGROUND);
       Font headerFont = swb.createFont();
       headerFont.setFontHeightInPoints((short)14);
-      headerFont.setBoldweight(Font.BOLDWEIGHT_BOLD);
+      headerFont.setBold(true);
       headerFont.setColor(IndexedColors.WHITE.getIndex());
       this.headerStyle.setFont(headerFont);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/contrib/extraction/src/resources/solr-default-tika-config.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/extraction/src/resources/solr-default-tika-config.xml b/solr/contrib/extraction/src/resources/solr-default-tika-config.xml
new file mode 100644
index 0000000..b598d9e
--- /dev/null
+++ b/solr/contrib/extraction/src/resources/solr-default-tika-config.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no" ?>
+<!--
+  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.
+  -->
+<properties>
+  <service-loader initializableProblemHandler="ignore"/>
+</properties>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 77d3bed..ff4fa48 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -130,7 +130,7 @@
     <dependency org="org.apache.directory.api" name="api-all" rev="${/org.apache.directory.api/api-all}" conf="test.MiniKdc"/>
     <dependency org="org.apache.directory.jdbm" name="apacheds-jdbm1" rev="${/org.apache.directory.jdbm/apacheds-jdbm1}" conf="test.MiniKdc"/>
     <dependency org="org.apache.mina" name="mina-core" rev="${/org.apache.mina/mina-core}" conf="test.MiniKdc"/>
-    <dependency org="org.bouncycastle" name="bcprov-jdk15" rev="${/org.bouncycastle/bcprov-jdk15}" conf="test.MiniKdc"/>
+    <dependency org="org.bouncycastle" name="bcprov-jdk15on" rev="${/org.bouncycastle/bcprov-jdk15on}" conf="test.MiniKdc"/>
     <dependency org="antlr" name="antlr" rev="${/antlr/antlr}" conf="test.MiniKdc"/>
     <dependency org="net.sf.ehcache" name="ehcache-core" rev="${/net.sf.ehcache/ehcache-core}" conf="test.MiniKdc"/>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
index 1793770..8c4006d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import com.google.common.collect.Sets;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestRuleLimitSysouts;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -38,6 +39,7 @@ import static org.hamcrest.core.Is.is;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.when;
 
+@TestRuleLimitSysouts.Limit(bytes=32000)
 public class ImplicitSnitchTest extends LuceneTestCase {
 
   private ImplicitSnitch snitch;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/apache-mime4j-core-0.7.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/apache-mime4j-core-0.7.2.jar.sha1 b/solr/licenses/apache-mime4j-core-0.7.2.jar.sha1
deleted file mode 100644
index f5a55e6..0000000
--- a/solr/licenses/apache-mime4j-core-0.7.2.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-a81264fe0265ebe8fd1d8128aad06dc320de6eef

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/apache-mime4j-core-0.8.1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/apache-mime4j-core-0.8.1.jar.sha1 b/solr/licenses/apache-mime4j-core-0.8.1.jar.sha1
new file mode 100644
index 0000000..bc78865
--- /dev/null
+++ b/solr/licenses/apache-mime4j-core-0.8.1.jar.sha1
@@ -0,0 +1 @@
+c62dfe18a3b827a2c626ade0ffba44562ddf3f61

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/apache-mime4j-dom-0.7.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/apache-mime4j-dom-0.7.2.jar.sha1 b/solr/licenses/apache-mime4j-dom-0.7.2.jar.sha1
deleted file mode 100644
index 9de5694..0000000
--- a/solr/licenses/apache-mime4j-dom-0.7.2.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-1c289aa264548a0a1f1b43685a9cb2ab23f67287

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/apache-mime4j-dom-0.8.1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/apache-mime4j-dom-0.8.1.jar.sha1 b/solr/licenses/apache-mime4j-dom-0.8.1.jar.sha1
new file mode 100644
index 0000000..727798e
--- /dev/null
+++ b/solr/licenses/apache-mime4j-dom-0.8.1.jar.sha1
@@ -0,0 +1 @@
+f2d653c617004193f3350330d907f77b60c88c56

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcmail-jdk15-1.45.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/bcmail-jdk15-1.45.jar.sha1 b/solr/licenses/bcmail-jdk15-1.45.jar.sha1
deleted file mode 100644
index 8fafe38..0000000
--- a/solr/licenses/bcmail-jdk15-1.45.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-3aed7e642dd8d39dc14ed1dec3ff79e084637148

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcmail-jdk15on-1.54.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/bcmail-jdk15on-1.54.jar.sha1 b/solr/licenses/bcmail-jdk15on-1.54.jar.sha1
new file mode 100644
index 0000000..c9176a6
--- /dev/null
+++ b/solr/licenses/bcmail-jdk15on-1.54.jar.sha1
@@ -0,0 +1 @@
+9d9b5432b4b29ef4a853223bc6e19379ef116cca

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcpkix-jdk15on-1.54.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/bcpkix-jdk15on-1.54.jar.sha1 b/solr/licenses/bcpkix-jdk15on-1.54.jar.sha1
new file mode 100644
index 0000000..f0602d6
--- /dev/null
+++ b/solr/licenses/bcpkix-jdk15on-1.54.jar.sha1
@@ -0,0 +1 @@
+b11bfee99bb11eea344de6e4a07fe89212c55c02

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt b/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt
new file mode 100644
index 0000000..9dfdf7c
--- /dev/null
+++ b/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt
@@ -0,0 +1,15 @@
+Copyright (c) 2000-2010 The Legion Of The Bouncy Castle (http://www.bouncycastle.org)
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and
+associated documentation files (the "Software"), to deal in the Software without restriction,
+including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
+and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
+ subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
+ TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
+ THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF
+ CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ IN THE SOFTWARE.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcpkix-jdk15on-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/bcpkix-jdk15on-NOTICE.txt b/solr/licenses/bcpkix-jdk15on-NOTICE.txt
new file mode 100644
index 0000000..be0638a
--- /dev/null
+++ b/solr/licenses/bcpkix-jdk15on-NOTICE.txt
@@ -0,0 +1,2 @@
+Copyright (c) 2000-2006 The Legion Of The Bouncy Castle
+(http://www.bouncycastle.org)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcprov-jdk15-1.45.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/bcprov-jdk15-1.45.jar.sha1 b/solr/licenses/bcprov-jdk15-1.45.jar.sha1
deleted file mode 100644
index bcc01fd..0000000
--- a/solr/licenses/bcprov-jdk15-1.45.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-7741883cb07b4634e8b5fd3337113b6ea770a9bb

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/bcprov-jdk15on-1.54.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/bcprov-jdk15on-1.54.jar.sha1 b/solr/licenses/bcprov-jdk15on-1.54.jar.sha1
new file mode 100644
index 0000000..3f869ac
--- /dev/null
+++ b/solr/licenses/bcprov-jdk15on-1.54.jar.sha1
@@ -0,0 +1 @@
+1acdedeb89f1d950d67b73d481eb7736df65eedb

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/fontbox-2.0.6.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/fontbox-2.0.6.jar.sha1 b/solr/licenses/fontbox-2.0.6.jar.sha1
deleted file mode 100644
index dc11e43..0000000
--- a/solr/licenses/fontbox-2.0.6.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-33f44ea67f1b5ab314e2d5768365b1a3e794fb3b

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/fontbox-2.0.8.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/fontbox-2.0.8.jar.sha1 b/solr/licenses/fontbox-2.0.8.jar.sha1
new file mode 100644
index 0000000..92adafe
--- /dev/null
+++ b/solr/licenses/fontbox-2.0.8.jar.sha1
@@ -0,0 +1 @@
+52f852fcfc7481d45efdffd224eb78b85981b17b

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jackcess-encrypt-2.1.4.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jackcess-encrypt-2.1.4.jar.sha1 b/solr/licenses/jackcess-encrypt-2.1.4.jar.sha1
new file mode 100644
index 0000000..b0153ec
--- /dev/null
+++ b/solr/licenses/jackcess-encrypt-2.1.4.jar.sha1
@@ -0,0 +1 @@
+dfe7134b759597276ff87b7acf662bef1c1c4fd8

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jackcess-encrypt-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/jackcess-encrypt-LICENSE-ASL.txt b/solr/licenses/jackcess-encrypt-LICENSE-ASL.txt
new file mode 100644
index 0000000..3f40d40
--- /dev/null
+++ b/solr/licenses/jackcess-encrypt-LICENSE-ASL.txt
@@ -0,0 +1,507 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+
+APACHE POI SUBCOMPONENTS:
+
+Apache POI includes subcomponents with separate copyright notices and
+license terms. Your use of these subcomponents is subject to the terms
+and conditions of the following licenses:
+
+
+Office Open XML schemas (ooxml-schemas-1.0.jar)
+
+    The Office Open XML schema definitions used by Apache POI are
+    a part of the Office Open XML ECMA Specification (ECMA-376, [1]).
+    As defined in section 9.4 of the ECMA bylaws [2], this specification
+    is available to all interested parties without restriction:
+
+        9.4 All documents when approved shall be made available to
+            all interested parties without restriction.
+
+    Furthermore, both Microsoft and Adobe have granted patent licenses
+    to this work [3,4,5].
+
+    [1] http://www.ecma-international.org/publications/standards/Ecma-376.htm
+    [2] http://www.ecma-international.org/memento/Ecmabylaws.htm
+    [3] http://www.microsoft.com/interop/osp/
+    [4] http://www.ecma-international.org/publications/files/ECMA-ST/Ecma%20PATENT/ECMA-376%20Edition%201%20Microsoft%20Patent%20Declaration.pdf
+    [5] http://www.ecma-international.org/publications/files/ECMA-ST/Ecma%20PATENT/ga-2006-191.pdf
+
+
+DOM4J library (dom4j-1.6.1.jar)
+
+    Copyright 2001-2005 (C) MetaStuff, Ltd. All Rights Reserved.
+
+    Redistribution and use of this software and associated documentation
+    ("Software"), with or without modification, are permitted provided
+    that the following conditions are met:
+
+    1. Redistributions of source code must retain copyright
+       statements and notices.  Redistributions must also contain a
+       copy of this document.
+
+    2. Redistributions in binary form must reproduce the
+       above copyright notice, this list of conditions and the
+       following disclaimer in the documentation and/or other
+       materials provided with the distribution.
+
+    3. The name "DOM4J" must not be used to endorse or promote
+       products derived from this Software without prior written
+       permission of MetaStuff, Ltd.  For written permission,
+       please contact dom4j-info@metastuff.com.
+
+    4. Products derived from this Software may not be called "DOM4J"
+       nor may "DOM4J" appear in their names without prior written
+       permission of MetaStuff, Ltd. DOM4J is a registered
+       trademark of MetaStuff, Ltd.
+
+    5. Due credit should be given to the DOM4J Project - 
+       http://www.dom4j.org
+ 
+    THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS
+    ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT
+    NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+    FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL
+    METASTUFF, LTD. OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT,
+    INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+    (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+    SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+    HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+    STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+    OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+JUnit test library (junit-3.8.1.jar)
+
+    Common Public License - v 1.0
+
+    THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON
+    PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION
+    OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+    1. DEFINITIONS
+
+    "Contribution" means:
+
+    a) in the case of the initial Contributor, the initial code and
+       documentation distributed under this Agreement, and
+
+    b) in the case of each subsequent Contributor:
+
+       i)  changes to the Program, and
+
+       ii) additions to the Program;
+
+       where such changes and/or additions to the Program originate from
+       and are distributed by that particular Contributor. A Contribution
+       'originates' from a Contributor if it was added to the Program by
+       such Contributor itself or anyone acting on such Contributor's behalf.
+       Contributions do not include additions to the Program which: (i) are
+       separate modules of software distributed in conjunction with the
+       Program under their own license agreement, and (ii) are not derivative
+       works of the Program.
+
+    "Contributor" means any person or entity that distributes the Program.
+
+    "Licensed Patents " mean patent claims licensable by a Contributor which
+    are necessarily infringed by the use or sale of its Contribution alone
+    or when combined with the Program.
+
+    "Program" means the Contributions distributed in accordance with this
+    Agreement.
+
+    "Recipient" means anyone who receives the Program under this Agreement,
+    including all Contributors.
+
+    2. GRANT OF RIGHTS
+
+    a) Subject to the terms of this Agreement, each Contributor hereby grants
+       Recipient a non-exclusive, worldwide, royalty-free copyright license
+       to reproduce, prepare derivative works of, publicly display, publicly
+       perform, distribute and sublicense the Contribution of such
+       Contributor, if any, and such derivative works, in source code and
+       object code form.
+
+    b) Subject to the terms of this Agreement, each Contributor hereby grants
+       Recipient a non-exclusive, worldwide, royalty-free patent license under
+       Licensed Patents to make, use, sell, offer to sell, import and
+       otherwise transfer the Contribution of such Contributor, if any, in
+       source code and object code form. This patent license shall apply to
+       the combination of the Contribution and the Program if, at the time
+       the Contribution is added by the Contributor, such addition of the
+       Contribution causes such combination to be covered by the Licensed
+       Patents. The patent license shall not apply to any other combinations
+       which include the Contribution. No hardware per se is licensed
+       hereunder.
+
+    c) Recipient understands that although each Contributor grants the
+       licenses to its Contributions set forth herein, no assurances are
+       provided by any Contributor that the Program does not infringe the
+       patent or other intellectual property rights of any other entity.
+       Each Contributor disclaims any liability to Recipient for claims
+       brought by any other entity based on infringement of intellectual
+       property rights or otherwise. As a condition to exercising the rights
+       and licenses granted hereunder, each Recipient hereby assumes sole
+       responsibility to secure any other intellectual property rights
+       needed, if any. For example, if a third party patent license is
+       required to allow Recipient to distribute the Program, it is
+       Recipient's responsibility to acquire that license before
+       distributing the Program.
+
+    d) Each Contributor represents that to its knowledge it has sufficient
+       copyright rights in its Contribution, if any, to grant the copyright
+       license set forth in this Agreement.
+
+    3. REQUIREMENTS
+
+    A Contributor may choose to distribute the Program in object code form
+    under its own license agreement, provided that:
+
+    a) it complies with the terms and conditions of this Agreement; and
+
+    b) its license agreement:
+
+       i)   effectively disclaims on behalf of all Contributors all warranties
+            and conditions, express and implied, including warranties or
+            conditions of title and non-infringement, and implied warranties
+            or conditions of merchantability and fitness for a particular
+            purpose;
+
+       ii)  effectively excludes on behalf of all Contributors all liability
+            for damages, including direct, indirect, special, incidental and
+            consequential damages, such as lost profits;
+
+       iii) states that any provisions which differ from this Agreement are
+            offered by that Contributor alone and not by any other party; and
+
+       iv)  states that source code for the Program is available from such
+            Contributor, and informs licensees how to obtain it in a
+            reasonable manner on or through a medium customarily used for
+            software exchange.
+
+    When the Program is made available in source code form:
+
+    a) it must be made available under this Agreement; and
+
+    b) a copy of this Agreement must be included with each copy of
+       the Program.
+
+    Contributors may not remove or alter any copyright notices contained
+    within the Program.
+
+    Each Contributor must identify itself as the originator of its
+    Contribution, if any, in a manner that reasonably allows subsequent
+    Recipients to identify the originator of the Contribution.
+
+    4. COMMERCIAL DISTRIBUTION
+
+    Commercial distributors of software may accept certain responsibilities
+    with respect to end users, business partners and the like. While this
+    license is intended to facilitate the commercial use of the Program,
+    the Contributor who includes the Program in a commercial product offering
+    should do so in a manner which does not create potential liability for
+    other Contributors. Therefore, if a Contributor includes the Program
+    in a commercial product offering, such Contributor ("Commercial
+    Contributor") hereby agrees to defend and indemnify every other
+    Contributor ("Indemnified Contributor") against any losses, damages
+    and costs (collectively "Losses") arising from claims, lawsuits and
+    other legal actions brought by a third party against the Indemnified
+    Contributor to the extent caused by the acts or omissions of such
+    Commercial Contributor in connection with its distribution of the
+    Program in a commercial product offering. The obligations in this
+    section do not apply to any claims or Losses relating to any actual
+    or alleged intellectual property infringement. In order to qualify,
+    an Indemnified Contributor must: a) promptly notify the Commercial
+    Contributor in writing of such claim, and b) allow the Commercial
+    Contributor to control, and cooperate with the Commercial Contributor
+    in, the defense and any related settlement negotiations. The Indemnified
+    Contributor may participate in any such claim at its own expense.
+
+    For example, a Contributor might include the Program in a commercial
+    product offering, Product X. That Contributor is then a Commercial
+    Contributor. If that Commercial Contributor then makes performance
+    claims, or offers warranties related to Product X, those performance
+    claims and warranties are such Commercial Contributor's responsibility
+    alone. Under this section, the Commercial Contributor would have to
+    defend claims against the other Contributors related to those
+    performance claims and warranties, and if a court requires any other
+    Contributor to pay any damages as a result, the Commercial Contributor
+    must pay those damages.
+
+    5. NO WARRANTY
+
+    EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED
+    ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER
+    EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR
+    CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR
+    A PARTICULAR PURPOSE. Each Recipient is solely responsible for
+    determining the appropriateness of using and distributing the Program
+    and assumes all risks associated with its exercise of rights under this
+    Agreement, including but not limited to the risks and costs of program
+    errors, compliance with applicable laws, damage to or loss of data,
+    programs or equipment, and unavailability or interruption of operations.
+
+    6. DISCLAIMER OF LIABILITY
+
+    EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR
+    ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT,
+    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING
+    WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF
+    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR
+    DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED
+    HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+    7. GENERAL
+
+    If any provision of this Agreement is invalid or unenforceable under
+    applicable law, it shall not affect the validity or enforceability of
+    the remainder of the terms of this Agreement, and without further
+    action by the parties hereto, such provision shall be reformed to the
+    minimum extent necessary to make such provision valid and enforceable.
+
+    If Recipient institutes patent litigation against a Contributor with
+    respect to a patent applicable to software (including a cross-claim or
+    counterclaim in a lawsuit), then any patent licenses granted by that
+    Contributor to such Recipient under this Agreement shall terminate as of
+    the date such litigation is filed. In addition, if Recipient institutes
+    patent litigation against any entity (including a cross-claim or
+    counterclaim in a lawsuit) alleging that the Program itself (excluding
+    combinations of the Program with other software or hardware) infringes
+    such Recipient's patent(s), then such Recipient's rights granted under
+    Section 2(b) shall terminate as of the date such litigation is filed.
+
+    All Recipient's rights under this Agreement shall terminate if it fails
+    to comply with any of the material terms or conditions of this Agreement
+    and does not cure such failure in a reasonable period of time after
+    becoming aware of such noncompliance. If all Recipient's rights under
+    this Agreement terminate, Recipient agrees to cease use and distribution
+    of the Program as soon as reasonably practicable. However, Recipient's
+    obligations under this Agreement and any licenses granted by Recipient
+    relating to the Program shall continue and survive.
+
+    Everyone is permitted to copy and distribute copies of this Agreement,
+    but in order to avoid inconsistency the Agreement is copyrighted and may
+    only be modified in the following manner. The Agreement Steward reserves
+    the right to publish new versions (including revisions) of this Agreement
+    from time to time. No one other than the Agreement Steward has the right
+    to modify this Agreement. IBM is the initial Agreement Steward. IBM may
+    assign the responsibility to serve as the Agreement Steward to a suitable
+    separate entity. Each new version of the Agreement will be given a
+    distinguishing version number. The Program (including Contributions) may
+    always be distributed subject to the version of the Agreement under which
+    it was received. In addition, after a new version of the Agreement is
+    published, Contributor may elect to distribute the Program (including
+    its Contributions) under the new version. Except as expressly stated in
+    Sections 2(a) and 2(b) above, Recipient receives no rights or licenses
+    to the intellectual property of any Contributor under this Agreement,
+    whether expressly, by implication, estoppel or otherwise. All rights in
+    the Program not expressly granted under this Agreement are reserved.
+
+    This Agreement is governed by the laws of the State of New York and the
+    intellectual property laws of the United States of America. No party to
+    this Agreement will bring a legal action under this Agreement more than
+    one year after the cause of action arose. Each party waives its rights
+    to a jury trial in any resulting litigation.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jackcess-encrypt-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/jackcess-encrypt-NOTICE.txt b/solr/licenses/jackcess-encrypt-NOTICE.txt
new file mode 100644
index 0000000..99cb4a0
--- /dev/null
+++ b/solr/licenses/jackcess-encrypt-NOTICE.txt
@@ -0,0 +1,2 @@
+Jackcess: http://jackcess.sourceforge.net/
+Copyright (C) 2008-2016 James Ahlborn

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jcl-over-slf4j-1.7.24.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jcl-over-slf4j-1.7.24.jar.sha1 b/solr/licenses/jcl-over-slf4j-1.7.24.jar.sha1
new file mode 100644
index 0000000..b3f8afc
--- /dev/null
+++ b/solr/licenses/jcl-over-slf4j-1.7.24.jar.sha1
@@ -0,0 +1 @@
+e6a8629079856a2aa7862c6327ccf6dd1988d7fc

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jcl-over-slf4j-1.7.7.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jcl-over-slf4j-1.7.7.jar.sha1 b/solr/licenses/jcl-over-slf4j-1.7.7.jar.sha1
deleted file mode 100644
index 6309ee8..0000000
--- a/solr/licenses/jcl-over-slf4j-1.7.7.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-56003dcd0a31deea6391b9e2ef2f2dc90b205a92

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jdom-1.0.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jdom-1.0.jar.sha1 b/solr/licenses/jdom-1.0.jar.sha1
deleted file mode 100644
index b7db64f..0000000
--- a/solr/licenses/jdom-1.0.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-a2ac1cd690ab4c80defe7f9bce14d35934c35cec

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jdom-2.0.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jdom-2.0.2.jar.sha1 b/solr/licenses/jdom-2.0.2.jar.sha1
new file mode 100644
index 0000000..2a2eb7a
--- /dev/null
+++ b/solr/licenses/jdom-2.0.2.jar.sha1
@@ -0,0 +1 @@
+d06c71e0df0ac4b94deb737718580ccce22d92e8

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jul-to-slf4j-1.7.24.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jul-to-slf4j-1.7.24.jar.sha1 b/solr/licenses/jul-to-slf4j-1.7.24.jar.sha1
new file mode 100644
index 0000000..38c351e
--- /dev/null
+++ b/solr/licenses/jul-to-slf4j-1.7.24.jar.sha1
@@ -0,0 +1 @@
+25a2be668cb2ad1d05d76c0773df73b4b53617fd

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/jul-to-slf4j-1.7.7.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/jul-to-slf4j-1.7.7.jar.sha1 b/solr/licenses/jul-to-slf4j-1.7.7.jar.sha1
deleted file mode 100644
index 98b2acc..0000000
--- a/solr/licenses/jul-to-slf4j-1.7.7.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-def21bc1a6e648ee40b41a84f1db443132913105

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/metadata-extractor-2.10.1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metadata-extractor-2.10.1.jar.sha1 b/solr/licenses/metadata-extractor-2.10.1.jar.sha1
new file mode 100644
index 0000000..9da3826
--- /dev/null
+++ b/solr/licenses/metadata-extractor-2.10.1.jar.sha1
@@ -0,0 +1 @@
+e1852a8f519dbb0196fdb41bf2c584a8858189f8

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/metadata-extractor-2.9.1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metadata-extractor-2.9.1.jar.sha1 b/solr/licenses/metadata-extractor-2.9.1.jar.sha1
deleted file mode 100644
index fcdcfd6..0000000
--- a/solr/licenses/metadata-extractor-2.9.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-53fdf22be10c9d426ec63431c7342895bc642261

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/pdfbox-2.0.6.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/pdfbox-2.0.6.jar.sha1 b/solr/licenses/pdfbox-2.0.6.jar.sha1
deleted file mode 100644
index 9c93973..0000000
--- a/solr/licenses/pdfbox-2.0.6.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-68616a583c5f9b9ba72140364d15a07cd937ce0e

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/pdfbox-2.0.8.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/pdfbox-2.0.8.jar.sha1 b/solr/licenses/pdfbox-2.0.8.jar.sha1
new file mode 100644
index 0000000..9700a41
--- /dev/null
+++ b/solr/licenses/pdfbox-2.0.8.jar.sha1
@@ -0,0 +1 @@
+17bdf273d66f3afe41eedb9d3ab6a7b819c44a0c

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/pdfbox-tools-2.0.6.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/pdfbox-tools-2.0.6.jar.sha1 b/solr/licenses/pdfbox-tools-2.0.6.jar.sha1
deleted file mode 100644
index dfc26a1..0000000
--- a/solr/licenses/pdfbox-tools-2.0.6.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-3b0c2622015c048a29496291cfe44f235f5c7cdf

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/pdfbox-tools-2.0.8.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/pdfbox-tools-2.0.8.jar.sha1 b/solr/licenses/pdfbox-tools-2.0.8.jar.sha1
new file mode 100644
index 0000000..1b7b9dc
--- /dev/null
+++ b/solr/licenses/pdfbox-tools-2.0.8.jar.sha1
@@ -0,0 +1 @@
+1c0ba702f74b2d1baca1f0d5e172738b3c122fac

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-3.17-beta1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-3.17-beta1.jar.sha1 b/solr/licenses/poi-3.17-beta1.jar.sha1
deleted file mode 100644
index 83a318c..0000000
--- a/solr/licenses/poi-3.17-beta1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-d2df762cc8301c257db1fd0f4fc2f37ed9fbbb78

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-3.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-3.17.jar.sha1 b/solr/licenses/poi-3.17.jar.sha1
new file mode 100644
index 0000000..4b5792b
--- /dev/null
+++ b/solr/licenses/poi-3.17.jar.sha1
@@ -0,0 +1 @@
+0ae92292a2043888b40d418da97dc0b669fde326

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-ooxml-3.17-beta1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-ooxml-3.17-beta1.jar.sha1 b/solr/licenses/poi-ooxml-3.17-beta1.jar.sha1
deleted file mode 100644
index f16c93f..0000000
--- a/solr/licenses/poi-ooxml-3.17-beta1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-96f537614c5f5ec232fb8832313280dcb93c59ab

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-ooxml-3.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-ooxml-3.17.jar.sha1 b/solr/licenses/poi-ooxml-3.17.jar.sha1
new file mode 100644
index 0000000..b124d1e
--- /dev/null
+++ b/solr/licenses/poi-ooxml-3.17.jar.sha1
@@ -0,0 +1 @@
+07d8c44407178b73246462842bf1e206e99c8e0a

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-ooxml-schemas-3.17-beta1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-ooxml-schemas-3.17-beta1.jar.sha1 b/solr/licenses/poi-ooxml-schemas-3.17-beta1.jar.sha1
deleted file mode 100644
index fc99957..0000000
--- a/solr/licenses/poi-ooxml-schemas-3.17-beta1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-99811dc063afea4cde813726ba6f45f724bf2c3b

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-ooxml-schemas-3.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-ooxml-schemas-3.17.jar.sha1 b/solr/licenses/poi-ooxml-schemas-3.17.jar.sha1
new file mode 100644
index 0000000..51eb0c0
--- /dev/null
+++ b/solr/licenses/poi-ooxml-schemas-3.17.jar.sha1
@@ -0,0 +1 @@
+890114bfa82f5b6380ea0e9b0bf49b0af797b414

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-scratchpad-3.17-beta1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-scratchpad-3.17-beta1.jar.sha1 b/solr/licenses/poi-scratchpad-3.17-beta1.jar.sha1
deleted file mode 100644
index a2b7c33..0000000
--- a/solr/licenses/poi-scratchpad-3.17-beta1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-d4ad39b023c377ec534ab25205344eb79da4996b

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/poi-scratchpad-3.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/poi-scratchpad-3.17.jar.sha1 b/solr/licenses/poi-scratchpad-3.17.jar.sha1
new file mode 100644
index 0000000..b23f0e8
--- /dev/null
+++ b/solr/licenses/poi-scratchpad-3.17.jar.sha1
@@ -0,0 +1 @@
+85d86a0e26c7f5c0db4ee63e8c7728e51c5d64ce

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/rome-utils-1.5.1.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/rome-utils-1.5.1.jar.sha1 b/solr/licenses/rome-utils-1.5.1.jar.sha1
new file mode 100644
index 0000000..bc388b9
--- /dev/null
+++ b/solr/licenses/rome-utils-1.5.1.jar.sha1
@@ -0,0 +1 @@
+3a3d6473a2f5d55fb31bf6c269af963fdea13b54

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/rome-utils-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/rome-utils-LICENSE-ASL.txt b/solr/licenses/rome-utils-LICENSE-ASL.txt
new file mode 100644
index 0000000..f43cdb1
--- /dev/null
+++ b/solr/licenses/rome-utils-LICENSE-ASL.txt
@@ -0,0 +1,14 @@
+Copyright 2004 Sun Microsystems, Inc.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/rome-utils-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/rome-utils-NOTICE.txt b/solr/licenses/rome-utils-NOTICE.txt
new file mode 100644
index 0000000..caefccb
--- /dev/null
+++ b/solr/licenses/rome-utils-NOTICE.txt
@@ -0,0 +1 @@
+Copyright 2004 Sun Microsystems, Inc.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/slf4j-api-1.7.24.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/slf4j-api-1.7.24.jar.sha1 b/solr/licenses/slf4j-api-1.7.24.jar.sha1
new file mode 100644
index 0000000..e2722e7
--- /dev/null
+++ b/solr/licenses/slf4j-api-1.7.24.jar.sha1
@@ -0,0 +1 @@
+3f6b4bd4f8dbe8d4bea06d107a3826469b85c3e9

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/slf4j-api-1.7.7.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/slf4j-api-1.7.7.jar.sha1 b/solr/licenses/slf4j-api-1.7.7.jar.sha1
deleted file mode 100644
index 1a82218..0000000
--- a/solr/licenses/slf4j-api-1.7.7.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-2b8019b6249bb05d81d3a3094e468753e2b21311

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/slf4j-log4j12-1.7.24.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/slf4j-log4j12-1.7.24.jar.sha1 b/solr/licenses/slf4j-log4j12-1.7.24.jar.sha1
new file mode 100644
index 0000000..b8ec050
--- /dev/null
+++ b/solr/licenses/slf4j-log4j12-1.7.24.jar.sha1
@@ -0,0 +1 @@
+309d7b29b14573c05d4c69cc7e44da0cb17212d3

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/slf4j-log4j12-1.7.7.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/slf4j-log4j12-1.7.7.jar.sha1 b/solr/licenses/slf4j-log4j12-1.7.7.jar.sha1
deleted file mode 100644
index c328cb3..0000000
--- a/solr/licenses/slf4j-log4j12-1.7.7.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-58f588119ffd1702c77ccab6acb54bfb41bed8bd

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/slf4j-simple-1.7.24.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/slf4j-simple-1.7.24.jar.sha1 b/solr/licenses/slf4j-simple-1.7.24.jar.sha1
new file mode 100644
index 0000000..043482c
--- /dev/null
+++ b/solr/licenses/slf4j-simple-1.7.24.jar.sha1
@@ -0,0 +1 @@
+d9841ffd9d794ab26446df2c46a2ab2b8d2a183e

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/slf4j-simple-1.7.7.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/slf4j-simple-1.7.7.jar.sha1 b/solr/licenses/slf4j-simple-1.7.7.jar.sha1
deleted file mode 100644
index 2da962e..0000000
--- a/solr/licenses/slf4j-simple-1.7.7.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8095d0b9f7e0a9cd79a663c740e0f8fb31d0e2c8

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-core-1.16.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-core-1.16.jar.sha1 b/solr/licenses/tika-core-1.16.jar.sha1
deleted file mode 100644
index e8ee028..0000000
--- a/solr/licenses/tika-core-1.16.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-7b75cb2b65f6d014b6a3e4793835f5759168c34e

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-core-1.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-core-1.17.jar.sha1 b/solr/licenses/tika-core-1.17.jar.sha1
new file mode 100644
index 0000000..684dd94
--- /dev/null
+++ b/solr/licenses/tika-core-1.17.jar.sha1
@@ -0,0 +1 @@
+b450102c2aee98107474d2f92661d947b9cef183

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-java7-1.16.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-java7-1.16.jar.sha1 b/solr/licenses/tika-java7-1.16.jar.sha1
deleted file mode 100644
index 97a229d..0000000
--- a/solr/licenses/tika-java7-1.16.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-b4d7dc8dd2f592824cb0fa0677d08fe9bbee2ab4

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-java7-1.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-java7-1.17.jar.sha1 b/solr/licenses/tika-java7-1.17.jar.sha1
new file mode 100644
index 0000000..c58ee42
--- /dev/null
+++ b/solr/licenses/tika-java7-1.17.jar.sha1
@@ -0,0 +1 @@
+e97e815dd1212c01b9e97d02402d57ab5bc835cc

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-parsers-1.16.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-parsers-1.16.jar.sha1 b/solr/licenses/tika-parsers-1.16.jar.sha1
deleted file mode 100644
index 4b16dde..0000000
--- a/solr/licenses/tika-parsers-1.16.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-bececafbe32c013eae8d3f3cf10d28b136a6f9d7

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-parsers-1.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-parsers-1.17.jar.sha1 b/solr/licenses/tika-parsers-1.17.jar.sha1
new file mode 100644
index 0000000..e4065fe
--- /dev/null
+++ b/solr/licenses/tika-parsers-1.17.jar.sha1
@@ -0,0 +1 @@
+4277c54fcaed542fbc8a0001fdb4c23baccc0132

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-xmp-1.16.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-xmp-1.16.jar.sha1 b/solr/licenses/tika-xmp-1.16.jar.sha1
deleted file mode 100644
index 0c4ce3a..0000000
--- a/solr/licenses/tika-xmp-1.16.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-06db0bda27d4edf452c5a72a29f78805bfd23a0f

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/tika-xmp-1.17.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/tika-xmp-1.17.jar.sha1 b/solr/licenses/tika-xmp-1.17.jar.sha1
new file mode 100644
index 0000000..2ae91e9
--- /dev/null
+++ b/solr/licenses/tika-xmp-1.17.jar.sha1
@@ -0,0 +1 @@
+f7845ff9b484799b2c57055f7253f816e48b4f66

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/xmpcore-5.1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/xmpcore-5.1.2.jar.sha1 b/solr/licenses/xmpcore-5.1.2.jar.sha1
deleted file mode 100644
index 19af7ca..0000000
--- a/solr/licenses/xmpcore-5.1.2.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-55615fa2582424e38705487d1d3969af8554f637

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e321d70/solr/licenses/xmpcore-5.1.3.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/xmpcore-5.1.3.jar.sha1 b/solr/licenses/xmpcore-5.1.3.jar.sha1
new file mode 100644
index 0000000..bab9957
--- /dev/null
+++ b/solr/licenses/xmpcore-5.1.3.jar.sha1
@@ -0,0 +1 @@
+57e70c3b10ff269fff9adfa7a31d61af0df30757


[08/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
index 8f65255..9732616 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
@@ -26,9 +26,11 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.cloud.DistributedQueue;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.cloud.overseer.OverseerAction;
@@ -79,56 +81,26 @@ public class SplitShardCmd implements Cmd {
   }
 
   public boolean split(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
-    String collectionName = message.getStr("collection");
-    String slice = message.getStr(ZkStateReader.SHARD_ID_PROP);
     boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
+    String collectionName = message.getStr(CoreAdminParams.COLLECTION);
 
     log.info("Split shard invoked");
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     zkStateReader.forceUpdateCollection(collectionName);
+    AtomicReference<String> slice = new AtomicReference<>();
+    slice.set(message.getStr(ZkStateReader.SHARD_ID_PROP));
 
     String splitKey = message.getStr("split.key");
-    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
-
     DocCollection collection = clusterState.getCollection(collectionName);
-    DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
-    PolicyHelper.SessionWrapper sessionWrapper = null;
-
-
-    Slice parentSlice;
 
-    if (slice == null) {
-      if (router instanceof CompositeIdRouter) {
-        Collection<Slice> searchSlices = router.getSearchSlicesSingle(splitKey, new ModifiableSolrParams(), collection);
-        if (searchSlices.isEmpty()) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to find an active shard for split.key: " + splitKey);
-        }
-        if (searchSlices.size() > 1) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Splitting a split.key: " + splitKey + " which spans multiple shards is not supported");
-        }
-        parentSlice = searchSlices.iterator().next();
-        slice = parentSlice.getName();
-        log.info("Split by route.key: {}, parent shard is: {} ", splitKey, slice);
-      } else {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Split by route key can only be used with CompositeIdRouter or subclass. Found router: "
-                + router.getClass().getName());
-      }
-    } else {
-      parentSlice = collection.getSlice(slice);
-    }
+    PolicyHelper.SessionWrapper sessionWrapper = null;
 
-    if (parentSlice == null) {
-      // no chance of the collection being null because ClusterState#getCollection(String) would have thrown
-      // an exception already
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
-    }
+    Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
 
     // find the leader for the shard
     Replica parentShardLeader = null;
     try {
-      parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice, 10000);
+      parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice.get(), 10000);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
     }
@@ -140,80 +112,13 @@ public class SplitShardCmd implements Cmd {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
     }
 
-    DocRouter.Range range = parentSlice.getRange();
-    if (range == null) {
-      range = new PlainIdRouter().fullRange();
-    }
+    List<DocRouter.Range> subRanges = new ArrayList<>();
+    List<String> subSlices = new ArrayList<>();
+    List<String> subShardNames = new ArrayList<>();
 
-    List<DocRouter.Range> subRanges = null;
-    String rangesStr = message.getStr(CoreAdminParams.RANGES);
-    if (rangesStr != null) {
-      String[] ranges = rangesStr.split(",");
-      if (ranges.length == 0 || ranges.length == 1) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least two ranges specified to split a shard");
-      } else {
-        subRanges = new ArrayList<>(ranges.length);
-        for (int i = 0; i < ranges.length; i++) {
-          String r = ranges[i];
-          try {
-            subRanges.add(DocRouter.DEFAULT.fromString(r));
-          } catch (Exception e) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception in parsing hexadecimal hash range: " + r, e);
-          }
-          if (!subRanges.get(i).isSubsetOf(range)) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Specified hash range: " + r + " is not a subset of parent shard's range: " + range.toString());
-          }
-        }
-        List<DocRouter.Range> temp = new ArrayList<>(subRanges); // copy to preserve original order
-        Collections.sort(temp);
-        if (!range.equals(new DocRouter.Range(temp.get(0).min, temp.get(temp.size() - 1).max))) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Specified hash ranges: " + rangesStr + " do not cover the entire range of parent shard: " + range);
-        }
-        for (int i = 1; i < temp.size(); i++) {
-          if (temp.get(i - 1).max + 1 != temp.get(i).min) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Specified hash ranges: " + rangesStr
-                + " either overlap with each other or " + "do not cover the entire range of parent shard: " + range);
-          }
-        }
-      }
-    } else if (splitKey != null) {
-      if (router instanceof CompositeIdRouter) {
-        CompositeIdRouter compositeIdRouter = (CompositeIdRouter) router;
-        subRanges = compositeIdRouter.partitionRangeByKey(splitKey, range);
-        if (subRanges.size() == 1) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey
-              + " has a hash range that is exactly equal to hash range of shard: " + slice);
-        }
-        for (DocRouter.Range subRange : subRanges) {
-          if (subRange.min == subRange.max) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey + " must be a compositeId");
-          }
-        }
-        log.info("Partitioning parent shard " + slice + " range: " + parentSlice.getRange() + " yields: " + subRanges);
-        rangesStr = "";
-        for (int i = 0; i < subRanges.size(); i++) {
-          DocRouter.Range subRange = subRanges.get(i);
-          rangesStr += subRange.toString();
-          if (i < subRanges.size() - 1) rangesStr += ',';
-        }
-      }
-    } else {
-      // todo: fixed to two partitions?
-      subRanges = router.partitionRange(2, range);
-    }
+    String rangesStr = fillRanges(ocmh.cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames);
 
     try {
-      List<String> subSlices = new ArrayList<>(subRanges.size());
-      List<String> subShardNames = new ArrayList<>(subRanges.size());
-      String nodeName = parentShardLeader.getNodeName();
-      for (int i = 0; i < subRanges.size(); i++) {
-        String subSlice = slice + "_" + i;
-        subSlices.add(subSlice);
-        String subShardName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), collection, subSlice, Replica.Type.NRT);
-        subShardNames.add(subShardName);
-      }
 
       boolean oldShardsDeleted = false;
       for (String subSlice : subSlices) {
@@ -252,6 +157,7 @@ public class SplitShardCmd implements Cmd {
 
       final String asyncId = message.getStr(ASYNC);
       Map<String, String> requestMap = new HashMap<>();
+      String nodeName = parentShardLeader.getNodeName();
 
       for (int i = 0; i < subRanges.size(); i++) {
         String subSlice = subSlices.get(i);
@@ -300,6 +206,8 @@ public class SplitShardCmd implements Cmd {
         ocmh.addReplica(clusterState, new ZkNodeProps(propMap), results, null);
       }
 
+      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
+
       ocmh.processResponses(results, shardHandler, true, "SPLITSHARD failed to create subshard leaders", asyncId, requestMap);
 
       for (String subShardName : subShardNames) {
@@ -388,7 +296,7 @@ public class SplitShardCmd implements Cmd {
 
       // TODO: change this to handle sharding a slice into > 2 sub-shards.
 
-      List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh,
+      List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh.cloudManager,
           clusterState,
           new ArrayList<>(clusterState.getLiveNodes()),
           collectionName,
@@ -401,15 +309,15 @@ public class SplitShardCmd implements Cmd {
       for (ReplicaPosition replicaPosition : replicaPositions) {
         String sliceName = replicaPosition.shard;
         String subShardNodeName = replicaPosition.node;
-        String shardName = collectionName + "_" + sliceName + "_replica" + (replicaPosition.index);
+        String solrCoreName = collectionName + "_" + sliceName + "_replica" + (replicaPosition.index);
 
-        log.info("Creating replica shard " + shardName + " as part of slice " + sliceName + " of collection "
+        log.info("Creating replica shard " + solrCoreName + " as part of slice " + sliceName + " of collection "
             + collectionName + " on " + subShardNodeName);
 
         ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
             ZkStateReader.COLLECTION_PROP, collectionName,
             ZkStateReader.SHARD_ID_PROP, sliceName,
-            ZkStateReader.CORE_NAME_PROP, shardName,
+            ZkStateReader.CORE_NAME_PROP, solrCoreName,
             ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
             ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
             ZkStateReader.NODE_NAME_PROP, subShardNodeName,
@@ -421,7 +329,7 @@ public class SplitShardCmd implements Cmd {
         propMap.put(COLLECTION_PROP, collectionName);
         propMap.put(SHARD_ID_PROP, sliceName);
         propMap.put("node", subShardNodeName);
-        propMap.put(CoreAdminParams.NAME, shardName);
+        propMap.put(CoreAdminParams.NAME, solrCoreName);
         // copy over property params:
         for (String key : message.keySet()) {
           if (key.startsWith(COLL_PROP_PREFIX)) {
@@ -478,7 +386,7 @@ public class SplitShardCmd implements Cmd {
         DistributedQueue inQueue = Overseer.getStateUpdateQueue(zkStateReader.getZkClient());
         Map<String, Object> propMap = new HashMap<>();
         propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
-        propMap.put(slice, Slice.State.INACTIVE.toString());
+        propMap.put(slice.get(), Slice.State.INACTIVE.toString());
         for (String subSlice : subSlices) {
           propMap.put(subSlice, Slice.State.ACTIVE.toString());
         }
@@ -507,7 +415,7 @@ public class SplitShardCmd implements Cmd {
 
       log.info("Successfully created all replica shards for all sub-slices " + subSlices);
 
-      ocmh.commit(results, slice, parentShardLeader);
+      ocmh.commit(results, slice.get(), parentShardLeader);
 
       return true;
     } catch (SolrException e) {
@@ -519,4 +427,116 @@ public class SplitShardCmd implements Cmd {
       if (sessionWrapper != null) sessionWrapper.release();
     }
   }
+
+  public static Slice getParentSlice(ClusterState clusterState, String collectionName, AtomicReference<String> slice, String splitKey) {
+    DocCollection collection = clusterState.getCollection(collectionName);
+    DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
+
+    Slice parentSlice;
+
+    if (slice.get() == null) {
+      if (router instanceof CompositeIdRouter) {
+        Collection<Slice> searchSlices = router.getSearchSlicesSingle(splitKey, new ModifiableSolrParams(), collection);
+        if (searchSlices.isEmpty()) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to find an active shard for split.key: " + splitKey);
+        }
+        if (searchSlices.size() > 1) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "Splitting a split.key: " + splitKey + " which spans multiple shards is not supported");
+        }
+        parentSlice = searchSlices.iterator().next();
+        slice.set(parentSlice.getName());
+        log.info("Split by route.key: {}, parent shard is: {} ", splitKey, slice);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Split by route key can only be used with CompositeIdRouter or subclass. Found router: "
+                + router.getClass().getName());
+      }
+    } else {
+      parentSlice = collection.getSlice(slice.get());
+    }
+
+    if (parentSlice == null) {
+      // no chance of the collection being null because ClusterState#getCollection(String) would have thrown
+      // an exception already
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
+    }
+    return parentSlice;
+  }
+
+  public static String fillRanges(SolrCloudManager cloudManager, ZkNodeProps message, DocCollection collection, Slice parentSlice,
+                                List<DocRouter.Range> subRanges, List<String> subSlices, List<String> subShardNames) {
+    String splitKey = message.getStr("split.key");
+    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) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least two ranges specified to split a shard");
+      } else {
+        for (int i = 0; i < ranges.length; i++) {
+          String r = ranges[i];
+          try {
+            subRanges.add(DocRouter.DEFAULT.fromString(r));
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception in parsing hexadecimal hash range: " + r, e);
+          }
+          if (!subRanges.get(i).isSubsetOf(range)) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                "Specified hash range: " + r + " is not a subset of parent shard's range: " + range.toString());
+          }
+        }
+        List<DocRouter.Range> temp = new ArrayList<>(subRanges); // copy to preserve original order
+        Collections.sort(temp);
+        if (!range.equals(new DocRouter.Range(temp.get(0).min, temp.get(temp.size() - 1).max))) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "Specified hash ranges: " + rangesStr + " do not cover the entire range of parent shard: " + range);
+        }
+        for (int i = 1; i < temp.size(); i++) {
+          if (temp.get(i - 1).max + 1 != temp.get(i).min) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Specified hash ranges: " + rangesStr
+                + " either overlap with each other or " + "do not cover the entire range of parent shard: " + range);
+          }
+        }
+      }
+    } else if (splitKey != null) {
+      if (router instanceof CompositeIdRouter) {
+        CompositeIdRouter compositeIdRouter = (CompositeIdRouter) router;
+        List<DocRouter.Range> tmpSubRanges = compositeIdRouter.partitionRangeByKey(splitKey, range);
+        if (tmpSubRanges.size() == 1) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey
+              + " has a hash range that is exactly equal to hash range of shard: " + parentSlice.getName());
+        }
+        for (DocRouter.Range subRange : tmpSubRanges) {
+          if (subRange.min == subRange.max) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey + " must be a compositeId");
+          }
+        }
+        subRanges.addAll(tmpSubRanges);
+        log.info("Partitioning parent shard " + parentSlice.getName() + " range: " + parentSlice.getRange() + " yields: " + subRanges);
+        rangesStr = "";
+        for (int i = 0; i < subRanges.size(); i++) {
+          DocRouter.Range subRange = subRanges.get(i);
+          rangesStr += subRange.toString();
+          if (i < subRanges.size() - 1) rangesStr += ',';
+        }
+      }
+    } else {
+      // todo: fixed to two partitions?
+      subRanges.addAll(router.partitionRange(2, range));
+    }
+
+    for (int i = 0; i < subRanges.size(); i++) {
+      String subSlice = parentSlice.getName() + "_" + i;
+      subSlices.add(subSlice);
+      String subShardName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), collection, subSlice, Replica.Type.NRT);
+      subShardNames.add(subShardName);
+    }
+    return rangesStr;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
index e61536b..039067c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
@@ -133,13 +133,13 @@ public class AutoScaling {
    */
   public static class TriggerFactoryImpl extends TriggerFactory {
 
-    private final SolrCloudManager dataProvider;
+    private final SolrCloudManager cloudManager;
     private final SolrResourceLoader loader;
 
-    public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager dataProvider) {
-      Objects.requireNonNull(dataProvider);
+    public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager cloudManager) {
+      Objects.requireNonNull(cloudManager);
       Objects.requireNonNull(loader);
-      this.dataProvider = dataProvider;
+      this.cloudManager = cloudManager;
       this.loader = loader;
     }
 
@@ -150,11 +150,11 @@ public class AutoScaling {
       }
       switch (type) {
         case NODEADDED:
-          return new NodeAddedTrigger(name, props, loader, dataProvider);
+          return new NodeAddedTrigger(name, props, loader, cloudManager);
         case NODELOST:
-          return new NodeLostTrigger(name, props, loader, dataProvider);
+          return new NodeLostTrigger(name, props, loader, cloudManager);
         case SEARCHRATE:
-          return new SearchRateTrigger(name, props, loader, dataProvider);
+          return new SearchRateTrigger(name, props, loader, cloudManager);
         default:
           throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
index 7ca0d36..86fd04a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
@@ -36,26 +36,23 @@ import java.util.stream.Stream;
 
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
-import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
 import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
 import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
-import org.apache.solr.cloud.ZkDistributedQueueFactory;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.RequestHandlerUtils;
 import org.apache.solr.request.SolrQueryRequest;
@@ -63,7 +60,7 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.PermissionNameProvider;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -80,15 +77,18 @@ import static org.apache.solr.common.params.CommonParams.JSON;
 public class AutoScalingHandler extends RequestHandlerBase implements PermissionNameProvider {
   public static final String HANDLER_PATH = "/admin/autoscaling";
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  protected final CoreContainer container;
+  protected final SolrCloudManager cloudManager;
+  protected final SolrResourceLoader loader;
   private final List<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
   private static Set<String> singletonCommands = Stream.of("set-cluster-preferences", "set-cluster-policy")
       .collect(collectingAndThen(toSet(), Collections::unmodifiableSet));
-  private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
 
+  private final TimeSource timeSource;
 
-  public AutoScalingHandler(CoreContainer container) {
-    this.container = container;
+  public AutoScalingHandler(SolrCloudManager cloudManager, SolrResourceLoader loader) {
+    this.cloudManager = cloudManager;
+    this.loader = loader;
+    this.timeSource = cloudManager.getTimeSource();
     Map<String, String> map = new HashMap<>(2);
     map.put(NAME, "compute_plan");
     map.put(CLASS, "solr.ComputePlanAction");
@@ -116,7 +116,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
         }
 
-        AutoScalingConfig autoScalingConf = container.getZkController().zkStateReader.getAutoScalingConfig();
+        AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
         if (parts.size() == 2)  {
           autoScalingConf.writeMap(new MapWriter.EntryWriter() {
 
@@ -154,21 +154,14 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
 
 
   private void handleSuggestions(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) throws IOException {
-    try (CloudSolrClient build = new CloudSolrClient.Builder()
-        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
-        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
-      DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
-      rsp.getValues().add("suggestions",
-          PolicyHelper.getSuggestions(autoScalingConf, new SolrClientCloudManager(queueFactory, build)));
-    }
-
-
+    rsp.getValues().add("suggestions",
+        PolicyHelper.getSuggestions(autoScalingConf, cloudManager));
   }
 
   public void processOps(SolrQueryRequest req, SolrQueryResponse rsp, List<CommandOperation> ops)
       throws KeeperException, InterruptedException, IOException {
     while (true) {
-      AutoScalingConfig initialConfig = container.getZkController().zkStateReader.getAutoScalingConfig();
+      AutoScalingConfig initialConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
       AutoScalingConfig currentConfig = initialConfig;
       for (CommandOperation op : ops) {
         switch (op.name) {
@@ -216,7 +209,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
 
       if (!currentConfig.equals(initialConfig)) {
         // update in ZK
-        if (zkSetAutoScalingConfig(container.getZkController().getZkStateReader(), currentConfig)) {
+        if (setAutoScalingConfig(currentConfig)) {
           break;
         } else {
           // someone else updated the config, get the latest one and re-apply our ops
@@ -244,12 +237,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
 
   private void handleDiagnostics(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) throws IOException {
     Policy policy = autoScalingConf.getPolicy();
-    try (CloudSolrClient build = new CloudSolrClient.Builder()
-        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
-        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
-      DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
-      rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, new SolrClientCloudManager(queueFactory, build)));
-    }
+    rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, cloudManager));
   }
 
   private AutoScalingConfig handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
@@ -302,7 +290,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
       return currentConfig;
     }
 
-    container.getZkController().getZkStateReader().getClusterState().forEachCollection(coll -> {
+    cloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> {
       if (policyName.equals(coll.getPolicyName()))
         op.addError(StrUtils.formatString("policy : {0} is being used by collection {1}", policyName, coll.getName()));
     });
@@ -470,7 +458,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
     // validate that we can load the listener class
     // todo allow creation from blobstore
     try {
-      container.getResourceLoader().findClass(listenerClass, TriggerListener.class);
+      loader.findClass(listenerClass, TriggerListener.class);
     } catch (Exception e) {
       log.warn("error loading listener class ", e);
       op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
@@ -535,7 +523,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
       }
       String klass = action.get(CLASS);
       try {
-        container.getResourceLoader().findClass(klass, TriggerAction.class);
+        loader.findClass(klass, TriggerAction.class);
       } catch (Exception e) {
         log.warn("Could not load class : ", e);
         op.addError("Action not found: " + klass + " " + e.getMessage());
@@ -632,26 +620,22 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
   }
 
 
-  private boolean zkSetAutoScalingConfig(ZkStateReader reader, AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
+  private boolean setAutoScalingConfig(AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
     verifyAutoScalingConf(currentConfig);
     try {
-      reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion(), true);
-    } catch (KeeperException.BadVersionException bve) {
+      cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion());
+    } catch (BadVersionException bve) {
       // somebody else has changed the configuration so we must retry
       return false;
     }
+    //log.debug("-- saved version " + currentConfig.getZkVersion() + ": " + currentConfig);
     return true;
   }
 
   private void verifyAutoScalingConf(AutoScalingConfig autoScalingConf) throws IOException {
-    try (CloudSolrClient build = new CloudSolrClient.Builder()
-        .withHttpClient(container.getUpdateShardHandler().getHttpClient())
-        .withZkHost(container.getZkController().getZkServerAddress()).build()) {
-      DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
-      Policy.Session session = autoScalingConf.getPolicy()
-          .createSession(new SolrClientCloudManager(queueFactory, build));
-      log.debug("Verified autoscaling configuration");
-    }
+    Policy.Session session = autoScalingConf.getPolicy()
+        .createSession(cloudManager);
+    log.debug("Verified autoscaling configuration");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
index ccffea7..8cce976 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
@@ -33,8 +33,10 @@ import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -57,12 +59,13 @@ public class ComputePlanAction extends TriggerActionBase {
       if (autoScalingConf.isEmpty()) {
         throw new Exception("Action: " + getName() + " executed but no policy is configured");
       }
-
-      //    Policy.Session session = cloudManager.getDistribStateManager().getAutoScalingConfig().getPolicy().createSession(cloudManager);
-//    return new PolicyHelper.SessionWrapper(session, null);
       PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(cloudManager);
       Policy.Session session = sessionWrapper.get();
-//      Policy policy = autoScalingConf.getPolicy();
+      if (log.isTraceEnabled()) {
+        ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
+        log.trace("-- session: {}", session);
+        log.trace("-- state: {}", state);
+      }
       try {
         Suggester suggester = getSuggester(session, event, cloudManager);
         while (true) {
@@ -99,12 +102,10 @@ public class ComputePlanAction extends TriggerActionBase {
       case NODEADDED:
         suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
             .hint(Suggester.Hint.TARGET_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
-        log.debug("NODEADDED Created suggester with targetNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
         break;
       case NODELOST:
         suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
             .hint(Suggester.Hint.SRC_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
-        log.debug("NODELOST Created suggester with srcNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
         break;
       case SEARCHRATE:
         Map<String, Map<String, Double>> hotShards = (Map<String, Map<String, Double>>)event.getProperty(AutoScalingParams.SHARD);
@@ -125,13 +126,10 @@ public class ComputePlanAction extends TriggerActionBase {
         } else {
           // collection || shard || replica -> ADDREPLICA
           suggester = session.getSuggester(CollectionParams.CollectionAction.ADDREPLICA);
-          Set<String> collections = new HashSet<>();
-          // XXX improve this when AddReplicaSuggester supports coll_shard hint
-          hotReplicas.forEach(r -> collections.add(r.getCollection()));
-          hotShards.forEach((coll, shards) -> collections.add(coll));
-          hotCollections.forEach((coll, rate) -> collections.add(coll));
-          for (String coll : collections) {
-            suggester = suggester.hint(Suggester.Hint.COLL, coll);
+          Set<Pair> collectionShards = new HashSet<>();
+          hotShards.forEach((coll, shards) -> shards.forEach((s, r) -> collectionShards.add(new Pair(coll, s))));
+          for (Pair<String, String> colShard : collectionShards) {
+            suggester = suggester.hint(Suggester.Hint.COLL_SHARD, colShard);
           }
         }
         break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
index ebe0660..47b3440 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
@@ -56,7 +56,7 @@ public class ExecutePlanAction extends TriggerActionBase {
   @Override
   public void process(TriggerEvent event, ActionContext context) throws Exception {
     log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
-    SolrCloudManager dataProvider = context.getCloudManager();
+    SolrCloudManager cloudManager = context.getCloudManager();
     List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations");
     if (operations == null || operations.isEmpty()) {
       log.info("No operations to execute for event: {}", event);
@@ -64,7 +64,7 @@ public class ExecutePlanAction extends TriggerActionBase {
     }
     try {
       for (SolrRequest operation : operations) {
-        log.info("Executing operation: {}", operation.getParams());
+        log.debug("Executing operation: {}", operation.getParams());
         try {
           SolrResponse response = null;
           int counter = 0;
@@ -73,22 +73,22 @@ public class ExecutePlanAction extends TriggerActionBase {
             // waitForFinalState so that the end effects of operations are visible
             req.setWaitForFinalState(true);
             String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
-            String znode = saveAsyncId(dataProvider.getDistribStateManager(), event, asyncId);
+            String znode = saveAsyncId(cloudManager.getDistribStateManager(), event, asyncId);
             log.debug("Saved requestId: {} in znode: {}", asyncId, znode);
             // TODO: find a better way of using async calls using dataProvider API !!!
             req.setAsyncId(asyncId);
-            SolrResponse asyncResponse = dataProvider.request(req);
+            SolrResponse asyncResponse = cloudManager.request(req);
             if (asyncResponse.getResponse().get("error") != null) {
               throw new IOException("" + asyncResponse.getResponse().get("error"));
             }
             asyncId = (String)asyncResponse.getResponse().get("requestid");
-            CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(dataProvider, asyncId,
+            CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(cloudManager, asyncId,
                 DEFAULT_TASK_TIMEOUT_SECONDS, TimeUnit.SECONDS);
             if (statusResponse != null) {
               RequestStatusState state = statusResponse.getRequestStatus();
               if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
                 try {
-                  dataProvider.getDistribStateManager().removeData(znode, -1);
+                  cloudManager.getDistribStateManager().removeData(znode, -1);
                 } catch (Exception e) {
                   log.warn("Unexpected exception while trying to delete znode: " + znode, e);
                 }
@@ -96,7 +96,7 @@ public class ExecutePlanAction extends TriggerActionBase {
               response = statusResponse;
             }
           } else {
-            response = dataProvider.request(operation);
+            response = cloudManager.request(operation);
           }
           NamedList<Object> result = response.getResponse();
           context.getProperties().compute("responses", (s, o) -> {
@@ -106,16 +106,15 @@ public class ExecutePlanAction extends TriggerActionBase {
             return responses;
           });
         } catch (IOException e) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to talk to ZooKeeper", e);
-//        } catch (InterruptedException e) {
-//          Thread.currentThread().interrupt();
-//          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Unexpected exception executing operation: " + operation.getParams(), e);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
         } catch (Exception e) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "Unexpected exception executing operation: " + operation.getParams(), e);
         }
-
-//        counter++;
       }
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -124,22 +123,22 @@ public class ExecutePlanAction extends TriggerActionBase {
   }
 
 
-  static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager dataProvider, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
+  static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager cloudManager, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
     long timeoutSeconds = timeUnit.toSeconds(duration);
     RequestStatusState state = RequestStatusState.NOT_FOUND;
     CollectionAdminRequest.RequestStatusResponse statusResponse = null;
     for (int i = 0; i < timeoutSeconds; i++) {
       try {
-        statusResponse = (CollectionAdminRequest.RequestStatusResponse)dataProvider.request(CollectionAdminRequest.requestStatus(requestId));
+        statusResponse = (CollectionAdminRequest.RequestStatusResponse)cloudManager.request(CollectionAdminRequest.requestStatus(requestId));
         state = statusResponse.getRequestStatus();
         if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
-          log.info("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
-          dataProvider.request(CollectionAdminRequest.deleteAsyncId(requestId));
+          log.debug("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
+          cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
           return statusResponse;
         } else if (state == RequestStatusState.NOT_FOUND) {
           // the request for this id was never actually submitted! no harm done, just bail out
           log.warn("Task with requestId={} was not found on overseer", requestId);
-          dataProvider.request(CollectionAdminRequest.deleteAsyncId(requestId));
+          cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
           return statusResponse;
         }
       } catch (Exception e) {
@@ -154,11 +153,12 @@ public class ExecutePlanAction extends TriggerActionBase {
           throw e;
         }
         log.error("Unexpected Exception while querying status of requestId=" + requestId, e);
+        throw e;
       }
       if (i > 0 && i % 5 == 0) {
         log.debug("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
       }
-      TimeUnit.SECONDS.sleep(5);
+      cloudManager.getTimeSource().sleep(5000);
     }
     log.debug("Task with requestId={} did not complete within 5 minutes. Last state={}", requestId, state);
     return statusResponse;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
index 2003cb8..0388472 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
@@ -66,8 +66,8 @@ public class HttpTriggerListener extends TriggerListenerBase {
   private boolean followRedirects;
 
   @Override
-  public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-    super.init(dataProvider, config);
+  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+    super.init(cloudManager, config);
     urlTemplate = (String)config.properties.get("url");
     payloadTemplate = (String)config.properties.get("payload");
     contentType = (String)config.properties.get("contentType");
@@ -148,7 +148,7 @@ public class HttpTriggerListener extends TriggerListenerBase {
     });
     headers.put("Content-Type", type);
     try {
-      dataProvider.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
+      cloudManager.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
     } catch (IOException e) {
       LOG.warn("Exception sending request for event " + event, e);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
index 2ef1b1d..a1e19e9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
@@ -35,7 +35,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,8 +44,6 @@ import org.slf4j.LoggerFactory;
 public class NodeAddedTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final TimeSource timeSource;
-
   private Set<String> lastLiveNodes;
 
   private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
@@ -55,7 +52,6 @@ public class NodeAddedTrigger extends TriggerBase {
                           SolrResourceLoader loader,
                           SolrCloudManager cloudManager) {
     super(TriggerEventType.NODEADDED, name, properties, loader, cloudManager);
-    this.timeSource = TimeSource.CURRENT_TIME;
     lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
     log.debug("Initial livenodes: {}", lastLiveNodes);
     log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
@@ -71,7 +67,7 @@ public class NodeAddedTrigger extends TriggerBase {
         // don't add nodes that have since gone away
         if (lastLiveNodes.contains(n)) {
           log.debug("Adding node from marker path: {}", n);
-          nodeNameVsTimeAdded.put(n, timeSource.getTime());
+          nodeNameVsTimeAdded.put(n, cloudManager.getTimeSource().getTime());
         }
         removeMarker(n);
       });
@@ -131,7 +127,7 @@ public class NodeAddedTrigger extends TriggerBase {
       log.debug("Running NodeAddedTrigger {}", name);
 
       Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
-      log.debug("Found livenodes: {}", newLiveNodes);
+      log.debug("Found livenodes: {}", newLiveNodes.size());
 
       // have any nodes that we were tracking been removed from the cluster?
       // if so, remove them from the tracking map
@@ -142,7 +138,7 @@ public class NodeAddedTrigger extends TriggerBase {
       Set<String> copyOfNew = new HashSet<>(newLiveNodes);
       copyOfNew.removeAll(lastLiveNodes);
       copyOfNew.forEach(n -> {
-        long eventTime = timeSource.getTime();
+        long eventTime = cloudManager.getTimeSource().getTime();
         log.debug("Tracking new node: {} at time {}", n, eventTime);
         nodeNameVsTimeAdded.put(n, eventTime);
       });
@@ -154,7 +150,7 @@ public class NodeAddedTrigger extends TriggerBase {
         Map.Entry<String, Long> entry = it.next();
         String nodeName = entry.getKey();
         Long timeAdded = entry.getValue();
-        long now = timeSource.getTime();
+        long now = cloudManager.getTimeSource().getTime();
         if (TimeUnit.SECONDS.convert(now - timeAdded, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
           nodeNames.add(nodeName);
           times.add(timeAdded);
@@ -163,7 +159,8 @@ public class NodeAddedTrigger extends TriggerBase {
       AutoScaling.TriggerEventProcessor processor = processorRef.get();
       if (!nodeNames.isEmpty()) {
         if (processor != null) {
-          log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name, nodeNames, times, timeSource.getTime());
+          log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name,
+              nodeNames, times, cloudManager.getTimeSource().getTime());
           if (processor.process(new NodeAddedEvent(getEventType(), getName(), times, nodeNames))) {
             // remove from tracking set only if the fire was accepted
             nodeNames.forEach(n -> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
index d53a354..57c76c0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
@@ -34,7 +34,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,8 +43,6 @@ import org.slf4j.LoggerFactory;
 public class NodeLostTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final TimeSource timeSource;
-
   private Set<String> lastLiveNodes;
 
   private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
@@ -54,7 +51,6 @@ public class NodeLostTrigger extends TriggerBase {
                          SolrResourceLoader loader,
                          SolrCloudManager dataProvider) {
     super(TriggerEventType.NODELOST, name, properties, loader, dataProvider);
-    this.timeSource = TimeSource.CURRENT_TIME;
     lastLiveNodes = new HashSet<>(dataProvider.getClusterStateProvider().getLiveNodes());
     log.debug("Initial livenodes: {}", lastLiveNodes);
   }
@@ -69,7 +65,7 @@ public class NodeLostTrigger extends TriggerBase {
         // don't add nodes that have since came back
         if (!lastLiveNodes.contains(n)) {
           log.debug("Adding lost node from marker path: {}", n);
-          nodeNameVsTimeRemoved.put(n, timeSource.getTime());
+          nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTime());
         }
         removeMarker(n);
       });
@@ -122,12 +118,12 @@ public class NodeLostTrigger extends TriggerBase {
       synchronized (this) {
         if (isClosed) {
           log.warn("NodeLostTrigger ran but was already closed");
-          throw new RuntimeException("Trigger has been closed");
+          return;
         }
       }
 
       Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
-      log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes);
+      log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes.size());
 
       // have any nodes that we were tracking been added to the cluster?
       // if so, remove them from the tracking map
@@ -139,7 +135,7 @@ public class NodeLostTrigger extends TriggerBase {
       copyOfLastLiveNodes.removeAll(newLiveNodes);
       copyOfLastLiveNodes.forEach(n -> {
         log.debug("Tracking lost node: {}", n);
-        nodeNameVsTimeRemoved.put(n, timeSource.getTime());
+        nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTime());
       });
 
       // has enough time expired to trigger events for a node?
@@ -149,7 +145,7 @@ public class NodeLostTrigger extends TriggerBase {
         Map.Entry<String, Long> entry = it.next();
         String nodeName = entry.getKey();
         Long timeRemoved = entry.getValue();
-        long now = timeSource.getTime();
+        long now = cloudManager.getTimeSource().getTime();
         if (TimeUnit.SECONDS.convert(now - timeRemoved, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
           nodeNames.add(nodeName);
           times.add(timeRemoved);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
index 3171404..7a9390b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
@@ -57,7 +57,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final SolrCloudManager dataProvider;
+  private final SolrCloudManager cloudManager;
 
   private final CloudConfig cloudConfig;
 
@@ -80,11 +80,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
 
   private AutoScalingConfig autoScalingConfig;
 
-  public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager dataProvider, CloudConfig cloudConfig) {
-    this.dataProvider = dataProvider;
+  public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager, CloudConfig cloudConfig) {
+    this.cloudManager = cloudManager;
     this.cloudConfig = cloudConfig;
-    scheduledTriggers = new ScheduledTriggers(loader, dataProvider);
-    triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, dataProvider);
+    scheduledTriggers = new ScheduledTriggers(loader, cloudManager);
+    triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
   }
 
   @Override
@@ -114,11 +114,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
     // we automatically add a trigger for auto add replicas if it does not exists already
     while (!isClosed)  {
       try {
-        AutoScalingConfig autoScalingConfig = dataProvider.getDistribStateManager().getAutoScalingConfig();
+        AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
         AutoScalingConfig withAutoAddReplicasTrigger = withAutoAddReplicasTrigger(autoScalingConfig);
         if (withAutoAddReplicasTrigger.equals(autoScalingConfig)) break;
         log.debug("Adding .autoAddReplicas trigger");
-        dataProvider.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion());
+        cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion());
         break;
       } catch (BadVersionException bve) {
         // somebody else has changed the configuration so we must retry
@@ -225,7 +225,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
           throw new IllegalStateException("Caught AlreadyClosedException from ScheduledTriggers, but we're not closed yet!", e);
         }
       }
-      DistribStateManager stateManager = dataProvider.getDistribStateManager();
+      DistribStateManager stateManager = cloudManager.getDistribStateManager();
       if (cleanOldNodeLostMarkers) {
         log.debug("-- clean old nodeLost markers");
         try {
@@ -259,7 +259,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
   private void removeNodeMarker(String path, String nodeName) {
     path = path + "/" + nodeName;
     try {
-      dataProvider.getDistribStateManager().removeData(path, -1);
+      cloudManager.getDistribStateManager().removeData(path, -1);
       log.debug("  -- deleted " + path);
     } catch (NoSuchElementException e) {
       // ignore
@@ -297,7 +297,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
       if (isClosed) {
         return;
       }
-      AutoScalingConfig currentConfig = dataProvider.getDistribStateManager().getAutoScalingConfig(watcher);
+      AutoScalingConfig currentConfig = cloudManager.getDistribStateManager().getAutoScalingConfig(watcher);
       log.debug("Refreshing {} with znode version {}", ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, currentConfig.getZkVersion());
       if (znodeVersion >= currentConfig.getZkVersion()) {
         // protect against reordered watcher fires by ensuring that we only move forward

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index 8278977..8ebdf1a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
@@ -38,12 +38,12 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
@@ -78,13 +78,14 @@ import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_SCHEDULE_D
  */
 public class ScheduledTriggers implements Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
-  static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
-  static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
-  static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
+  public static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
+  public static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
+  public static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
+  public static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
 
   static final Map<String, Object> DEFAULT_PROPERTIES = new HashMap<>();
 
+  // Note: values must be all in milliseconds!
   static {
     DEFAULT_PROPERTIES.put(TRIGGER_SCHEDULE_DELAY_SECONDS, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
     DEFAULT_PROPERTIES.put(TRIGGER_COOLDOWN_PERIOD_SECONDS, DEFAULT_COOLDOWN_PERIOD_SECONDS);
@@ -114,7 +115,7 @@ public class ScheduledTriggers implements Closeable {
 
   private final AtomicLong cooldownPeriod = new AtomicLong(TimeUnit.SECONDS.toNanos(DEFAULT_COOLDOWN_PERIOD_SECONDS));
 
-  private final AtomicInteger triggerDelay = new AtomicInteger(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
+  private final AtomicLong triggerDelay = new AtomicLong(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
 
   private final AtomicReference<ActionThrottle> actionThrottle;
 
@@ -136,14 +137,14 @@ public class ScheduledTriggers implements Closeable {
     scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
     scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
     actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
-    actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS)));
+    actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS), cloudManager.getTimeSource()));
     this.cloudManager = cloudManager;
     this.stateManager = cloudManager.getDistribStateManager();
     this.loader = loader;
     queueStats = new Stats();
     listeners = new TriggerListeners();
     // initialize cooldown timer
-    cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
+    cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
   }
 
   /**
@@ -168,7 +169,9 @@ public class ScheduledTriggers implements Closeable {
               scheduledTriggers.forEach((s, scheduledTrigger) -> {
                 if (scheduledTrigger.scheduledFuture.cancel(false)) {
                   scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(
-                      scheduledTrigger, 0, triggerDelay.get(), TimeUnit.SECONDS);
+                      scheduledTrigger, 0,
+                      cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
+                      TimeUnit.MILLISECONDS);
                 } else  {
                   log.debug("Failed to cancel scheduled task: {}", s);
                 }
@@ -188,9 +191,10 @@ public class ScheduledTriggers implements Closeable {
             if (oldThrottle.getLastActionStartedAt() != null) {
               newThrottle = new ActionThrottle("action",
                   minMsBetweenActions,
-                  oldThrottle.getLastActionStartedAt());
+                  oldThrottle.getLastActionStartedAt(),
+                  cloudManager.getTimeSource());
             } else  {
-              newThrottle = new ActionThrottle("action", minMsBetweenActions);
+              newThrottle = new ActionThrottle("action", minMsBetweenActions, cloudManager.getTimeSource());
             }
             this.actionThrottle.set(newThrottle);
             break;
@@ -200,12 +204,17 @@ public class ScheduledTriggers implements Closeable {
     this.autoScalingConfig = autoScalingConfig;
 
     // reset cooldown and actionThrottle
-    cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
+    cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
     actionThrottle.get().reset();
 
     listeners.setAutoScalingConfig(autoScalingConfig);
   }
 
+  @VisibleForTesting
+  void resetActionThrottle() {
+    actionThrottle.get().reset();
+  }
+
   /**
    * Adds a new trigger or replaces an existing one. The replaced trigger, if any, is closed
    * <b>before</b> the new trigger is run. If a trigger is replaced with itself then this
@@ -261,7 +270,7 @@ public class ScheduledTriggers implements Closeable {
       }
       boolean replaying = event.getProperty(TriggerEvent.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEvent.REPLAYING) : false;
       AutoScaling.Trigger source = scheduledSource.trigger;
-      if (source.isClosed()) {
+      if (scheduledSource.isClosed || source.isClosed()) {
         String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s has already been closed", event.toString(), source);
         listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
         log.warn(msg);
@@ -269,7 +278,7 @@ public class ScheduledTriggers implements Closeable {
         return false;
       }
       // reject events during cooldown period
-      if (cooldownStart.get() + cooldownPeriod.get() > System.nanoTime()) {
+      if (cooldownStart.get() + cooldownPeriod.get() > cloudManager.getTimeSource().getTime()) {
         log.debug("-------- Cooldown period - rejecting event: " + event);
         event.getProperties().put(TriggerEvent.COOLDOWN, true);
         listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "In cooldown period.");
@@ -288,8 +297,16 @@ public class ScheduledTriggers implements Closeable {
         listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.STARTED);
         List<TriggerAction> actions = source.getActions();
         if (actions != null) {
+          if (actionExecutor.isShutdown()) {
+            String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the executor has already been closed", event.toString(), source);
+            listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
+            log.warn(msg);
+            // we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
+            return false;
+          }
           actionExecutor.submit(() -> {
             assert hasPendingActions.get();
+            long eventProcessingStart = cloudManager.getTimeSource().getTime();
             log.debug("-- processing actions for " + event);
             try {
               // let the action executor thread wait instead of the trigger thread so we use the throttle here
@@ -324,9 +341,11 @@ public class ScheduledTriggers implements Closeable {
             } catch (Exception e) {
               log.warn("Exception executing actions", e);
             } finally {
-              cooldownStart.set(System.nanoTime());
+              cooldownStart.set(cloudManager.getTimeSource().getTime());
               hasPendingActions.set(false);
             }
+            log.debug("-- processing took {} ms for event id={}",
+                TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getTime() - eventProcessingStart), event.id);
           });
         } else {
           if (enqueued) {
@@ -347,7 +366,9 @@ public class ScheduledTriggers implements Closeable {
       }
     });
     newTrigger.init(); // mark as ready for scheduling
-    scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0, triggerDelay.get(), TimeUnit.SECONDS);
+    scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0,
+        cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
+        TimeUnit.MILLISECONDS);
   }
 
   private void waitForPendingTasks(AutoScaling.Trigger newTrigger, List<TriggerAction> actions) throws AlreadyClosedException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
index 0c6ffd4..ec3110e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
@@ -37,7 +37,6 @@ import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.metrics.SolrCoreMetricManager;
-import org.apache.solr.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +46,6 @@ import org.slf4j.LoggerFactory;
 public class SearchRateTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final TimeSource timeSource;
   private final String handler;
   private final String collection;
   private final String shard;
@@ -63,7 +61,6 @@ public class SearchRateTrigger extends TriggerBase {
                            SolrResourceLoader loader,
                            SolrCloudManager cloudManager) {
     super(TriggerEventType.SEARCHRATE, name, properties, loader, cloudManager);
-    this.timeSource = TimeSource.CURRENT_TIME;
     this.state.put("lastCollectionEvent", lastCollectionEvent);
     this.state.put("lastNodeEvent", lastNodeEvent);
     this.state.put("lastShardEvent", lastShardEvent);
@@ -168,6 +165,9 @@ public class SearchRateTrigger extends TriggerBase {
           });
         });
       });
+      if (metricTags.isEmpty()) {
+        continue;
+      }
       Map<String, Object> rates = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
       rates.forEach((tag, rate) -> {
         ReplicaInfo info = metricTags.get(tag);
@@ -184,7 +184,7 @@ public class SearchRateTrigger extends TriggerBase {
       });
     }
 
-    long now = timeSource.getTime();
+    long now = cloudManager.getTimeSource().getTime();
     // check for exceeded rates and filter out those with less than waitFor from previous events
     Map<String, Double> hotNodes = nodeRates.entrySet().stream()
         .filter(entry -> node.equals(Policy.ANY) || node.equals(entry.getKey()))
@@ -274,7 +274,7 @@ public class SearchRateTrigger extends TriggerBase {
   private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
     Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
     long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
-    log.debug("name=" + name + ", lastTime=" + lastTime + ", elapsed=" + elapsed);
+    log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
     if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
index 34761f2..3282075 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
@@ -72,8 +72,8 @@ public class SystemLogListener extends TriggerListenerBase {
   private boolean enabled = true;
 
   @Override
-  public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-    super.init(dataProvider, config);
+  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+    super.init(cloudManager, config);
     collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
     enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
   }
@@ -119,7 +119,7 @@ public class SystemLogListener extends TriggerListenerBase {
       UpdateRequest req = new UpdateRequest();
       req.add(doc);
       req.setParam(CollectionAdminParams.COLLECTION, collection);
-      dataProvider.request(req);
+      cloudManager.request(req);
     } catch (Exception e) {
       if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
         // relatively benign

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
index f40a49f..311c735 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
@@ -27,7 +27,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.cloud.Stats;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.TimeSource;
+import org.apache.solr.common.util.TimeSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,7 +48,7 @@ public class TriggerEventQueue {
     // TODO: collect stats
     this.delegate = cloudManager.getDistributedQueueFactory().makeQueue(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName);
     this.triggerName = triggerName;
-    this.timeSource = TimeSource.CURRENT_TIME;
+    this.timeSource = cloudManager.getTimeSource();
   }
 
   public boolean offerEvent(TriggerEvent event) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
index 507c77d..61a95db 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
@@ -27,11 +27,11 @@ import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 public abstract class TriggerListenerBase implements TriggerListener {
 
   protected AutoScalingConfig.TriggerListenerConfig config;
-  protected SolrCloudManager dataProvider;
+  protected SolrCloudManager cloudManager;
 
   @Override
-  public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
-    this.dataProvider = dataProvider;
+  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+    this.cloudManager = cloudManager;
     this.config = config;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index c14f180..951c752 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -43,12 +43,12 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 public class CollectionMutator {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  protected final SolrCloudManager dataProvider;
+  protected final SolrCloudManager cloudManager;
   protected final DistribStateManager stateManager;
 
-  public CollectionMutator(SolrCloudManager dataProvider) {
-    this.dataProvider = dataProvider;
-    this.stateManager = dataProvider.getDistribStateManager();
+  public CollectionMutator(SolrCloudManager cloudManager) {
+    this.cloudManager = cloudManager;
+    this.stateManager = cloudManager.getDistribStateManager();
   }
 
   public ZkWriteCommand createShard(final ClusterState clusterState, ZkNodeProps message) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index 035de68..dbcdd3d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -245,7 +245,7 @@ public class ReplicaMutator {
         log.debug("node=" + coreNodeName + " is already registered");
       } else {
         // if coreNodeName is null, auto assign one
-        coreNodeName = Assign.assignNode(stateManager, collection);
+        coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
       }
       message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP,
           coreNodeName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index 6e820b0..6718a80 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -73,7 +73,7 @@ public class SliceMutator {
     if (message.getStr(ZkStateReader.CORE_NODE_NAME_PROP) != null) {
       coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
     } else {
-      coreNodeName = Assign.assignNode(stateManager, collection);
+      coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
     }
     Replica replica = new Replica(coreNodeName,
         makeMap(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 5ed72d7..eb13775 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -540,8 +540,6 @@ public class CoreContainer {
     // may want to add some configuration here in the future
     metricsCollectorHandler.init(null);
 
-    autoScalingHandler = createHandler(AutoScalingHandler.HANDLER_PATH, AutoScalingHandler.class.getName(), AutoScalingHandler.class);
-
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
     securityConfHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
@@ -690,6 +688,10 @@ public class CoreContainer {
 
     if (isZooKeeperAware()) {
       zkSys.getZkController().checkOverseerDesignate();
+      // initialize this handler here when SolrCloudManager is ready
+      autoScalingHandler = new AutoScalingHandler(getZkController().getSolrCloudManager(), loader);
+      containerHandlers.put(AutoScalingHandler.HANDLER_PATH, autoScalingHandler);
+      autoScalingHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AutoScalingHandler.HANDLER_PATH);
     }
     // This is a bit redundant but these are two distinct concepts for all they're accomplished at the same time.
     status |= LOAD_COMPLETE | INITIAL_CORE_LOAD_COMPLETE;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
index 6ef5ebd..d8f3d52 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
@@ -45,6 +45,7 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.update.CdcrUpdateLog;
 import org.apache.solr.util.TimeOut;
@@ -273,7 +274,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
         while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED)  {
           Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
         }
-        TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+        TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
         while (!timeOut.hasTimedOut()) {
           if (closed) {
             log.warn("Cancelling waiting for bootstrap on target: {} shard: {} to complete", targetCollection, shard);
@@ -285,7 +286,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
             try {
               log.info("CDCR bootstrap running for {} seconds, sleeping for {} ms",
                   BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS), BOOTSTRAP_RETRY_DELAY_MS);
-              Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
+              timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
             } catch (InterruptedException e) {
               Thread.currentThread().interrupt();
             }
@@ -309,7 +310,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
               while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED)  {
                 Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
               }
-              timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS); // reset the timer
+              timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
               retries++;
             }
           } else if (status == BootstrapStatus.NOTFOUND || status == BootstrapStatus.CANCELLED) {
@@ -321,13 +322,13 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
               Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
             }
             retries = 1;
-            timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS); // reset the timer
+            timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
           } else if (status == BootstrapStatus.UNKNOWN || status == BootstrapStatus.SUBMITTED) {
             log.info("CDCR bootstrap is " + (status == BootstrapStatus.UNKNOWN ? "unknown" : "submitted"),
                 BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
             // we were not able to query the status on the remote end
             // so just sleep for a bit and try again
-            Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
+            timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
           }
         }
       } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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 de066d5..d339f27 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
@@ -391,7 +391,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     results.add("status", status);
   }
 
-  enum CollectionOperation implements CollectionOp {
+  public enum CollectionOperation implements CollectionOp {
     /**
      * very simple currently, you can pass a template collection, and the new collection is created on
      * every node the template collection is on

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
index d75fccd..217c726 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -93,7 +94,7 @@ public class SchemaManager {
     if (timeout < 1) {
       timeout = 600;
     }
-    TimeOut timeOut = new TimeOut(timeout, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     SolrCore core = req.getCore();
     String errorMsg = "Unable to persist managed schema. ";
     List errors = Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 0e575d6..ba4aa13 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -78,6 +78,7 @@ import org.apache.solr.common.util.JsonSchemaValidator;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.core.CoreContainer;
@@ -347,7 +348,7 @@ public class HttpSolrCall {
       if (rsp.getValues().get("success") == null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Could not auto-create " + SYSTEM_COLL + " collection: "+ Utils.toJSONString(rsp.getValues()));
       }
-      TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       for (; ; ) {
         if (cores.getZkController().getClusterState().getCollectionOrNull(SYSTEM_COLL) != null) {
           break;
@@ -355,7 +356,7 @@ public class HttpSolrCall {
           if (timeOut.hasTimedOut()) {
             throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find " + SYSTEM_COLL + " collection even after 3 seconds");
           }
-          Thread.sleep(50);
+          timeOut.sleep(50);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 9f9b742..de031a2 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -68,6 +68,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.Hash;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.component.RealTimeGetComponent;
@@ -1142,7 +1143,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   private long waitForDependentUpdates(AddUpdateCommand cmd, long versionOnUpdate,
                                boolean isReplayOrPeersync, VersionBucket bucket) throws IOException {
     long lastFoundVersion = 0;
-    TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS); 
+    TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     vinfo.lockForUpdate();
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/java/org/apache/solr/util/IdUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/IdUtils.java b/solr/core/src/java/org/apache/solr/util/IdUtils.java
index a6ea7d2..4f841f3 100644
--- a/solr/core/src/java/org/apache/solr/util/IdUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/IdUtils.java
@@ -19,6 +19,7 @@ package org.apache.solr.util;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.util.StringHelper;
+import org.apache.solr.common.util.TimeSource;
 
 /**
  * Helper class for generating unique ID-s.


[34/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8077: Fixed wrong modulo usage in CheckIndex.

Posted by da...@apache.org.
LUCENE-8077: Fixed wrong modulo usage in CheckIndex.


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

Branch: refs/heads/jira/solr-11702
Commit: b5ad3148358cea4f24abaacadbc9a4ae03f1b7c7
Parents: 9f7f76f
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Dec 20 12:48:04 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 20 12:48:04 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                                           | 5 +++++
 lucene/core/src/java/org/apache/lucene/index/CheckIndex.java | 2 +-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b5ad3148/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d8e1237..98cdcd9 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -94,6 +94,11 @@ Improvements
 
 * LUCENE-8093: TrimFilterFactory implements MultiTermAwareComponent (Alan Woodward)
 
+Bug Fixes
+
+* LUCENE-8077: Fixed bug in how CheckIndex verifies doc-value iterators.
+  (Xiaoshan Sun via Adrien Grand)
+
 ======================= Lucene 7.2.0 =======================
 
 API Changes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b5ad3148/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 028da24..c676568 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -2195,7 +2195,7 @@ public final class CheckIndex implements Closeable {
           throw new RuntimeException("dv iterator field=" + field + ": doc=" + (doc-1) + " has unstable advanceExact");
         }
 
-        if (i % 1 == 0) {
+        if (i % 2 == 0) {
           int doc2 = it2.nextDoc();
           if (doc != doc2) {
             throw new RuntimeException("dv iterator field=" + field + ": doc=" + doc + " was not found through advance() (got: " + doc2 + ")");


[20/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11742: Add documentation for 7.2 release statistical functions

Posted by da...@apache.org.
SOLR-11742: Add documentation for 7.2 release statistical functions


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

Branch: refs/heads/jira/solr-11702
Commit: 9dee35f92d1d2742f120b8e99207230c22cc5074
Parents: 565d13c
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 15 15:32:40 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 15 15:33:05 2017 -0500

----------------------------------------------------------------------
 .../src/statistical-programming.adoc            |   6 +-
 .../src/stream-evaluator-reference.adoc         | 668 ++++++++++++++-----
 2 files changed, 497 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dee35f9/solr/solr-ref-guide/src/statistical-programming.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/statistical-programming.adoc b/solr/solr-ref-guide/src/statistical-programming.adoc
index af56ae0..08693b2 100644
--- a/solr/solr-ref-guide/src/statistical-programming.adoc
+++ b/solr/solr-ref-guide/src/statistical-programming.adoc
@@ -197,12 +197,12 @@ Returns the following response:
 }
 ----
 
-We can nest arrays within arrays to form a matrix:
+We can nest arrays within a matrix function to return matrix:
 
 [source,text]
 ----
-array(array(1, 2, 3),
-      array(4, 5, 6))
+matrix(array(1, 2, 3),
+       array(4, 5, 6))
 ----
 
 Returns the following response:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dee35f9/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/stream-evaluator-reference.adoc b/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
index e9679b0..cf2544c 100644
--- a/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
+++ b/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
@@ -256,24 +256,6 @@ A probability distribution function.
 [source,text]
 binomialDistribution(1000, .5)
 
-== canberraDistance
-
-The `canberraDistance` function calculates the https://en.wikipedia.org/wiki/Canberra_distance[Canberra distance] of two numeric arrays.
-
-=== canberraDistance Parameters
-
-* `numeric array`
-* `numeric array`
-
-=== canberraDistance Returns
-
-A numeric.
-
-=== canberraDistance Syntax
-
-[source,text]
-canberraDistance(numericArray1, numuericArray2))
-
 == cbrt
 
 The `cbrt` function returns the trigonometric cube root of a number.
@@ -309,25 +291,6 @@ ceil(fieldA) // returns the next highest whole number for fieldA.
 if(gt(fieldA,fieldB),ceil(fieldA),ceil(fieldB)) // if fieldA > fieldB then return the ceil of fieldA, else return the ceil of fieldB.
 ----
 
-== chebyshevDistance
-
-The `chebyshevDistance` function calculates the https://en.wikipedia.org/wiki/Chebyshev_distance[Chebyshev distance] of two numeric arrays.
-
-=== chebyshevDistance Parameters
-
-* `numeric array`
-* `numeric array`
-
-=== chebyshevDistance Returns
-
-A numeric.
-
-=== chebyshevDistance Syntax
-
-[source,text]
-chebyshevDistance(numericArray1, numuericArray2))
-
-
 == col
 
 The `col` function returns a numeric array from a list of Tuples. The `col`
@@ -412,22 +375,34 @@ copyOfRange(numericArray, startIndex, endIndex)
 
 == corr
 
-The `corr` function returns the Pearson Product Moment Correlation of two numeric arrays.
+The `corr` function returns the correlation of two numeric arrays or the correlation matrix for a matrix.
 
-=== corr Parameters
+The `corr` function support Pearsons, Kendals and Spearmans correlation.
 
-//TODO fill in details of Parameters
-* `numeric array`
-* `numeric array`
+=== corr Positional Parameters
 
-=== corr Returns
+* `numeric array`: The first numeric array
+* `numeric array`: The second numeric array
+
+OR
+
+* `matrix`: The matrix to compute the correlation matrix for. Note that correlation is computed between the `columns` in the matrix.
 
-A double between -1 and 1.
+=== corr Named Parameters
+
+* `type`: (Optional) pearsons | kendalls | spearmans, Defaults to pearsons.
 
 === corr Syntax
 
 [source,text]
-corr(numericArray1, numericArray2)
+corr(numericArray1, numericArray2) // Compute the Pearsons correlation for two numeric arrays
+corr(numericArray1, numericArray2, type=kendalls) // Compute the Kendalls correlation for two numeric arrays
+corr(matrix) // Compute the Pearsons correlation matrix for a matrix
+corr(matrix, type=spearmans) // Compute the Spearmans correlation matrix for a matrix
+
+=== corr Returns
+
+number | matrix : Either the correlation or correlation matrix.
 
 == cos
 The `cos` function returns the trigonometric cosine of a number.
@@ -467,18 +442,26 @@ cosineSimilarity(numericArray, numericArray)
 
 == cov
 
-The `cov` function returns the covariance of two numeric arrays.
+The `cov` function returns the covariance of two numeric array or the covariance matrix for matrix.
 
 === cov Parameters
 
-//TODO fill in details of Parameters
-* `numeric array`
-* `numeric array`
+* `numeric array`: The first numeric array
+* `numeric array`: The second numeric array
+
+OR
+
+* `matrix`: The matrix to compute the covariance matrix from. Note that covariance is computed between the `columns` in the matrix.
 
 === cov Syntax
 
 [source,text]
-cov(numericArray, numericArray)
+cov(numericArray, numericArray) // Computes the covariance of a two numeric arrays
+cov(matrix) // Computes the covariance matrix for the matrix.
+
+=== cov Returns
+
+number | matrix : Either the covariance or covariance matrix.
 
 == cumulativeProbability
 
@@ -500,6 +483,27 @@ A double: the cumulative probability.
 [source,text]
 cumulativeProbability(normalDistribution(500, 25), 502) // Returns the cumulative probability of the random sample 502 in a normal distribution with a mean of 500 and standard deviation of 25.
 
+== derivative
+
+The `derivative` function returns the https://en.wikipedia.org/wiki/Derivative[derivative] of a function. The derivative function
+can compute the derivative of the <<spline>> function and the <<loess>> function. The derivative can also
+take the derivative of a derivative.
+
+=== derivative Parameters
+
+* `spline` | `loess` | `derivative`: The functions to compute the derivative for.
+
+=== derivative Syntax
+
+[source,text]
+derivative(spline(...))
+derivative(loess(...))
+derivative(derivative(...))
+
+=== derivative Returns
+
+function: The function can be treated as both a `numeric array` and `function`.
+
 == describe
 
 The `describe` function returns a tuple containing the descriptive statistics for an array.
@@ -513,19 +517,55 @@ The `describe` function returns a tuple containing the descriptive statistics fo
 [source,text]
 describe(numericArray)
 
+== diff
+
+The `diff` functions performs https://www.otexts.org/fpp/8/1[time series differencing].
+
+Time series differencing is often used to make a time series stationary before further analysis.
+
+=== diff Parameters
+
+* `numeric array`: The time series data
+* `integer`: (Optional)lag. Defaults to 1.
+
+=== diff Syntax
+
+[source,text]
+diff(numericArray1) // Perform time series differencing with a default lag of 1.
+diff(numericArray1, 30) // Perform time series differencing with a lag of 30.
+
+=== diff Returns
+
+numeric array: The differenced time series data. The size of the array will be equal to (original array size - lag).
+
 == distance
 
-The `distance` function calculates the Euclidian distance of two numeric arrays.
+The `distance` function computes the distance of two numeric arrays or the distance matrix for a matrix.
 
-=== distance Parameters
+=== distance Positional Parameters
 
-* `numeric array`
-* `numeric array`
+* `numeric array` : The first numeric array
+* `numeric array` : The second numeric array
+
+OR
+
+* `matrix` : The matrix to compute the distance matrix for. Note that distance is computed between the `columns` in the matrix.
+
+=== distance Named Parameters
+
+* `type` : (Optional) euclidean | manhattan | canberra | earthMovers. Defaults to euclidean.
 
 === distance Syntax
 
 [source,text]
-distance(numericArray1, numuericArray2))
+distance(numericArray1, numuericArray2) // Computes the euclidean distance for two numeric arrays.
+distance(numericArray1, numuericArray2, type=manhattan) // Computes the manhattan distance for two numeric arrays.
+distance(matrix) // Computes the euclidean distance matrix for a matrix.
+distance(matrix, type=canberra) // Computes the canberra distance matrix for a matrix.
+
+=== distance Returns
+
+number | matrix : Either the distance or distance matrix.
 
 == div
 
@@ -565,24 +605,6 @@ A number.
 [source,text]
 dotProduct(numericArray)
 
-== earthMoversDistance
-
-The `earthMoversDistance` function calculates the https://en.wikipedia.org/wiki/Earth_mover%27s_distance[Earth Movers distance] of two numeric arrays.
-
-=== earthMoversDistance Parameters
-
-* `numeric array`
-* `numeric array`
-
-=== earthMoversDistance Returns
-
-A numeric.
-
-=== earthMoversDistance Syntax
-
-[source,text]
-earthMoversDistance(numericArray1, numericArray2))
-
 == ebeAdd
 
 The `ebeAdd` function performs an element-by-element addition of two numeric arrays.
@@ -847,6 +869,24 @@ A list of tuples containing the frequency information for each discrete value.
 freqTable(integerArray)
 ----
 
+== geometricDistribution
+
+The `geometricDistribution` function returns a https://en.wikipedia.org/wiki/Geometric_distribution[geometric probability distribution] based on its parameters. This function is part of the
+probability distribution framework and is designed to work with the <<sample>>, <<probability>> and <<cumulativeProbability>> functions.
+
+=== geometricDistribution Parameters
+
+* `double`: probability
+
+=== geometricDistribution Syntax
+
+[source,text]
+geometricDistribution(.5) // Creates a geometric distribution with probability of .5
+
+=== geometricDistribution Returns
+
+A probability distribution function
+
 == gammaDistribution
 
 The `gammaDistribution` function returns a https://en.wikipedia.org/wiki/Gamma_distribution[gamma probability distribution] based on its parameters. This function is part of the
@@ -866,6 +906,23 @@ A probability distribution function,
 [source,text]
 gammaDistribution(1, 10)
 
+== grandSum
+
+The `grandSum` function sums all the values in a matrix.
+
+=== grandSum Parameters
+
+* `matrix`: The matrix to operate on
+
+=== grandSum Syntax
+
+[source,text]
+grandSum(matrix)
+
+=== grandSum Returns
+
+number: the sum of all the values in the matrix.
+
 == gt
 
 The `gt` function will return whether the first parameter is greater than the second parameter. The function accepts numeric or string parameters, but will fail to execute if all the parameters are not of the same type. That is, all are String or all are Numeric. If any any parameters are null then an error will be raised. Returns a boolean value.
@@ -962,38 +1019,47 @@ if(gt(fieldA,5), fieldA, 5) // if fieldA > 5 then fieldA else 5
 if(eq(fieldB,null), null, div(fieldA,fieldB)) // if fieldB is null then null else fieldA / fieldB
 ----
 
+== length
 
-== kendallsCorr
-
-The `kendallsCorr` function returns the https://en.wikipedia.org/wiki/Kendall_rank_correlation_coefficient[Kendall's Tau-b Rank Correlation] of two numeric arrays.
+The `length` function returns the length of a numeric array.
 
-=== kendallsCorr Parameters
+=== length Parameters
 
-* `numeric array`
+//TODO fill in details of Parameters
 * `numeric array`
 
-=== kendalsCorr Returns
+=== length Syntax
 
-A double between -1 and 1.
+[source,text]
+length(numericArray)
 
-=== kendalsCorr Syntax
+== loess
 
-[source,text]
-kendallsCorr(numericArray1, numericArray2)
+The `leoss` function is a smoothing curve fitter which uses a https://en.wikipedia.org/wiki/Local_regression[local regression] algorithm.
+Unlike the <<spline>> function which touches each control point, the loess function puts a smooth curve through
+the control points without having to touch the control points. The loess result can be used by the <<derivative>> function to produce smooth derivatives from
+data that is not smooth.
 
-== length
+=== loess Positional Parameters
 
-The `length` function returns the length of a numeric array.
+* `numeric array`: (Optional) x values. If omitted a sequence will be created for the x values.
+* `numeric array`: y values
 
-=== length Parameters
+=== loess Named Parameters
 
-//TODO fill in details of Parameters
-* `numeric array`
+* `bandwidth` : (Optional) The percent of the data points to use when drawing the local regression line, defaults to .25. Decreasing the bandwidth increases the number of curves that loess can fit.
+* `robustIterations` : (Optional)The number of iterations used to smooth outliers, defaults to 2.
 
-=== length Syntax
+=== loess Syntax
 
 [source,text]
-length(numericArray)
+loess(yValues) // This creates the xValues automatically and fits a smooth curve through the data points.
+loess(xValues, yValues) // This will fit a smooth curve through the data points.
+loess(xValues, yValues, bandwidth=.15) // This will fit a smooth curve through the data points using 15 percent of the data points for each local regression line.
+
+=== loess Returns
+
+function : The function can be treated as both a `numeric array` of the smoothed data points and `function`.
 
 == log
 
@@ -1096,23 +1162,42 @@ lteq(fieldA,val(foo)) fieldA <= "foo"
 lteq(add(fieldA,fieldB),6) // fieldA + fieldB <= 6
 ----
 
-== manhattanDistance
+== markovChain
 
-The `manhattanDistance` function calculates the https://en.wiktionary.org/wiki/Manhattan_distance[Manhattan distance] of two numeric arrays.
+The `markovChain` function can be used to perform https://en.wikipedia.org/wiki/Markov_chain[Markov Chain] simulations.
+The markovChain function takes as its parameter a https://en.wikipedia.org/wiki/Stochastic_matrix[transition matrix] and
+returns a mathematical model that can be sampled using the <<sample>> function. Each sample taken
+from the Markov Chain represents the current state of system.
 
-=== manhattanDistance Parameters
+=== markovChain Parameters
 
-* `numeric array`
-* `numeric array`
+* `matrix`: Transition matrix
 
-=== manhattanDistance Returns
+=== markovChain Syntax
 
-A numeric.
+[source,text]
+sample(markovChain(transitionMatrix), 5)  // This creates a Markov Chain given a specific transition matrix. The sample function takes 5 samples from the Markov Chain, representing the next five states of the system.
+
+=== markovChain Returns
+
+Markov Chain model: The Markoff Chain model can be used with <<sample>> function.
+
+== matrix
 
-=== manhattanDistance Syntax
+The matrix function returns a https://en.wikipedia.org/wiki/Matrix_(mathematics)[matrix] which can be operated on by functions that support matrix operations.
+
+=== matrix Parameters
+
+* `numeric array` ...: One or more numeric arrays that will be the rows of the matrix.
+
+=== matrix Syntax
 
 [source,text]
-manhattanDistance(numericArray1, numuericArray2))
+matrix(numericArray1, numericArray2, numericArray3) // Returns a matrix with three rows of data: numericaArray1, numericArray2, numericArray3
+
+=== matrix Returns
+
+matrix
 
 == meanDifference
 
@@ -1134,6 +1219,32 @@ A numeric.
 meanDifference(numericArray, numericArray)
 ----
 
+== minMaxScale
+
+The `minMaxScale` function scales numeric arrays within a min and max value.
+By default minMaxScale scales between 0 and 1. The minMaxScale function can operate on
+both numeric arrays and matrices.
+
+When operating on a matrix the minMaxScale function operates on each row of the matrix.
+
+=== minMaxScale Parameters
+
+* `numeric array` | `matrix` : The array or matrix to scale
+* `double` : (Optional) The min value. Defaults to 0.
+* `double` : (Optional) The max value. Defaults to 1.
+
+=== minMaxScale Syntax
+
+[source,text]
+minMaxScale(numericArray) // scale a numeric array between 0 and 1
+minMaxScale(numericArray, 0, 100) // scale a numeric array between 1 and 100
+minMaxScale(matrix) // Scale each row in a matrix between 0 and 1
+minMaxScale(matrix, 0, 100) // Scale each row in a matrix between 0 and 100
+
+=== minMaxScale Returns
+
+numeric array or matrix
+
 == mod
 The `mod` function returns the remainder (modulo) of the first parameter divided by the second parameter.
 
@@ -1265,19 +1376,27 @@ A probability distribution function.
 [source,text]
 normalDistribution(mean, stddev)
 
-== normalize
+== normalizeSum
 
-The `normalize` function normalizes a numeric array so that values within the array
-have a mean of 0 and standard deviation of 1.
+The `normalizeSum` function scales numeric arrays so that they sum to 1.
+The normalizeSum function can operate on both numeric arrays and matrices.
 
-=== normalize Parameters
+When operating on a matrix the normalizeSum function operates on each row of the matrix.
 
-* `numeric array`
+=== normalizeSum Parameters
+
+* `numeric array` | matrix
 
-=== normalize Syntax
+=== normalizeSum Syntax
 
 [source,text]
-normalize(numericArray)
+normalizeSum(numericArray)
+normalizeSum(matrix)
+
+=== normalizeSum Returns
+
+numeric array | matrix
+
 
 == not
 
@@ -1298,6 +1417,27 @@ not(fieldA) // true if fieldA is false else false
 not(eq(fieldA,fieldB)) // true if fieldA != fieldB
 ----
 
+== olsRegress
+
+The `olsRegress` function performs https://en.wikipedia.org/wiki/Ordinary_least_squares[ordinary least squares], multivariate, linear regression.
+
+The `olsRegress` function returns a single Tuple containing the regression model with estimated regression parameters, RSquared and regression diagnostics.
+
+The output of olsRegress can be used with the <<predict>> function to predict values based on the regression model.
+
+=== olsRegress Parameters
+
+* `matrix`: The regressor observation matrix. Each row in the matrix represents a single multi-variate regressor observation. Note that there is no need to add an initial unitary column (column of 1's) when specifying a model including an intercept term, this column will be added automatically.
+* `numeric array`: The outcomes array which matches up with each row in the regressor observation matrix.
+
+=== olsRegress Syntax
+
+olsRegress(matrix, numericArray) // This performs the olsRegression analysis on given regressor matrix and outcome array.
+
+=== olsRegress Returns
+
+Tuple: The regression model including the estimated regression parameters and diagnostics.
+
 == or
 
 The `or` function will return the logical OR of at least 2 boolean parameters. The function will fail to execute if any parameters are non-boolean or null. Returns a boolean value.
@@ -1361,27 +1501,6 @@ polyFit(yValues) // This creates the xValues automatically and fits a curve thro
 polyFit(yValues, 5) // This creates the xValues automatically and fits a curve through the data points using a 5 degree polynomial.
 polyFit(xValues, yValues, 5) // This will fit a curve through the data points using a 5 degree polynomial.
 
-== polyfitDerivative
-
-The `polyfitDerivative` function returns the derivative of the curve created by the polynomial curve fitter.
-
-=== polyfitDerivative Parameters
-
-* `numeric array`: (Optional) x values. If omitted a sequence will be created for the x values.
-* `numeric array`: y values
-* `integer`: (Optional) polynomial degree. Defaults to 3.
-
-=== polyfitDerivative Returns
-
-A numeric array: The curve for the derivative created by the polynomial curve fitter.
-
-=== polyfitDerivative Syntax
-
-[source,text]
-polyfitDerivative(yValues) // This creates the xValues automatically and returns the polyfit derivative
-polyfitDerivative(yValues, 5) //  This creates the xValues automatically and fits a curve through the data points using a 5 degree polynomial and returns the polyfit derivative.
-polyfitDerivative(xValues, yValues, 5) // This will fit a curve through the data points using a 5 degree polynomial and returns the polyfit derivative.
-
 == pow
 The `pow` function returns the value of its first parameter raised to the power of its second parameter.
 
@@ -1404,19 +1523,27 @@ if(gt(fieldA,fieldB),pow(fieldA,fieldB),pow(fieldB,fieldA)) // if fieldA > field
 
 == predict
 
-The `predict` function predicts the value of an dependent variable based on
-the output of the regress function.
+The `predict` function predicts the value of dependant variables based on regression models or functions.
+
+The `predict` function can predict values based on the output of the following functions:
+
+<<spline>>, <<loess>>, <<regress>>, <<olsRegress>>
+
 
 === predict Parameters
 
-//TODO fill in details of Parameters
-* `regress output`
-* `numeric predictor`
+* `regression model` | `function`: The model or function used for the prediction
+* `number` | `numeric array` | `matrix`: Depending on the regression model or function used, the predictor variable can be a number, numeric array or matrix.
 
 === predict Syntax
 
 [source,text]
-predict(regressOutput, predictor)
+predict(regressModel, number) // predict using the output of the <<regress>> function and single numeric predictor. This will return a single numeric prediction.
+predict(regressModel, numericArray) // predict using the output of the <<regress>> function and a numeric array of predictors. This will return a numeric array of predictions.
+predict(splineFunc, number) // predict using the output of the <<spline>> function and single numeric predictor. This will return a single numeric prediction.
+predict(splineFunc, numericArray) // predict using the output of the <<spline>> function and a numeric array of predictors. This will return a numeric array of predictions.
+predict(olsRegressModel, numericArray) // predict using the output of the <<olsRegress>> function and a numeric array containing one multi-variate predictor. This will return a single numeric prediction.
+predict(olsRegressModel, matrix) // predict using the output of the <<olsRegress>> function and a matrix containing rows of multi-variate predictor arrays. This will return a numeric array of predictions.
 
 == primes
 The `primes` function returns an array of prime numbers starting from a specified number.
@@ -1439,21 +1566,40 @@ primes(100, 2000) // returns 100 primes starting from 2000
 
 == probability
 
-The `probability` function returns the probability of a random variable within a discrete probability distribution.
+The `probability` function returns the probability of a random variable within a probability distribution.
 
-=== probability Parameters
+The `probability` function computes the probability between random variable ranges for both https://en.wikipedia.org/wiki/Probability_distribution#Continuous_probability_distribution[continuous] and
+https://en.wikipedia.org/wiki/Probability_distribution#Discrete_probability_distribution[discrete] probability distributions.
 
-* `discrete probability distribution`: poissonDistribution | binomialDistribution | uniformDistribution | enumeratedDistribution
-* `integer`: Value of the random variable to compute the probability for.
+The `probability` function can compute probabilities for a specific random variable for
+discrete probability distributions only.
 
-=== probability Returns
+The supported continuous distribution functions are:
+<<normalDistribution>>, <<logNormalDistribution>>, <<betaDistribution>>, <<gammaDistribution>>,
+<<empiricalDistribution>>, <<triangularDistribution>>, <<weibullDistribution>>,
+<<uniformDistribution>>, <<constantDistribution>>
 
-A double: the probability.
+The supported discreet distributions are:
+<<poissonDistribution>>, <<binomialDistribution>>, <<enumeratedDistribution>>, <<zipFDistribution>>,
+<<geometricDistribution>>, <<uniformIntegerDistribution>>
+
+=== probability Parameters
+
+* `probability distribution`: the probability distribution to compute the probability from.
+* `number`: low value of the range.
+* `number`: (Optional for discrete probability distributions) high value of the range. If the high range is omitted then the probability function will compute a probability for the low range value.
 
 === probability Syntax
 
 [source,text]
 probability(poissonDistribution(10), 7) // Returns the probability of a random sample of 7 in a poisson distribution with a mean of 10.
+probability(normalDistribution(10, 2), 7.5, 8.5) // Returns the probability between the range of 7.5 to 8.5 for a normal distribution with a mean of 10 and standard deviation of 2.
+
+
+=== probability Returns
+
+double: probability
+
 
 == rank
 
@@ -1507,28 +1653,6 @@ The result of this expression is also used by the `<<predict>>` and `<<residuals
 [source,text]
 regress(numericArray1, numericArray2)
 
-== residuals
-
-The `residuals` function takes three parameters: a simple regression model, an array of predictor values
-and an array of actual values. The residuals function applies the simple regression model to the
-array of predictor values and computes a predictions array. The predicted values array is then
-subtracted from the actual value array to compute the residuals array.
-
-=== residuals Parameters
-
-* `regress output`
-* `numeric array`: The array of predictor values
-* `numeric array`: The array of actual values
-
-=== residuals Returns
-
-A numeric array of residuals.
-
-=== residuals Syntax
-
-[source,text]
-residuals(regressOutput, numericArray, numericArray)
-
 == rev
 
 The `rev` function reverses the order of a numeric array.
@@ -1561,11 +1685,11 @@ if(gt(fieldA,fieldB),sqrt(fieldA),sqrt(fieldB)) // if fieldA > fieldB then retur
 
 == sample
 
-The `sample` function can be used to draw random samples from a probability distribution.
+The `sample` function can be used to draw random samples from a probability distribution or Markov Chain.
 
 === sample Parameters
 
-* `probability distribution`: The distribution to sample.
+* `probability distribution` | `Markov Chain` : The distribution or Markov Chain to sample.
 * `integer`: (Optional) Sample size. Defaults to 1.
 
 === sample Returns
@@ -1577,6 +1701,87 @@ Either a single numeric random sample, or a numeric array depending on the sampl
 [source,text]
 sample(poissonDistribution(5)) // Returns a single random sample from a poissonDistribution with mean of 5.
 sample(poissonDistribution(5), 1000) // Returns 1000 random samples from poissonDistribution with a mean of 5.
+sample(markovChain(transitionMatrix), 1000) // Returns 1000 random samples from a Markov Chain.
+
+== scalarAdd
+
+The `scalarAdd` function adds a scalar value to every value in a numeric array or matrix.
+When working with numeric arrays, `scalarAdd` returns a new array with the new values. When working
+with a matrix, `scalarAdd` returns a new matrix with new values.
+
+=== scalarAdd Parameters
+
+number: value to add
+numeric array | matrix: the numeric array or matrix to add the value to.
+
+=== scalarAdd Syntax
+
+scalarAdd(number, numericArray) // Adds the number to each element in the number in the array.
+scalarAdd(number, matrix) // Adds the number to each value in a matrix
+
+=== scalarAdd Returns
+
+numericArray | matrix: Depending on what is being operated on.
+
+== scalarDivide
+
+The `scalarDivide` function divides each number in numeric array or matrix by a scalar value.
+When working with numeric arrays, `scalarDivide` returns a new array with the new values. When working
+with a matrix, `scalarDivide` returns a new matrix with new values.
+
+=== scalarDivide Parameters
+
+number : value to divide by
+numeric array | matrix : the numeric array or matrix to divide by the value to.
+
+=== scalarDivide Syntax
+
+scalarDivide(number, numericArray) // Divides each element in the numeric array by the number.
+scalarDivide(number, matrix) // Divides each element in the matrix by the number.
+
+=== scalarDivide Returns
+
+numericArray | matrix: depending on what is being operated on.
+
+== scalarMultiply
+
+The `scalarMultiply` function multiplies each element in a numeric array or matrix by a
+scalar value. When working with numeric arrays, `scalarMultiply` returns a new array with the new values. When working
+with a matrix, `scalarMultiply` returns a new matrix with new values.
+
+=== scalarMultiply Parameters
+
+number: value to divide by
+numeric array | matrix: the numeric array or matrix to divide by the value to.
+
+=== scalarMultiply Syntax
+
+scalarMultiply(number, numericArray) // Multiplies each element in the numeric array by the number.
+scalarMultiply(number, matrix) // Multiplies each element in the matrix by the number.
+
+=== scalarMultiply Returns
+
+numericArray | matrix: depending on what is being operated on
+
+== scalarSubtract
+
+The `scalarSubtract` function subtracts a scalar value from every value in a numeric array or matrix.
+When working with numeric arrays, `scalarSubtract` returns a new array with the new values. When working
+with a matrix, `scalarSubtract` returns a new matrix with new values.
+
+=== scalarSubtract Parameters
+
+number : value to add
+numeric array | matrix : the numeric array or matrix to subtract the value from.
+
+=== scalarSubtract Syntax
+
+scalarSubtract(number, numericArray) // Subtracts the number from each element in the number in the array.
+scalarSubtract(number, matrix) // Subtracts the number from each value in a matrix
+
+=== scalarSubtract Returns
+
+numericArray | matrix: depending on what is being operated on.
 
 == scale
 
@@ -1625,23 +1830,27 @@ sine(fieldA) // returns the sine for fieldA.
 if(gt(fieldA,fieldB),sin(fieldA),sin(fieldB)) // if fieldA > fieldB then return the sine of fieldA, else return the sine of fieldB
 ----
 
-== spearmansCorr
+== spline
 
-The `spearmansCorr` function returns the https://en.wikipedia.org/wiki/Spearman%27s_rank_correlation_coefficient[Spearmans Rank Correlation] of two numeric arrays.
+The `spline` function performs a cubic spline interpolation (https://en.wikiversity.org/wiki/Cubic_Spline_Interpolation) of a curve
+given a set of x,y coordinates. The return value of the spline function is an
+interpolation function which can be used to <<predict>> values along the curve and generate a <<derivative>> of
+the curve.
 
-=== spearmansCorr Parameters
+=== spline Parameters
 
-* `numeric array`
-* `numeric array`
+* `numeric array`: (Optional) x values. If omitted a sequence will be created for the x values.
+* `numeric array`: y values
 
-=== spearmansCorr Returns
+=== spline Syntax
 
-A double between -1 and 1.
+[source,text]
+spline(yValues) // This creates the xValues automatically and fits a spline through the data points.
+spline(xValues, yValues) // This will fit a spline through the data points.
 
-=== spearmansCorr Syntax
+=== spline Returns
 
-[source,text]
-spearmansCorr(numericArray1, numericArray2)
+function: the function can be treated as both a `numeric array` and `function`.
 
 == sqrt
 
@@ -1660,6 +1869,25 @@ sqrt(fieldA) // returns the square root for fieldA.
 if(gt(fieldA,fieldB),sqrt(fieldA),sqrt(fieldB)) // if fieldA > fieldB then return the sqrt of fieldA, else return the sqrt of fieldB
 ----
 
+
+== standardize
+
+The `standardize` function standardizes a numeric array so that values within the array
+have a mean of 0 and standard deviation of 1.
+
+=== standardize Parameters
+
+* `numeric array`: the array to standardize
+
+=== standardize Syntax
+
+[source,text]
+standardize(numericArray)
+
+=== standardize Returns
+
+numeric array: the standardized values
+
 == sub
 
 The `sub` function will take 2 or more numeric values and subtract them, from left to right. The `sub` function will fail to execute if any of the values are non-numeric. If a null value is found then `null` will be returned as the result.
@@ -1705,6 +1933,78 @@ A numeric.
 sumDifference(numericArray, numericArray)
 ----
 
+== sumColumns
+
+The `sumColumns` function sums the columns in a matrix and returns a numeric array with the result.
+
+=== sumColumns Parameters
+
+* `matrix`: the matrix to operate on
+
+=== sumColumns Syntax
+
+[source,text]
+sumColumns(matrix)
+
+=== sumColumns Returns
+
+numeric array: the sum of the columns
+
+== sumRows
+
+The `sumRows` function sums the rows in a matrix and returns a numeric array with the result.
+
+=== sumRows Parameters
+
+* `matrix`: the matrix to operate on
+
+=== sumRows Syntax
+
+[source,text]
+sumRows(matrix)
+
+=== sumRows Returns
+
+numeric array: sum of the rows.
+
+== transpose
+
+The `transpose` function https://en.wikipedia.org/wiki/Transpose[transposes] a matrix .
+
+=== transpose Parameters
+
+* `matrix`: the matrix to transpose
+
+=== transpose Syntax
+
+[source,text]
+transpose(matrix)
+
+=== transpose Returns
+
+matrix: the transposed matrix
+
+== triangularDistribution
+
+The `triangularDistribution` function returns a https://en.wikipedia.org/wiki/Triangular_distribution[triangular probability distribution]
+based on its parameters. This function is part of the
+probability distribution framework and is designed to work with the `<<sample>>`, `<<probability>>` and `<<cumulativeProbability>>` functions.
+
+=== triangularDistribution Parameters
+
+* `double` : low value
+* `double` : most likely value
+* `double` : high value
+
+=== triangularDistribution Syntax
+
+[source,text]
+triangularDistribution(10, 15, 20) // A triangular distribution with a low value of 10, most likely value of 15 and high value of 20.
+
+=== triangularDistribution Returns
+
+Probability distribution function
+
 == uniformDistribution
 
 The `uniformDistribution` function returns a https://en.wikipedia.org/wiki/Uniform_distribution_(continuous)[continuous uniform probability distribution]
@@ -1718,7 +2018,7 @@ probability distribution framework and is designed to work with the `<<sample>>`
 
 === uniformDistribution Returns
 
-A probability distribution function.
+Probability distribution function.
 
 === uniformDistribution Syntax
 
@@ -1745,6 +2045,26 @@ A probability distribution function.
 [source,text]
 uniformDistribution(1, 6)
 
+== unitize
+
+The `unitize` function scales numeric arrays to a magnitude of 1, often called https://en.wikipedia.org/wiki/Unit_vector[unit vectors].
+The unitize function can operate on both numeric arrays and matrices.
+
+When operating on a matrix the unitize function unitizes each row of the matrix.
+
+=== unitize Parameters
+
+* numeric array | matrix: The array or matrix to unitize
+
+=== unitize Syntax
+
+unitize(numericArray) // Unitize a numeric array
+unitize(matrix) // Unitize each row in a matrix
+
+=== unitize Returns
+
+numeric array | matrix
+
 == weibullDistribution
 
 The `weibullDistribution` function returns a https://en.wikipedia.org/wiki/Weibull_distribution[Weibull probability distribution]


[25/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11753: minor typos

Posted by da...@apache.org.
SOLR-11753: minor typos


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

Branch: refs/heads/jira/solr-11702
Commit: 33a64afee75afe912ad7761afcfbbfd449c5715c
Parents: c27099b
Author: Cassandra Targett <ct...@apache.org>
Authored: Mon Dec 18 11:54:51 2017 -0600
Committer: Cassandra Targett <ct...@apache.org>
Committed: Mon Dec 18 11:55:29 2017 -0600

----------------------------------------------------------------------
 .../src/stream-evaluator-reference.adoc         | 115 +++++++++++--------
 1 file changed, 64 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/33a64afe/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/stream-evaluator-reference.adoc b/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
index e38284e..e679eac 100644
--- a/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
+++ b/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
@@ -377,7 +377,7 @@ copyOfRange(numericArray, startIndex, endIndex)
 
 The `corr` function returns the correlation of two numeric arrays or the correlation matrix for a matrix.
 
-The `corr` function support Pearsons, Kendals and Spearmans correlation.
+The `corr` function support Pearson's, Kendall's and Spearman's correlations.
 
 === corr Positional Parameters
 
@@ -390,7 +390,7 @@ OR
 
 === corr Named Parameters
 
-* `type`: (Optional) pearsons | kendalls | spearmans, Defaults to pearsons.
+* `type`: (Optional) The type of correlation. Possible values are `pearsons`, `kendalls`, or `spearmans`. The default is `pearsons`.
 
 === corr Syntax
 
@@ -402,7 +402,7 @@ corr(matrix, type=spearmans) // Compute the Spearmans correlation matrix for a m
 
 === corr Returns
 
-number | matrix : Either the correlation or correlation matrix.
+number | matrix: Either the correlation or correlation matrix.
 
 == cos
 The `cos` function returns the trigonometric cosine of a number.
@@ -461,7 +461,7 @@ cov(matrix) // Computes the covariance matrix for the matrix.
 
 === cov Returns
 
-number | matrix : Either the covariance or covariance matrix.
+number | matrix: Either the covariance or covariance matrix.
 
 == cumulativeProbability
 
@@ -525,8 +525,8 @@ Time series differencing is often used to make a time series stationary before f
 
 === diff Parameters
 
-* `numeric array`: The time series data
-* `integer`: (Optional)lag. Defaults to 1.
+* `numeric array`: The time series data.
+* `integer`: (Optional) The lag. Defaults to 1.
 
 === diff Syntax
 
@@ -544,16 +544,16 @@ The `distance` function computes the distance of two numeric arrays or the dista
 
 === distance Positional Parameters
 
-* `numeric array` : The first numeric array
-* `numeric array` : The second numeric array
+* `numeric array`: The first numeric array
+* `numeric array`: The second numeric array
 
 OR
 
-* `matrix` : The matrix to compute the distance matrix for. Note that distance is computed between the `columns` in the matrix.
+* `matrix`: The matrix to compute the distance matrix for. Note that distance is computed between the `columns` in the matrix.
 
 === distance Named Parameters
 
-* `type` : (Optional) euclidean | manhattan | canberra | earthMovers. Defaults to euclidean.
+* `type`: (Optional) The distance type. Possible values are `euclidean`, `manhattan`, `canberra`, or `earthMovers`. The default is `euclidean`.
 
 === distance Syntax
 
@@ -565,7 +565,7 @@ distance(matrix, type=canberra) // Computes the canberra distance matrix for a m
 
 === distance Returns
 
-number | matrix : Either the distance or distance matrix.
+number | matrix: Either the distance or distance matrix.
 
 == div
 
@@ -695,6 +695,7 @@ A probability distribution function.
 
 === empiricalDistribution Syntax
 
+[source,text]
 empiricalDistribution(numericArray)
 
 == enumeratedDistribution
@@ -912,7 +913,7 @@ The `grandSum` function sums all the values in a matrix.
 
 === grandSum Parameters
 
-* `matrix`: The matrix to operate on
+* `matrix`: The matrix to operate on.
 
 === grandSum Syntax
 
@@ -1036,8 +1037,8 @@ length(numericArray)
 == loess
 
 The `leoss` function is a smoothing curve fitter which uses a https://en.wikipedia.org/wiki/Local_regression[local regression] algorithm.
-Unlike the <<spline>> function which touches each control point, the loess function puts a smooth curve through
-the control points without having to touch the control points. The loess result can be used by the <<derivative>> function to produce smooth derivatives from
+Unlike the <<spline>> function which touches each control point, the `loess` function puts a smooth curve through
+the control points without having to touch the control points. The `loess` result can be used by the <<derivative>> function to produce smooth derivatives from
 data that is not smooth.
 
 === loess Positional Parameters
@@ -1047,8 +1048,8 @@ data that is not smooth.
 
 === loess Named Parameters
 
-* `bandwidth` : (Optional) The percent of the data points to use when drawing the local regression line, defaults to .25. Decreasing the bandwidth increases the number of curves that loess can fit.
-* `robustIterations` : (Optional)The number of iterations used to smooth outliers, defaults to 2.
+* `bandwidth`: (Optional) The percent of the data points to use when drawing the local regression line, defaults to .25. Decreasing the bandwidth increases the number of curves that loess can fit.
+* `robustIterations`: (Optional) The number of iterations used to smooth outliers, defaults to 2.
 
 === loess Syntax
 
@@ -1059,7 +1060,7 @@ loess(xValues, yValues, bandwidth=.15) // This will fit a smooth curve through t
 
 === loess Returns
 
-function : The function can be treated as both a `numeric array` of the smoothed data points and `function`.
+function: The function can be treated as both a `numeric array` of the smoothed data points and `function`.
 
 == log
 
@@ -1165,7 +1166,7 @@ lteq(add(fieldA,fieldB),6) // fieldA + fieldB <= 6
 == markovChain
 
 The `markovChain` function can be used to perform https://en.wikipedia.org/wiki/Markov_chain[Markov Chain] simulations.
-The markovChain function takes as its parameter a https://en.wikipedia.org/wiki/Stochastic_matrix[transition matrix] and
+The `markovChain` function takes as its parameter a https://en.wikipedia.org/wiki/Stochastic_matrix[transition matrix] and
 returns a mathematical model that can be sampled using the <<sample>> function. Each sample taken
 from the Markov Chain represents the current state of system.
 
@@ -1221,17 +1222,17 @@ meanDifference(numericArray, numericArray)
 
 == minMaxScale
 
-The `minMaxScale` function scales numeric arrays within a min and max value.
-By default minMaxScale scales between 0 and 1. The minMaxScale function can operate on
+The `minMaxScale` function scales numeric arrays within a minimum and maximum value.
+By default `minMaxScale` scales between 0 and 1. The `minMaxScale` function can operate on
 both numeric arrays and matrices.
 
-When operating on a matrix the minMaxScale function operates on each row of the matrix.
+When operating on a matrix the `minMaxScale` function operates on each row of the matrix.
 
 === minMaxScale Parameters
 
-* `numeric array` | `matrix` : The array or matrix to scale
-* `double` : (Optional) The min value. Defaults to 0.
-* `double` : (Optional) The max value. Defaults to 1.
+* `numeric array` | `matrix`: The array or matrix to scale
+* `double`: (Optional) The min value. Defaults to 0.
+* `double`: (Optional) The max value. Defaults to 1.
 
 === minMaxScale Syntax
 
@@ -1243,7 +1244,7 @@ minMaxScale(matrix, 0, 100) // Scale each row in a matrix between 0 and 100
 
 === minMaxScale Returns
 
-numeric array or matrix
+A numeric array or matrix
 
 == mod
 The `mod` function returns the remainder (modulo) of the first parameter divided by the second parameter.
@@ -1379,13 +1380,13 @@ normalDistribution(mean, stddev)
 == normalizeSum
 
 The `normalizeSum` function scales numeric arrays so that they sum to 1.
-The normalizeSum function can operate on both numeric arrays and matrices.
+The `normalizeSum` function can operate on both numeric arrays and matrices.
 
-When operating on a matrix the normalizeSum function operates on each row of the matrix.
+When operating on a matrix the `normalizeSum` function operates on each row of the matrix.
 
 === normalizeSum Parameters
 
-* `numeric array` | matrix
+* `numeric array` | `matrix`
 
 === normalizeSum Syntax
 
@@ -1423,7 +1424,7 @@ The `olsRegress` function performs https://en.wikipedia.org/wiki/Ordinary_least_
 
 The `olsRegress` function returns a single Tuple containing the regression model with estimated regression parameters, RSquared and regression diagnostics.
 
-The output of olsRegress can be used with the <<predict>> function to predict values based on the regression model.
+The output of `olsRegress` can be used with the <<predict>> function to predict values based on the regression model.
 
 === olsRegress Parameters
 
@@ -1432,6 +1433,7 @@ The output of olsRegress can be used with the <<predict>> function to predict va
 
 === olsRegress Syntax
 
+[source,text]
 olsRegress(matrix, numericArray) // This performs the olsRegression analysis on given regressor matrix and outcome array.
 
 === olsRegress Returns
@@ -1506,8 +1508,8 @@ The `pow` function returns the value of its first parameter raised to the power
 
 === pow Parameters
 
-* `Field Name | Raw Number | Number Evaluator`: Parameter 1
-* `Field Name | Raw Number | Number Evaluator`: Parameter 2
+* `Field Name` | `Raw Number` | `Number Evaluator`: Parameter 1
+* `Field Name` | `Raw Number` | `Number Evaluator`: Parameter 2
 
 === pow Syntax
 
@@ -1523,12 +1525,9 @@ if(gt(fieldA,fieldB),pow(fieldA,fieldB),pow(fieldB,fieldA)) // if fieldA > field
 
 == predict
 
-The `predict` function predicts the value of dependant variables based on regression models or functions.
-
-The `predict` function can predict values based on the output of the following functions:
-
-<<spline>>, <<loess>>, <<regress>>, <<olsRegress>>
+The `predict` function predicts the value of dependent variables based on regression models or functions.
 
+The `predict` function can predict values based on the output of the following functions: <<spline>>, <<loess>>, <<regress>>, <<olsRegress>>.
 
 === predict Parameters
 
@@ -1538,12 +1537,19 @@ The `predict` function can predict values based on the output of the following f
 === predict Syntax
 
 [source,text]
+----
 predict(regressModel, number) // predict using the output of the <<regress>> function and single numeric predictor. This will return a single numeric prediction.
+
 predict(regressModel, numericArray) // predict using the output of the <<regress>> function and a numeric array of predictors. This will return a numeric array of predictions.
+
 predict(splineFunc, number) // predict using the output of the <<spline>> function and single numeric predictor. This will return a single numeric prediction.
+
 predict(splineFunc, numericArray) // predict using the output of the <<spline>> function and a numeric array of predictors. This will return a numeric array of predictions.
+
 predict(olsRegressModel, numericArray) // predict using the output of the <<olsRegress>> function and a numeric array containing one multi-variate predictor. This will return a single numeric prediction.
+
 predict(olsRegressModel, matrix) // predict using the output of the <<olsRegress>> function and a matrix containing rows of multi-variate predictor arrays. This will return a numeric array of predictions.
+----
 
 == primes
 The `primes` function returns an array of prime numbers starting from a specified number.
@@ -1592,9 +1598,11 @@ The supported discreet distributions are:
 === probability Syntax
 
 [source,text]
+----
 probability(poissonDistribution(10), 7) // Returns the probability of a random sample of 7 in a poisson distribution with a mean of 10.
-probability(normalDistribution(10, 2), 7.5, 8.5) // Returns the probability between the range of 7.5 to 8.5 for a normal distribution with a mean of 10 and standard deviation of 2.
 
+probability(normalDistribution(10, 2), 7.5, 8.5) // Returns the probability between the range of 7.5 to 8.5 for a normal distribution with a mean of 10 and standard deviation of 2.
+----
 
 === probability Returns
 
@@ -1672,7 +1680,7 @@ The `round` function returns the closest whole number to the argument.
 
 === round Parameters
 
-* `Field Name | Raw Number | Number Evaluator`: The value to return the square root of.
+* `Field Name` | `Raw Number` | `Number Evaluator`: The value to return the square root of.
 
 === round Syntax
 
@@ -1689,7 +1697,7 @@ The `sample` function can be used to draw random samples from a probability dist
 
 === sample Parameters
 
-* `probability distribution` | `Markov Chain` : The distribution or Markov Chain to sample.
+* `probability distribution` | `Markov Chain`: The distribution or Markov Chain to sample.
 * `integer`: (Optional) Sample size. Defaults to 1.
 
 === sample Returns
@@ -1711,11 +1719,12 @@ with a matrix, `scalarAdd` returns a new matrix with new values.
 
 === scalarAdd Parameters
 
-number: value to add
-numeric array | matrix: the numeric array or matrix to add the value to.
+`number`: value to add
+`numeric array` | `matrix`: the numeric array or matrix to add the value to.
 
 === scalarAdd Syntax
 
+[source,text]
 scalarAdd(number, numericArray) // Adds the number to each element in the number in the array.
 scalarAdd(number, matrix) // Adds the number to each value in a matrix
 
@@ -1731,11 +1740,12 @@ with a matrix, `scalarDivide` returns a new matrix with new values.
 
 === scalarDivide Parameters
 
-number : value to divide by
-numeric array | matrix : the numeric array or matrix to divide by the value to.
+`number`: value to divide by
+`numeric array` | `matrix`: the numeric array or matrix to divide by the value to.
 
 === scalarDivide Syntax
 
+[source,text]
 scalarDivide(number, numericArray) // Divides each element in the numeric array by the number.
 scalarDivide(number, matrix) // Divides each element in the matrix by the number.
 
@@ -1751,11 +1761,12 @@ with a matrix, `scalarMultiply` returns a new matrix with new values.
 
 === scalarMultiply Parameters
 
-number: value to divide by
-numeric array | matrix: the numeric array or matrix to divide by the value to.
+`number`: value to divide by
+`numeric array` | `matrix`: the numeric array or matrix to divide by the value to.
 
 === scalarMultiply Syntax
 
+[source,text]
 scalarMultiply(number, numericArray) // Multiplies each element in the numeric array by the number.
 scalarMultiply(number, matrix) // Multiplies each element in the matrix by the number.
 
@@ -1771,11 +1782,12 @@ with a matrix, `scalarSubtract` returns a new matrix with new values.
 
 === scalarSubtract Parameters
 
-number : value to add
-numeric array | matrix : the numeric array or matrix to subtract the value from.
+`number`: value to add
+`numeric array` | `matrix`: the numeric array or matrix to subtract the value from.
 
 === scalarSubtract Syntax
 
+[source,text]
 scalarSubtract(number, numericArray) // Subtracts the number from each element in the number in the array.
 scalarSubtract(number, matrix) // Subtracts the number from each value in a matrix
 
@@ -1992,9 +2004,9 @@ probability distribution framework and is designed to work with the `<<sample>>`
 
 === triangularDistribution Parameters
 
-* `double` : low value
-* `double` : most likely value
-* `double` : high value
+* `double`: low value
+* `double`: most likely value
+* `double`: high value
 
 === triangularDistribution Syntax
 
@@ -2054,10 +2066,11 @@ When operating on a matrix the unitize function unitizes each row of the matrix.
 
 === unitize Parameters
 
-* numeric array | matrix: The array or matrix to unitize
+* `numeric array` | `matrix`: The array or matrix to unitize
 
 === unitize Syntax
 
+[source,text]
 unitize(numericArray) // Unitize a numeric array
 unitize(matrix) // Unitize each row in a matrix
 


[30/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: tests: remove unused constants

Posted by da...@apache.org.
LUCENE-2899: tests: remove unused constants


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

Branch: refs/heads/jira/solr-11702
Commit: 827595233751d97d8a2408e69be5dbaf004c7d55
Parents: f8c93db
Author: Steve Rowe <sa...@apache.org>
Authored: Mon Dec 18 19:01:46 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Mon Dec 18 19:03:29 2017 -0500

----------------------------------------------------------------------
 .../lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java      | 3 ---
 .../lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java      | 1 -
 2 files changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82759523/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
index 10372d0..814f480 100644
--- a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
@@ -41,9 +41,6 @@ public class TestOpenNLPPOSFilterFactory extends BaseTokenStreamTestCase {
   private static final int[] SENTENCES_endOffsets = {8, 15, 17, 21, 23, 29, 30, 39, 46, 48, 49, 51, 57, 58};
   private static final String[] SENTENCES_posTags
       = {"NN", "NN", "CD", "VBZ", "CD", "NNS", ".", "NN", "NN", "CD", ",", "CD", "NNS", "."};
-  private static final String NAMES2 = "Royal Flash is a tale about Harry Flashman.";
-  private static final String[] NAMES2_punc = {"Royal", "Flash", "is", "a", "tale", "about", "Harry", "Flashman", "."};
-  private static final String[] NAMES2_OUT = { "word", "word", "word", "word", "word", "word", "word", "person", "word" };
 
   private static final String NO_BREAK = "No period";
   private static final String[] NO_BREAK_terms = {"No", "period"};

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82759523/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
index db2bbb2..59a5220 100644
--- a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
@@ -36,7 +36,6 @@ import org.junit.Test;
 public class TestOpenNLPTokenizerFactory extends BaseTokenStreamTestCase {
 
   static private String SENTENCES = "Sentence number 1 has 6 words. Sentence number 2, 5 words.";
-  static private String[] SENTENCES_split = {"Sentence number 1 has 6 words. ", "Sentence number 2, 5 words."};
   static private String[] SENTENCES_punc = {"Sentence", "number", "1", "has", "6", "words", ".", "Sentence", "number", "2", ",", "5", "words", "."};
   static private int[] SENTENCES_startOffsets = {0, 9, 16, 18, 22, 24, 29, 31, 40, 47, 48, 50, 52, 57};
   static private int[] SENTENCES_endOffsets = {8, 15, 17, 21, 23, 29, 30, 39, 46, 48, 49, 51, 57, 58};


[47/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11201: Implement autoscaling trigger for arbitrary metrics that creates events when a given metric breaches a threshold

Posted by da...@apache.org.
SOLR-11201: Implement autoscaling trigger for arbitrary metrics that creates events when a given metric breaches a threshold


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

Branch: refs/heads/jira/solr-11702
Commit: 43f17f7af1831b71f3f53f600bcf04579c4b960a
Parents: 6045066
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Dec 22 21:18:37 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Dec 22 21:18:50 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../solr/cloud/autoscaling/AutoScaling.java     |   2 +
 .../cloud/autoscaling/ComputePlanAction.java    |  26 ++-
 .../solr/cloud/autoscaling/MetricTrigger.java   | 190 +++++++++++++++++++
 .../cloud/autoscaling/MetricTriggerTest.java    | 133 +++++++++++++
 .../autoscaling/TriggerIntegrationTest.java     | 132 ++++++++++++-
 .../src/solrcloud-autoscaling-triggers.adoc     |  32 +++-
 .../cloud/autoscaling/TriggerEventType.java     |   3 +-
 .../solr/common/params/AutoScalingParams.java   |   4 +
 9 files changed, 521 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 80d0d11..d182e36 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -60,6 +60,9 @@ New Features
   processor.  (Lance Norskog, Grant Ingersoll, Joern Kottmann, Em, Kai Gülzau,
   Rene Nederhand, Robert Muir, Steven Bower, Steve Rowe)
 
+* SOLR-11201: Implement autoscaling trigger for arbitrary metrics that creates events when
+  a given metric breaches a threshold (shalin)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
index 039067c..3ebfbd0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
@@ -155,6 +155,8 @@ public class AutoScaling {
           return new NodeLostTrigger(name, props, loader, cloudManager);
         case SEARCHRATE:
           return new SearchRateTrigger(name, props, loader, cloudManager);
+        case METRIC:
+          return new MetricTrigger(name, props, loader, cloudManager);
         default:
           throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
index 8cce976..b1e33e1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
@@ -40,6 +40,8 @@ import org.apache.solr.common.util.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
+
 /**
  * This class is responsible for using the configured policy and preferences
  * with the hints provided by the trigger event to compute the required cluster operations.
@@ -133,8 +135,30 @@ public class ComputePlanAction extends TriggerActionBase {
           }
         }
         break;
+      case METRIC:
+        Map<String, Number> sourceNodes = (Map<String, Number>) event.getProperty(AutoScalingParams.NODE);
+        String collection = (String) event.getProperty(AutoScalingParams.COLLECTION);
+        String shard = (String) event.getProperty(AutoScalingParams.SHARD);
+        String preferredOp = (String) event.getProperty(PREFERRED_OP);
+        if (sourceNodes.isEmpty()) {
+          log.warn("No nodes reported in event: " + event);
+          return NoneSuggester.INSTANCE;
+        }
+        CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp == null ? CollectionParams.CollectionAction.MOVEREPLICA.toLower() : preferredOp);
+        suggester = session.getSuggester(action);
+        for (String node : sourceNodes.keySet()) {
+          suggester = suggester.hint(Suggester.Hint.SRC_NODE, node);
+        }
+        if (collection != null) {
+          if (shard == null) {
+            suggester = suggester.hint(Suggester.Hint.COLL, collection);
+          } else {
+            suggester = suggester.hint(Suggester.Hint.COLL_SHARD, new Pair(collection, shard));
+          }
+        }
+        break;
       default:
-        throw new UnsupportedOperationException("No support for events other than nodeAdded, nodeLost and searchRate, received: " + event.getEventType());
+        throw new UnsupportedOperationException("No support for events other than nodeAdded, nodeLost, searchRate and metric. Received: " + event.getEventType());
     }
     return suggester;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
new file mode 100644
index 0000000..531e4e6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
@@ -0,0 +1,190 @@
+/*
+ * 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.autoscaling;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.params.AutoScalingParams;
+import org.apache.solr.core.SolrResourceLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.params.AutoScalingParams.ABOVE;
+import static org.apache.solr.common.params.AutoScalingParams.BELOW;
+import static org.apache.solr.common.params.AutoScalingParams.METRIC;
+import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
+
+public class MetricTrigger extends TriggerBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final String metric;
+  private final Number above, below;
+  private final String collection, shard, node, preferredOp;
+
+  private final Map<String, Long> lastNodeEvent = new ConcurrentHashMap<>();
+
+  public MetricTrigger(String name, Map<String, Object> properties, SolrResourceLoader loader, SolrCloudManager cloudManager) {
+    super(TriggerEventType.METRIC, name, properties, loader, cloudManager);
+    this.metric = (String) properties.get(METRIC);
+    this.above = (Number) properties.get(ABOVE);
+    this.below = (Number) properties.get(BELOW);
+    this.collection = (String) properties.getOrDefault(AutoScalingParams.COLLECTION, Policy.ANY);
+    shard = (String) properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
+    if (collection.equals(Policy.ANY) && !shard.equals(Policy.ANY)) {
+      throw new IllegalArgumentException("When 'shard' is other than #ANY then collection name must be also other than #ANY");
+    }
+    node = (String) properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
+    preferredOp = (String) properties.getOrDefault(PREFERRED_OP, null);
+  }
+
+  @Override
+  protected Map<String, Object> getState() {
+    return null;
+  }
+
+  @Override
+  protected void setState(Map<String, Object> state) {
+    lastNodeEvent.clear();
+    Map<String, Long> nodeTimes = (Map<String, Long>) state.get("lastNodeEvent");
+    if (nodeTimes != null) {
+      lastNodeEvent.putAll(nodeTimes);
+    }
+  }
+
+  @Override
+  public void restoreState(AutoScaling.Trigger old) {
+    assert old.isClosed();
+    if (old instanceof MetricTrigger) {
+      MetricTrigger that = (MetricTrigger) old;
+      assert this.name.equals(that.name);
+      this.lastNodeEvent.clear();
+      this.lastNodeEvent.putAll(that.lastNodeEvent);
+    } else {
+      throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
+          "Unable to restore state from an unknown type of trigger");
+    }
+  }
+
+  @Override
+  public void run() {
+    AutoScaling.TriggerEventProcessor processor = processorRef.get();
+    if (processor == null) {
+      return;
+    }
+
+    Set<String> liveNodes = null;
+    if (node.equals(Policy.ANY)) {
+      if (collection.equals(Policy.ANY)) {
+        liveNodes = cloudManager.getClusterStateProvider().getLiveNodes();
+      } else {
+        final Set<String> nodes = new HashSet<>();
+        ClusterState.CollectionRef ref = cloudManager.getClusterStateProvider().getState(collection);
+        DocCollection docCollection;
+        if (ref == null || (docCollection = ref.get()) == null) {
+          log.warn("MetricTrigger could not find collection: {}", collection);
+          return;
+        }
+        if (shard.equals(Policy.ANY)) {
+          docCollection.getReplicas().forEach(replica -> {
+            nodes.add(replica.getNodeName());
+          });
+        } else {
+          Slice slice = docCollection.getSlice(shard);
+          if (slice == null) {
+            log.warn("MetricTrigger could not find collection: {} shard: {}", collection, shard);
+            return;
+          }
+          slice.getReplicas().forEach(replica -> nodes.add(replica.getNodeName()));
+        }
+        liveNodes = nodes;
+      }
+    } else {
+      liveNodes = Collections.singleton(node);
+    }
+
+    Map<String, Number> rates = new HashMap<>(liveNodes.size());
+    for (String node : liveNodes) {
+      Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, Collections.singletonList(metric));
+      values.forEach((tag, rate) -> rates.computeIfAbsent(node, s -> (Number) rate));
+    }
+
+    long now = cloudManager.getTimeSource().getTime();
+    // check for exceeded rates and filter out those with less than waitFor from previous events
+    Map<String, Number> hotNodes = rates.entrySet().stream()
+        .filter(entry -> waitForElapsed(entry.getKey(), now, lastNodeEvent))
+        .filter(entry -> (below != null && Double.compare(entry.getValue().doubleValue(), below.doubleValue()) < 0) || (above != null && Double.compare(entry.getValue().doubleValue(), above.doubleValue()) > 0))
+        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    if (hotNodes.isEmpty()) return;
+
+    final AtomicLong eventTime = new AtomicLong(now);
+    hotNodes.forEach((n, r) -> {
+      long time = lastNodeEvent.get(n);
+      if (eventTime.get() > time) {
+        eventTime.set(time);
+      }
+    });
+
+    if (processor.process(new MetricBreachedEvent(getName(), collection, shard, preferredOp, eventTime.get(), metric, hotNodes))) {
+      hotNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
+    }
+  }
+
+  private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
+    Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
+    long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
+    log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
+    if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
+      return false;
+    }
+    return true;
+  }
+
+  public static class MetricBreachedEvent extends TriggerEvent {
+    public MetricBreachedEvent(String source, String collection, String shard, String preferredOp, long eventTime, String metric, Map<String, Number> hotNodes) {
+      super(TriggerEventType.METRIC, source, eventTime, null);
+      properties.put(METRIC, metric);
+      properties.put(AutoScalingParams.NODE, hotNodes);
+      if (!collection.equals(Policy.ANY)) {
+        properties.put(AutoScalingParams.COLLECTION, collection);
+      }
+      if (!shard.equals(Policy.ANY))  {
+        properties.put(AutoScalingParams.SHARD, shard);
+      }
+      if (preferredOp != null)  {
+        properties.put(PREFERRED_OP, preferredOp);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
new file mode 100644
index 0000000..96083f4
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.autoscaling;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.ZkDistributedQueueFactory;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class MetricTriggerTest extends SolrCloudTestCase {
+
+  private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
+    fail("Did not expect the listener to fire on first run!");
+    return true;
+  };
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(DEFAULT_TEST_COLLECTION_NAME,
+        "conf", 1, 1);
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    create.setMaxShardsPerNode(1);
+    create.process(solrClient);
+  }
+
+  @Test
+  public void test() throws Exception {
+    CoreDescriptor coreDescriptor = cluster.getJettySolrRunner(0).getCoreContainer().getCoreDescriptors().iterator().next();
+    String shardId = coreDescriptor.getCloudDescriptor().getShardId();
+    String coreName = coreDescriptor.getName();
+    String replicaName = Utils.parseMetricsReplicaName(DEFAULT_TEST_COLLECTION_NAME, coreName);
+    long waitForSeconds = 2 + random().nextInt(5);
+    String registry = SolrCoreMetricManager.createRegistryName(true, DEFAULT_TEST_COLLECTION_NAME, shardId, replicaName, null);
+    String tag = "metrics:" + registry + ":ADMIN./admin/file.requests";
+
+    Map<String, Object> props = createTriggerProps(waitForSeconds, tag, 1.0d, null, DEFAULT_TEST_COLLECTION_NAME, null, null);
+
+    final List<TriggerEvent> events = new ArrayList<>();
+    SolrZkClient zkClient = cluster.getSolrClient().getZkStateReader().getZkClient();
+    SolrResourceLoader loader = cluster.getJettySolrRunner(0).getCoreContainer().getResourceLoader();
+    SolrCloudManager cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(zkClient), cluster.getSolrClient());
+
+    try (MetricTrigger metricTrigger = new MetricTrigger("metricTrigger", props, loader, cloudManager)) {
+      metricTrigger.setProcessor(noFirstRunProcessor);
+      metricTrigger.run();
+      metricTrigger.setProcessor(event -> events.add(event));
+      assertEquals(0, events.size());
+      Thread.sleep(waitForSeconds * 1000 + 2000);
+      metricTrigger.run();
+      assertEquals(1, events.size());
+    }
+
+    events.clear();
+    tag = "metrics:" + registry + ":ADMIN./admin/file.handlerStart";
+    props = createTriggerProps(waitForSeconds, tag, null, 100.0d, DEFAULT_TEST_COLLECTION_NAME, null, null);
+    try (MetricTrigger metricTrigger = new MetricTrigger("metricTrigger", props, loader, cloudManager)) {
+      metricTrigger.setProcessor(noFirstRunProcessor);
+      metricTrigger.run();
+      metricTrigger.setProcessor(event -> events.add(event));
+      assertEquals(0, events.size());
+      Thread.sleep(waitForSeconds * 1000 + 2000);
+      metricTrigger.run();
+      assertEquals(1, events.size());
+    }
+  }
+
+  private Map<String, Object> createTriggerProps(long waitForSeconds, String metric, Double below, Double above, String collection, String shard, String node) {
+    Map<String, Object> props = new HashMap<>();
+    props.put("metric", metric);
+    if (above != null) {
+      props.put("above", above);
+    }
+    if (below != null) {
+      props.put("below", below);
+    }
+    if (collection != null) {
+      props.put("collection", collection);
+    }
+    if (shard != null) {
+      props.put("shard", shard);
+    }
+    if (node != null) {
+      props.put("node", node);
+    }
+    props.put("event", "metric");
+    props.put("waitFor", waitForSeconds);
+    props.put("enabled", true);
+
+    List<Map<String, String>> actions = new ArrayList<>(3);
+    Map<String, String> map = new HashMap<>(2);
+    map.put("name", "compute_plan");
+    map.put("class", "solr.ComputePlanAction");
+    actions.add(map);
+    map = new HashMap<>(2);
+    map.put("name", "execute_plan");
+    map.put("class", "solr.ExecutePlanAction");
+    actions.add(map);
+    props.put("actions", actions);
+    return props;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index c104a99..eb196c1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -45,18 +45,22 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.LiveNodesListener;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZKUtil;
 import org.apache.zookeeper.data.Stat;
@@ -1497,4 +1501,130 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(collectionRate, totalShardRate.get(), 5.0);
     assertEquals(collectionRate, totalReplicaRate.get(), 5.0);
   }
+
+  @Test
+  public void testMetricTrigger() throws Exception {
+    // at least 3 nodes
+    for (int i = cluster.getJettySolrRunners().size(); i < 3; i++) {
+      cluster.startJettySolrRunner();
+    }
+    cluster.waitForAllNodes(5);
+
+    String collectionName = "testMetricTrigger";
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 2, 1);
+    create.process(solrClient);
+    solrClient.setDefaultCollection(collectionName);
+
+    waitForState("Timed out waiting for collection:" + collectionName + " to become active", collectionName, clusterShape(2, 1));
+
+    DocCollection docCollection = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    String shardId = "shard1";
+    Replica replica = docCollection.getSlice(shardId).getReplicas().iterator().next();
+    String coreName = replica.getCoreName();
+    String replicaName = Utils.parseMetricsReplicaName(collectionName, coreName);
+    long waitForSeconds = 2 + random().nextInt(5);
+    String registry = SolrCoreMetricManager.createRegistryName(true, collectionName, shardId, replicaName, null);
+    String tag = "metrics:" + registry + ":INDEX.sizeInBytes";
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'metric_trigger'," +
+        "'event' : 'metric'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'metric': '" + tag + "'" +
+        "'above' : 100.0," +
+        "'collection': '" + collectionName + "'" +
+        "'shard':'"  + shardId + "'" +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+        "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'srt'," +
+        "'trigger' : 'metric_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'afterAction': ['compute', 'execute', 'test']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    for (int i = 0; i < 500; i++) {
+      solrClient.add(new SolrInputDocument("id", String.valueOf(i), "x_s", "x" + i));
+    }
+    solrClient.commit();
+
+    boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    Thread.sleep(2000);
+    assertEquals(listenerEvents.toString(), 4, listenerEvents.get("srt").size());
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
+    long now = timeSource.getTime();
+    // verify waitFor
+    assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());
+    assertEquals(collectionName, ev.event.getProperties().get("collection"));
+
+    String oldReplicaName = replica.getName();
+    docCollection = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    assertEquals(2, docCollection.getReplicas().size());
+    assertNull(docCollection.getReplica(oldReplicaName));
+
+    // todo uncomment the following code once SOLR-11714 is fixed
+    // find a new replica and create its metric name
+//    replica = docCollection.getSlice(shardId).getReplicas().iterator().next();
+//    coreName = replica.getCoreName();
+//    replicaName = Utils.parseMetricsReplicaName(collectionName, coreName);
+//    registry = SolrCoreMetricManager.createRegistryName(true, collectionName, shardId, replicaName, null);
+//    tag = "metrics:" + registry + ":INDEX.sizeInBytes";
+//
+//    setTriggerCommand = "{" +
+//        "'set-trigger' : {" +
+//        "'name' : 'metric_trigger'," +
+//        "'event' : 'metric'," +
+//        "'waitFor' : '" + waitForSeconds + "s'," +
+//        "'enabled' : true," +
+//        "'metric': '" + tag + "'" +
+//        "'above' : 100.0," +
+//        "'collection': '" + collectionName + "'" +
+//        "'shard':'"  + shardId + "'" +
+//        "'preferredOperation':'addreplica'" +
+//        "'actions' : [" +
+//        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+//        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+//        "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
+//        "]" +
+//        "}}";
+//    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+//    response = solrClient.request(req);
+//    assertEquals(response.get("result").toString(), "success");
+//
+//    triggerFiredLatch = new CountDownLatch(1);
+//    listenerEvents.clear();
+//    await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
+//    assertTrue("The trigger did not fire at all", await);
+//    // wait for listener to capture the SUCCEEDED stage
+//    Thread.sleep(2000);
+//    assertEquals(listenerEvents.toString(), 4, listenerEvents.get("srt").size());
+//    ev = listenerEvents.get("srt").get(0);
+//    now = timeSource.getTime();
+//    // verify waitFor
+//    assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());
+//    assertEquals(collectionName, ev.event.getProperties().get("collection"));
+//    docCollection = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+//    assertEquals(3, docCollection.getReplicas().size());
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
index 70cbb5f..5a98657 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
@@ -34,6 +34,7 @@ Currently the following event types (and corresponding trigger implementations)
 
 * `nodeAdded` - generated when a new node joins the cluster
 * `nodeLost` - generated when a node leaves the cluster
+* `metric` - generated when the configured metric crosses a configured lower or upper threshold value
 
 Events are not necessarily generated immediately after the corresponding state change occurred - the
 maximum rate of events is controlled by the `waitFor` configuration parameter (see below).
@@ -57,6 +58,35 @@ to add replicas on the live nodes to maintain the expected replication factor).
 
 You can see the section <<solrcloud-autoscaling-auto-add-replicas.adoc#solrcloud-autoscaling-auto-add-replicas, Autoscaling Automatically Adding Replicas>> to learn more about how the `.autoAddReplicas` trigger works.
 
+== Metric Trigger
+
+The metric trigger can be used to monitor any metric exposed by the Metrics API. It supports lower and upper threshold configurations as well as optional filters to limit operation to specific collection, shards and nodes.
+
+This trigger supports the following configuration:
+
+* `metric` - (string, required) The metric property name to be watched in the format metrics:group:prefix e.g. `metric:solr.node:CONTAINER.fs.coreRoot.usableSpace`
+* `below` - (double, optional) The lower threshold for the metric value. The trigger produces a metric breached event if the metric's value falls below this value
+* `above` - (double, optional) The upper threshold for the metric value. The trigger produces a metric breached event if the metric's value crosses above this value
+* `collection` - (string, optional) The collection used to limit the nodes on which the given metric is watched. When the metric is breached, trigger actions will limit operations to this collection only.
+* `shard` - (string, optional) The shard used to limit the nodes on which the given metric is watched. When the metric is breached, trigger actions will limit operations to this shard only.
+* `node` - (string, optional) The node on which the given metric is watched. Trigger actions will operate on this node only.
+* `preferredOperation` (string, optional, defaults to `MOVEREPLICA`) - The operation to be performed in response to an event generated by this trigger. By default, replicas will be moved from the hot node to others. The only other supported value is `ADDREPLICA` which adds more replicas if the metric is breached.
+
+.Example: Metric Trigger that fires when total usable space on a node having replicas of "mycollection" falls below 100GB
+[source,json]
+----
+{
+  "set-trigger": {
+    "name": "metric_trigger",
+    "event": "metric",
+    "waitFor": "5s",
+    "metric": "metric:solr.node:CONTAINER.fs.coreRoot.usableSpace",
+    "below": 107374182400,
+    "collection": "mycollection"
+  }
+}
+----
+
 == Trigger Configuration
 Trigger configurations are managed using the Autoscaling Write API and the commands `set-trigger`, `remove-trigger`,
 `suspend-trigger`, and `resume-trigger`.
@@ -74,7 +104,7 @@ Action configuration consists of the following properties:
 
 * `name` - (string, required) A unique name of the action configuration.
 * `class` - (string, required) The action implementation class.
-* A dditional implementation-specific properties may be provided
+* Additional implementation-specific properties may be provided
 
 If the Action configuration is omitted, then by default, the `ComputePlanAction` and the `ExecutePlanAction` are automatically added to the trigger configuration.
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java
index 238d7e1..96bc773 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java
@@ -27,5 +27,6 @@ public enum TriggerEventType {
   SCHEDULED,
   SEARCHRATE,
   INDEXRATE,
-  INVALID
+  INVALID,
+  METRIC
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/43f17f7a/solr/solrj/src/java/org/apache/solr/common/params/AutoScalingParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/AutoScalingParams.java b/solr/solrj/src/java/org/apache/solr/common/params/AutoScalingParams.java
index cf259c6..4f00e28 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/AutoScalingParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/AutoScalingParams.java
@@ -46,6 +46,10 @@ public interface AutoScalingParams {
   String RATE = "rate";
   String REMOVE_LISTENERS = "removeListeners";
   String ZK_VERSION = "zkVersion";
+  String METRIC = "metric";
+  String ABOVE = "above";
+  String BELOW = "below";
+  String PREFERRED_OP = "preferredOperation";
 
   // commands
   String CMD_SET_TRIGGER = "set-trigger";


[21/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11742: Fix error

Posted by da...@apache.org.
SOLR-11742: Fix 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/ca5e2a33
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/ca5e2a33
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/ca5e2a33

Branch: refs/heads/jira/solr-11702
Commit: ca5e2a335b3ec5d2fc9cd4bcdb79a87e1c86a3af
Parents: 9dee35f
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 15 17:05:32 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 15 17:05:32 2017 -0500

----------------------------------------------------------------------
 solr/solr-ref-guide/src/stream-evaluator-reference.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca5e2a33/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/stream-evaluator-reference.adoc b/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
index cf2544c..e38284e 100644
--- a/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
+++ b/solr/solr-ref-guide/src/stream-evaluator-reference.adoc
@@ -1640,7 +1640,7 @@ eq(raw(fieldA), fieldA) // true if the value of fieldA equals the string "fieldA
 
 The `regress` function performs a simple regression of two numeric arrays.
 
-The result of this expression is also used by the `<<predict>>` and `<<residuals>>` functions.
+The result of this expression is also used by the `<<predict>>` function.
 
 === regress Parameters
 


[03/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
new file mode 100644
index 0000000..9b4e2bc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
@@ -0,0 +1,306 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScaling;
+import org.apache.solr.cloud.autoscaling.NodeAddedTrigger;
+import org.apache.solr.cloud.autoscaling.TriggerAction;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.common.util.TimeSource;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link NodeAddedTrigger}
+ */
+public class TestNodeAddedTrigger extends SimSolrCloudTestCase {
+  private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
+
+  private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
+    fail("Did not expect the listener to fire on first run!");
+    return true;
+  };
+
+  private static int SPEED = 50;
+
+  // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
+  private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(2);
+
+  private static TimeSource timeSource;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1, TimeSource.get("simTime:" + SPEED));
+    timeSource = cluster.getTimeSource();
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    actionConstructorCalled = new AtomicBoolean(false);
+    actionInitCalled = new AtomicBoolean(false);
+    actionCloseCalled = new AtomicBoolean(false);
+  }
+
+  @Test
+  public void testTrigger() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+
+      String newNode1 = cluster.simAddNode();
+      String newNode2 = cluster.simAddNode();
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeAddedTrigger was fired more than once!");
+        }
+        return true;
+      });
+      int counter = 0;
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Newly added node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      TriggerEvent nodeAddedEvent = eventRef.get();
+      assertNotNull(nodeAddedEvent);
+      List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+      assertTrue(nodeNames.contains(newNode1));
+      assertTrue(nodeNames.contains(newNode2));
+    }
+
+    // add a new node but remove it before the waitFor period expires
+    // and assert that the trigger doesn't fire at all
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      final long waitTime = 2;
+      props.put("waitFor", waitTime);
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+
+      String newNode = cluster.simAddNode();
+      AtomicBoolean fired = new AtomicBoolean(false);
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeAddedTrigger was fired more than once!");
+        }
+        return true;
+      });
+      trigger.run(); // first run should detect the new node
+      cluster.simRemoveNode(newNode, true);
+      int counter = 0;
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > waitTime + 1) { // run it a little more than the wait time
+          break;
+        }
+      } while (true);
+
+      // ensure the event was not fired
+      assertFalse(fired.get());
+    }
+  }
+
+  public void testActionLifecycle() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+    Map<String, String> action = new HashMap<>(2);
+    action.put("name", "testActionInit");
+    action.put("class", TestNodeAddedTrigger.AssertInitTriggerAction.class.getName());
+    actions.add(action);
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      assertEquals(true, actionConstructorCalled.get());
+      assertEquals(false, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+      trigger.init();
+      assertEquals(true, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+    }
+    assertEquals(true, actionCloseCalled.get());
+  }
+
+  public static class AssertInitTriggerAction implements TriggerAction  {
+    public AssertInitTriggerAction() {
+      actionConstructorCalled.set(true);
+    }
+
+    @Override
+    public String getName() {
+      return "";
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+      actionCloseCalled.compareAndSet(false, true);
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      actionInitCalled.compareAndSet(false, true);
+    }
+  }
+
+  @Test
+  public void testListenerAcceptance() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run(); // starts tracking live nodes
+
+      String newNode = cluster.simAddNode();
+      AtomicInteger callCount = new AtomicInteger(0);
+      AtomicBoolean fired = new AtomicBoolean(false);
+
+      trigger.setProcessor(event -> {
+        if (callCount.incrementAndGet() < 2) {
+          return false;
+        } else  {
+          fired.compareAndSet(false, true);
+          return true;
+        }
+      });
+
+      trigger.run(); // first run should detect the new node and fire immediately but listener isn't ready
+      assertEquals(1, callCount.get());
+      assertFalse(fired.get());
+      trigger.run(); // second run should again fire
+      assertEquals(2, callCount.get());
+      assertTrue(fired.get());
+      trigger.run(); // should not fire
+      assertEquals(2, callCount.get());
+    }
+  }
+
+  @Test
+  public void testRestoreState() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    // add a new node but update the trigger before the waitFor period expires
+    // and assert that the new trigger still fires
+    NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster);
+    trigger.setProcessor(noFirstRunProcessor);
+    trigger.run();
+
+    String newNode = cluster.simAddNode();
+    trigger.run(); // this run should detect the new node
+    trigger.close(); // close the old trigger
+
+    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, cluster.getLoader(), cluster))  {
+      try {
+        newTrigger.restoreState(trigger);
+        fail("Trigger should only be able to restore state from an old trigger of the same name");
+      } catch (AssertionError e) {
+        // expected
+      }
+    }
+
+    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster))  {
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      newTrigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeAddedTrigger was fired more than once!");
+        }
+        return true;
+      });
+      newTrigger.restoreState(trigger); // restore state from the old trigger
+      int counter = 0;
+      do {
+        newTrigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Newly added node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      // ensure the event was fired
+      assertTrue(fired.get());
+      TriggerEvent nodeAddedEvent = eventRef.get();
+      assertNotNull(nodeAddedEvent);
+      //TODO assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
+    }
+  }
+
+  private Map<String, Object> createTriggerProps(long waitForSeconds) {
+    Map<String, Object> props = new HashMap<>();
+    props.put("event", "nodeLost");
+    props.put("waitFor", waitForSeconds);
+    props.put("enabled", true);
+    List<Map<String, String>> actions = new ArrayList<>(3);
+    Map<String, String> map = new HashMap<>(2);
+    map.put("name", "compute_plan");
+    map.put("class", "solr.ComputePlanAction");
+    actions.add(map);
+    map = new HashMap<>(2);
+    map.put("name", "execute_plan");
+    map.put("class", "solr.ExecutePlanAction");
+    actions.add(map);
+    props.put("actions", actions);
+    return props;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
new file mode 100644
index 0000000..109cee3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
@@ -0,0 +1,331 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScaling;
+import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
+import org.apache.solr.cloud.autoscaling.TriggerAction;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.common.util.TimeSource;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link NodeLostTrigger}
+ */
+public class TestNodeLostTrigger extends SimSolrCloudTestCase {
+  private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
+
+  private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
+    fail("Did not expect the listener to fire on first run!");
+    return true;
+  };
+
+  private static final int SPEED = 50;
+  // use the same time source as the trigger
+  private static TimeSource timeSource;
+  // currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
+  private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(5, TimeSource.get("simTime:" + SPEED));
+    timeSource = cluster.getTimeSource();
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    actionConstructorCalled = new AtomicBoolean(false);
+    actionInitCalled = new AtomicBoolean(false);
+    actionCloseCalled = new AtomicBoolean(false);
+  }
+
+  @Test
+  public void testTrigger() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+      Iterator<String> it = cluster.getLiveNodesSet().get().iterator();
+      String lostNodeName1 = it.next();
+      String lostNodeName2 = it.next();
+      cluster.simRemoveNode(lostNodeName1, true);
+      cluster.simRemoveNode(lostNodeName2, true);
+      timeSource.sleep(1000);
+
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeLostListener was fired more than once!");
+        }
+        return true;
+      });
+      int counter = 0;
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Lost node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      TriggerEvent nodeLostEvent = eventRef.get();
+      assertNotNull(nodeLostEvent);
+      List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+      assertTrue(nodeNames + " doesn't contain " + lostNodeName1, nodeNames.contains(lostNodeName1));
+      assertTrue(nodeNames + " doesn't contain " + lostNodeName2, nodeNames.contains(lostNodeName2));
+
+    }
+
+    // remove a node but add it back before the waitFor period expires
+    // and assert that the trigger doesn't fire at all
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
+      final long waitTime = 2;
+      props.put("waitFor", waitTime);
+      trigger.setProcessor(noFirstRunProcessor);
+      trigger.run();
+
+      String lostNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+      cluster.simRemoveNode(lostNode, false);
+      AtomicBoolean fired = new AtomicBoolean(false);
+      trigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitTime, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" +  eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeLostListener was fired more than once!");
+        }
+        return true;
+      });
+      trigger.run(); // first run should detect the lost node
+      int counter = 0;
+      do {
+        if (cluster.getLiveNodesSet().get().size() == 2) {
+          break;
+        }
+        timeSource.sleep(100);
+        if (counter++ > 20) {
+          fail("Live nodes not updated!");
+        }
+      } while (true);
+      counter = 0;
+      cluster.getSimClusterStateProvider().simRestoreNode(lostNode);
+      do {
+        trigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > waitTime + 1) { // run it a little more than the wait time
+          break;
+        }
+      } while (true);
+
+      // ensure the event was not fired
+      assertFalse(fired.get());
+    }
+  }
+
+  public void testActionLifecycle() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+    Map<String, String> action = new HashMap<>(2);
+    action.put("name", "testActionInit");
+    action.put("class", AssertInitTriggerAction.class.getName());
+    actions.add(action);
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      assertEquals(true, actionConstructorCalled.get());
+      assertEquals(false, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+      trigger.init();
+      assertEquals(true, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+    }
+    assertEquals(true, actionCloseCalled.get());
+  }
+
+  public static class AssertInitTriggerAction implements TriggerAction  {
+    public AssertInitTriggerAction() {
+      actionConstructorCalled.set(true);
+    }
+
+    @Override
+    public String getName() {
+      return "";
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+      actionCloseCalled.compareAndSet(false, true);
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      actionInitCalled.compareAndSet(false, true);
+    }
+  }
+
+  @Test
+  public void testListenerAcceptance() throws Exception {
+    Map<String, Object> props = createTriggerProps(0);
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
+      trigger.setProcessor(noFirstRunProcessor);
+
+      String newNode = cluster.simAddNode();
+
+      trigger.run(); // starts tracking live nodes
+
+      // stop the newly created node
+      cluster.simRemoveNode(newNode, true);
+
+      AtomicInteger callCount = new AtomicInteger(0);
+      AtomicBoolean fired = new AtomicBoolean(false);
+
+      trigger.setProcessor(event -> {
+        if (callCount.incrementAndGet() < 2) {
+          return false;
+        } else  {
+          fired.compareAndSet(false, true);
+          return true;
+        }
+      });
+
+      trigger.run(); // first run should detect the lost node and fire immediately but listener isn't ready
+      assertEquals(1, callCount.get());
+      assertFalse(fired.get());
+      trigger.run(); // second run should again fire
+      assertEquals(2, callCount.get());
+      assertTrue(fired.get());
+      trigger.run(); // should not fire
+      assertEquals(2, callCount.get());
+    }
+  }
+
+  @Test
+  public void testRestoreState() throws Exception {
+    long waitForSeconds = 1 + random().nextInt(5);
+    Map<String, Object> props = createTriggerProps(waitForSeconds);
+
+    String newNode = cluster.simAddNode();
+
+    // remove a node but update the trigger before the waitFor period expires
+    // and assert that the new trigger still fires
+
+    NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster);
+    trigger.setProcessor(noFirstRunProcessor);
+    trigger.run();
+
+    // stop the newly created node
+    cluster.simRemoveNode(newNode, true);
+
+    trigger.run(); // this run should detect the lost node
+    trigger.close(); // close the old trigger
+
+    try (NodeLostTrigger newTrigger = new NodeLostTrigger("some_different_name", props, cluster.getLoader(), cluster))  {
+      try {
+        newTrigger.restoreState(trigger);
+        fail("Trigger should only be able to restore state from an old trigger of the same name");
+      } catch (AssertionError e) {
+        // expected
+      }
+    }
+
+    try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
+      AtomicBoolean fired = new AtomicBoolean(false);
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+      newTrigger.setProcessor(event -> {
+        if (fired.compareAndSet(false, true)) {
+          eventRef.set(event);
+          long currentTimeNanos = timeSource.getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
+          }
+        } else {
+          fail("NodeLostListener was fired more than once!");
+        }
+        return true;
+      });
+      newTrigger.restoreState(trigger); // restore state from the old trigger
+      int counter = 0;
+      do {
+        newTrigger.run();
+        timeSource.sleep(1000);
+        if (counter++ > 10) {
+          fail("Lost node was not discovered by trigger even after 10 seconds");
+        }
+      } while (!fired.get());
+
+      TriggerEvent nodeLostEvent = eventRef.get();
+      assertNotNull(nodeLostEvent);
+      List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+      assertTrue(nodeNames.contains(newNode));
+    }
+  }
+
+  private Map<String, Object> createTriggerProps(long waitForSeconds) {
+    Map<String, Object> props = new HashMap<>();
+    props.put("event", "nodeLost");
+    props.put("waitFor", waitForSeconds);
+    props.put("enabled", true);
+    List<Map<String, String>> actions = new ArrayList<>(3);
+    Map<String, String> map = new HashMap<>(2);
+    map.put("name", "compute_plan");
+    map.put("class", "solr.ComputePlanAction");
+    actions.add(map);
+    map = new HashMap<>(2);
+    map.put("name", "execute_plan");
+    map.put("class", "solr.ExecutePlanAction");
+    actions.add(map);
+    props.put("actions", actions);
+    return props;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java
new file mode 100644
index 0000000..22736db
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestPolicyCloud.java
@@ -0,0 +1,357 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.Row;
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
+public class TestPolicyCloud extends SimSolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  @org.junit.Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(5, TimeSource.get("simTime:50"));
+  }
+
+  public void testDataProviderPerReplicaDetails() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    CollectionAdminRequest.createCollection("perReplicaDataColl", "conf", 1, 5)
+        .process(solrClient);
+
+    waitForState("Timeout waiting for collection to become active", "perReplicaDataColl", clusterShape(1, 5));
+    DocCollection coll = getCollectionState("perReplicaDataColl");
+    String autoScaleJson = "{" +
+        "  'cluster-preferences': [" +
+        "    { maximize : freedisk , precision: 50}," +
+        "    { minimize : cores, precision: 2}" +
+        "  ]," +
+        "  'cluster-policy': [" +
+        "    { replica : '0' , 'nodeRole': 'overseer'}," +
+        "    { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'" +
+        "    }" +
+        "  ]," +
+        "  'policies': {" +
+        "    'policy1': [" +
+        "      { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
+        "    ]" +
+        "  }" +
+        "}";
+    AutoScalingConfig config = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScaleJson));
+    Policy.Session session = config.getPolicy().createSession(cluster);
+
+    AtomicInteger count = new AtomicInteger(0);
+    for (Row row : session.getSorted()) {
+      row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
+        for (ReplicaInfo replicaInfo : replicaInfos) {
+          if (replicaInfo.getVariables().containsKey(Suggestion.ConditionType.CORE_IDX.tagName)) count.incrementAndGet();
+        }
+      }));
+    }
+    assertTrue(count.get() > 0);
+
+    CollectionAdminRequest.deleteCollection("perReplicaDataColl").process(solrClient);
+
+  }
+
+  public void testCreateCollectionAddReplica() throws Exception  {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
+
+    String commands =  "{set-policy :{c1 : [{replica:0 , shard:'#EACH', port: '!" + port + "'}]}}";
+    solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+
+    String collectionName = "testCreateCollectionAddReplica";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
+        .setPolicy("c1")
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", collectionName, clusterShape(1, 1));
+
+    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(solrClient);
+    waitForState("Timed out waiting to see 2 replicas for collection: " + collectionName,
+        collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 2);
+
+    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+  }
+
+  public void testCreateCollectionSplitShard() throws Exception  {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String firstNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    int firstNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(firstNode, ImplicitSnitch.PORT);
+
+    String secondNode;
+    int secondNodePort;
+    while (true)  {
+      secondNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+      secondNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(secondNode, ImplicitSnitch.PORT);
+      if (secondNodePort != firstNodePort)  break;
+    }
+
+    String commands =  "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + firstNodePort + "'}, {replica:1, shard:'#EACH', port:'" + secondNodePort + "'}]}}";
+    NamedList<Object> response = solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+    assertEquals("success", response.get("result"));
+
+    String collectionName = "testCreateCollectionSplitShard";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
+        .setPolicy("c1")
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", collectionName, clusterShape(1, 2));
+
+    DocCollection docCollection = getCollectionState(collectionName);
+    List<Replica> list = docCollection.getReplicas(firstNode);
+    int replicasOnNode1 = list != null ? list.size() : 0;
+    list = docCollection.getReplicas(secondNode);
+    int replicasOnNode2 = list != null ? list.size() : 0;
+
+    assertEquals("Expected exactly one replica of collection on node with port: " + firstNodePort, 1, replicasOnNode1);
+    assertEquals("Expected exactly one replica of collection on node with port: " + secondNodePort, 1, replicasOnNode2);
+
+    CollectionAdminRequest.splitShard(collectionName).setShardName("shard1").process(solrClient);
+
+    waitForState("Timed out waiting to see 6 replicas for collection: " + collectionName,
+        collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 6);
+
+    docCollection = getCollectionState(collectionName);
+    list = docCollection.getReplicas(firstNode);
+    replicasOnNode1 = list != null ? list.size() : 0;
+    list = docCollection.getReplicas(secondNode);
+    replicasOnNode2 = list != null ? list.size() : 0;
+
+    assertEquals("Expected exactly three replica of collection on node with port: " + firstNodePort, 3, replicasOnNode1);
+    assertEquals("Expected exactly three replica of collection on node with port: " + secondNodePort, 3, replicasOnNode2);
+    CollectionAdminRequest.deleteCollection(collectionName).process(solrClient);
+
+  }
+
+  public void testMetricsTag() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'metrics:abc':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    try {
+      solrClient.request(req);
+      fail("expected exception");
+    } catch (Exception e) {
+      // expected
+      assertTrue(e.toString().contains("Invalid metrics: param in"));
+    }
+    setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'metrics:solr.node:ADMIN./admin/authorization.clientErrors:count':'>58768765', 'replica':0}" +
+        "    ]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    solrClient.request(req);
+
+    //org.eclipse.jetty.server.handler.DefaultHandler.2xx-responses
+    CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1)
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "metricsTest", clusterShape(1, 1));
+
+    DocCollection collection = getCollectionState("metricsTest");
+    List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
+        "metrics:solr.jvm:buffers.direct.Count");
+    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNodeName(), tags);
+    for (String tag : tags) {
+      assertNotNull( "missing : "+ tag , val.get(tag));
+    }
+
+
+  }
+
+  public void testCreateCollectionAddShardWithReplicaTypeUsingPolicy() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    List<String> nodes = new ArrayList<>(cluster.getClusterStateProvider().getLiveNodes());
+    String nrtNodeName = nodes.get(0);
+    int nrtPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nrtNodeName, ImplicitSnitch.PORT);
+
+
+    String pullNodeName = nodes.get(1);
+    int pullPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(pullNodeName, ImplicitSnitch.PORT);
+
+    String tlogNodeName = nodes.get(1);
+    int tlogPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(tlogNodeName, ImplicitSnitch.PORT);
+    log.info("NRT {} PULL {} , TLOG {} ", nrtNodeName, pullNodeName, tlogNodeName);
+
+    String commands = "{set-cluster-policy :[" +
+        "{replica:0 , shard:'#EACH', type: NRT, port: '!" + nrtPort + "'}" +
+        "{replica:0 , shard:'#EACH', type: PULL, port: '!" + pullPort + "'}" +
+        "{replica:0 , shard:'#EACH', type: TLOG, port: '!" + tlogPort + "'}" +
+        "]}";
+
+
+    solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+    Map<String, Object> json = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    assertEquals("full json:" + Utils.toJSONString(json), "!" + nrtPort,
+        Utils.getObjectByPath(json, true, "cluster-policy[0]/port"));
+    assertEquals("full json:" + Utils.toJSONString(json), "!" + pullPort,
+        Utils.getObjectByPath(json, true, "cluster-policy[1]/port"));
+    assertEquals("full json:" + Utils.toJSONString(json), "!" + tlogPort,
+        Utils.getObjectByPath(json, true, "cluster-policy[2]/port"));
+
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1", 1, 1, 1)
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(1, 3));
+
+    DocCollection coll = getCollectionState("policiesTest");
+
+
+    BiConsumer<String, Replica> verifyReplicas = (s, replica) -> {
+      switch (replica.getType()) {
+        case NRT: {
+          assertTrue("NRT replica should be in " + nrtNodeName, replica.getNodeName().equals(nrtNodeName));
+          break;
+        }
+        case TLOG: {
+          assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNodeName().equals(tlogNodeName));
+          break;
+        }
+        case PULL: {
+          assertTrue("PULL replica should be in " + pullNodeName, replica.getNodeName().equals(pullNodeName));
+          break;
+        }
+      }
+
+    };
+    coll.forEachReplica(verifyReplicas);
+
+    CollectionAdminRequest.createShard("policiesTest", "s3").
+        process(solrClient);
+    coll = getCollectionState("policiesTest");
+    assertEquals(3, coll.getSlice("s3").getReplicas().size());
+    coll.forEachReplica(verifyReplicas);
+  }
+
+  public void testCreateCollectionAddShardUsingPolicy() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
+
+    String commands =  "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + port + "'}]}}";
+    solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
+    Map<String, Object> json = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    assertEquals("full json:"+ Utils.toJSONString(json) , "#EACH",
+        Utils.getObjectByPath(json, true, "/policies/c1[0]/shard"));
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1,s2", 1)
+        .setPolicy("c1")
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(2, 1));
+
+    DocCollection coll = getCollectionState("policiesTest");
+    assertEquals("c1", coll.getPolicyName());
+    assertEquals(2,coll.getReplicas().size());
+    coll.forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+    CollectionAdminRequest.createShard("policiesTest", "s3").process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(3, 1));
+
+    coll = getCollectionState("policiesTest");
+    assertEquals(1, coll.getSlice("s3").getReplicas().size());
+    coll.getSlice("s3").forEach(replica -> assertEquals(nodeId, replica.getNodeName()));
+  }
+
+  public void testDataProvider() throws IOException, SolrServerException, KeeperException, InterruptedException {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
+        .process(solrClient);
+    waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(1, 2));
+    DocCollection rulesCollection = getCollectionState("policiesTest");
+
+    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
+        "freedisk",
+        "cores",
+        "heapUsage",
+        "sysLoadAvg"));
+    assertNotNull(val.get("freedisk"));
+    assertNotNull(val.get("heapUsage"));
+    assertNotNull(val.get("sysLoadAvg"));
+    assertTrue(((Number) val.get("cores")).intValue() > 0);
+    assertTrue("freedisk value is " + ((Number) val.get("freedisk")).doubleValue(),  Double.compare(((Number) val.get("freedisk")).doubleValue(), 0.0d) > 0);
+    assertTrue("heapUsage value is " + ((Number) val.get("heapUsage")).doubleValue(), Double.compare(((Number) val.get("heapUsage")).doubleValue(), 0.0d) > 0);
+    if (!Constants.WINDOWS)  {
+      // the system load average metrics is not available on windows platform
+      assertTrue("sysLoadAvg value is " + ((Number) val.get("sysLoadAvg")).doubleValue(), Double.compare(((Number) val.get("sysLoadAvg")).doubleValue(), 0.0d) > 0);
+    }
+    // simulator doesn't have Overseer, so just pick a random node
+    String overseerNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    solrClient.request(CollectionAdminRequest.addRole(overseerNode, "overseer"));
+    for (int i = 0; i < 10; i++) {
+      Map<String, Object> data = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.ROLES);
+      if (i >= 9 && data.isEmpty()) {
+        throw new RuntimeException("NO overseer node created");
+      }
+      cluster.getTimeSource().sleep(100);
+    }
+    val = cluster.getNodeStateProvider().getNodeValues(overseerNode, Arrays.asList(
+        "nodeRole",
+        "ip_1", "ip_2", "ip_3", "ip_4",
+        "sysprop.java.version",
+        "sysprop.java.vendor"));
+    assertEquals("overseer", val.get("nodeRole"));
+    assertNotNull(val.get("ip_1"));
+    assertNotNull(val.get("ip_2"));
+    assertNotNull(val.get("ip_3"));
+    assertNotNull(val.get("ip_4"));
+    assertNotNull(val.get("sysprop.java.version"));
+    assertNotNull(val.get("sysprop.java.vendor"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
new file mode 100644
index 0000000..1c41795
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistributedQueue.java
@@ -0,0 +1,220 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.nio.charset.Charset;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Predicate;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class TestSimDistributedQueue extends SolrTestCaseJ4 {
+  private static final Charset UTF8 = Charset.forName("UTF-8");
+  protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("sdqtest-"));
+
+  @Test
+  public void testDistributedQueue() throws Exception {
+    String dqZNode = "/distqueue/test1";
+    byte[] data = "hello world".getBytes(UTF8);
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    // basic ops
+    assertNull(dq.poll());
+    try {
+      dq.remove();
+      fail("NoSuchElementException expected");
+    } catch (NoSuchElementException expected) {
+      // expected
+    }
+
+    dq.offer(data);
+    assertArrayEquals(dq.peek(500), data);
+    assertArrayEquals(dq.remove(), data);
+    assertNull(dq.poll());
+
+    dq.offer(data);
+    assertArrayEquals(dq.take(), data); // waits for data
+    assertNull(dq.poll());
+
+    dq.offer(data);
+    dq.peek(true); // wait until data is definitely there before calling remove
+    assertArrayEquals(dq.remove(), data);
+    assertNull(dq.poll());
+
+    // should block until the background thread makes the offer
+    (new QueueChangerThread(dq, 1000)).start();
+    assertNotNull(dq.peek(true));
+    assertNotNull(dq.remove());
+    assertNull(dq.poll());
+
+    // timeout scenario ... background thread won't offer until long after the peek times out
+    QueueChangerThread qct = new QueueChangerThread(dq, 1000);
+    qct.start();
+    assertNull(dq.peek(500));
+    qct.join();
+  }
+
+  @Test
+  public void testDistributedQueueBlocking() throws Exception {
+    String dqZNode = "/distqueue/test2";
+    String testData = "hello world";
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    assertNull(dq.peek());
+    Future<String> future = executor.submit(() -> new String(dq.peek(true), UTF8));
+    try {
+      future.get(1000, TimeUnit.MILLISECONDS);
+      fail("TimeoutException expected");
+    } catch (TimeoutException expected) {
+      assertFalse(future.isDone());
+    }
+
+    dq.offer(testData.getBytes(UTF8));
+    assertEquals(testData, future.get(1000, TimeUnit.MILLISECONDS));
+    assertNotNull(dq.poll());
+
+    assertNull(dq.peek(100));
+
+    // Rerun the earlier test make sure updates are still seen, post reconnection.
+    future = executor.submit(() -> new String(dq.peek(true), UTF8));
+    try {
+      future.get(1000, TimeUnit.MILLISECONDS);
+      fail("TimeoutException expected");
+    } catch (TimeoutException expected) {
+      assertFalse(future.isDone());
+    }
+
+    dq.offer(testData.getBytes(UTF8));
+    assertEquals(testData, future.get(1000, TimeUnit.MILLISECONDS));
+    assertNotNull(dq.poll());
+    assertNull(dq.poll());
+  }
+
+  @Test
+  public void testLocallyOffer() throws Exception {
+    String dqZNode = "/distqueue/test3";
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+    dq.peekElements(1, 1, s -> true);
+    for (int i = 0; i < 100; i++) {
+      byte[] data = String.valueOf(i).getBytes(UTF8);
+      dq.offer(data);
+      assertNotNull(dq.peek());
+      dq.poll();
+      dq.peekElements(1, 1, s -> true);
+    }
+  }
+
+
+  @Test
+  public void testPeekElements() throws Exception {
+    String dqZNode = "/distqueue/test4";
+    byte[] data = "hello world".getBytes(UTF8);
+
+    DistributedQueue dq = makeDistributedQueue(dqZNode);
+
+    // Populate with data.
+    dq.offer(data);
+    dq.offer(data);
+    dq.offer(data);
+
+    Predicate<String> alwaysTrue = s -> true;
+    Predicate<String> alwaysFalse = s -> false;
+
+    // Should be able to get 0, 1, 2, or 3 instantly
+    for (int i = 0; i <= 3; ++i) {
+      assertEquals(i, dq.peekElements(i, 0, alwaysTrue).size());
+    }
+
+    // Asking for more should return only 3.
+    assertEquals(3, dq.peekElements(4, 0, alwaysTrue).size());
+
+    // If we filter everything out, we should block for the full time.
+    long start = System.nanoTime();
+    assertEquals(0, dq.peekElements(4, 1000, alwaysFalse).size());
+    assertTrue(System.nanoTime() - start >= TimeUnit.MILLISECONDS.toNanos(500));
+
+    // If someone adds a new matching element while we're waiting, we should return immediately.
+    executor.submit(() -> {
+      try {
+        Thread.sleep(500);
+        dq.offer(data);
+      } catch (Exception e) {
+        // ignore
+      }
+    });
+    start = System.nanoTime();
+    assertEquals(1, dq.peekElements(4, 2000, child -> {
+      // The 4th element in the queue will end with a "3".
+      return child.endsWith("3");
+    }).size());
+    long elapsed = System.nanoTime() - start;
+    assertTrue(elapsed < TimeUnit.MILLISECONDS.toNanos(1000));
+    assertTrue(elapsed >= TimeUnit.MILLISECONDS.toNanos(250));
+  }
+
+
+  protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
+    return new SimDistributedQueueFactory.SimDistributedQueue(dqZNode);
+  }
+
+  private static class QueueChangerThread extends Thread {
+
+    DistributedQueue dq;
+    long waitBeforeOfferMs;
+
+    QueueChangerThread(DistributedQueue dq, long waitBeforeOfferMs) {
+      this.dq = dq;
+      this.waitBeforeOfferMs = waitBeforeOfferMs;
+    }
+
+    public void run() {
+      try {
+        Thread.sleep(waitBeforeOfferMs);
+        dq.offer(getName().getBytes(UTF8));
+      } catch (InterruptedException ie) {
+        // do nothing
+      } catch (Exception exc) {
+        throw new RuntimeException(exc);
+      }
+    }
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    try {
+      super.tearDown();
+    } catch (Exception exc) {
+    }
+    executor.shutdown();
+  }
+
+}


[46/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11791: Add density Stream Evaluator

Posted by da...@apache.org.
SOLR-11791: Add density Stream Evaluator


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

Branch: refs/heads/jira/solr-11702
Commit: 6045066495093757b53e662fe49a36f7d740fafe
Parents: efbbc9e
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 22 10:41:42 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 22 10:41:42 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  1 +
 .../client/solrj/io/eval/DensityEvaluator.java  | 55 ++++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  6 ++-
 3 files changed, 61 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60450664/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 67d6f7f..fa1aaaa 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -285,6 +285,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("pairedTtest", PairedTTestEvaluator.class)
         .withFunctionName("multiVariateNormalDistribution", MultiVariateNormalDistributionEvaluator.class)
         .withFunctionName("integrate", IntegrateEvaluator.class)
+        .withFunctionName("density", DensityEvaluator.class)
 
         // Boolean Stream Evaluators
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60450664/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DensityEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DensityEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DensityEvaluator.java
new file mode 100644
index 0000000..4910e6f
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/DensityEvaluator.java
@@ -0,0 +1,55 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.List;
+
+import org.apache.commons.math3.distribution.MultivariateRealDistribution;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class DensityEvaluator extends RecursiveObjectEvaluator implements TwoValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public DensityEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+  }
+
+  @Override
+  public Object doWork(Object first, Object second) throws IOException{
+
+    if (!(first instanceof MultivariateRealDistribution)) {
+      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - found type %s for the first value, expecting a MultiVariateRealDistribution for density", toExpression(constructingFactory), first.getClass().getSimpleName()));
+    }
+    if (!(second instanceof List)) {
+      throw new IOException(String.format(Locale.ROOT, "Invalid expression %s - found type %s for the second value, expecting a numeric array.", toExpression(constructingFactory), first.getClass().getSimpleName()));
+    }
+
+    MultivariateRealDistribution multivariateRealDistribution = (MultivariateRealDistribution) first;
+    List<Number> nums = (List<Number>) second;
+
+    double[] vec = new double[nums.size()];
+
+    for(int i=0; i<vec.length; i++) {
+      vec[i] = nums.get(i).doubleValue();
+    }
+
+    return multivariateRealDistribution.density(vec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60450664/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index a9f0f66..c564700 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -7190,7 +7190,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         "     f=multiVariateNormalDistribution(d, e)," +
         "     g=sample(f, 10000)," +
         "     h=cov(g)," +
-        "     i=sample(f))";
+        "     i=sample(f)," +
+        "     j=density(f, array(4.016093243274465, 138.7283428008585)))";
 
     ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
     paramsLoc.set("expr", cexpr);
@@ -7224,6 +7225,9 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     Number sample2 = sample.get(1);
     assertTrue(sample1.doubleValue() > -30 && sample1.doubleValue() < 30);
     assertTrue(sample2.doubleValue() > 50 && sample2.doubleValue() < 250);
+
+    Number density = (Number)tuples.get(0).get("j");
+    assertEquals(density.doubleValue(), 0.007852638121596995, .00001);
   }
 
 


[05/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/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
new file mode 100644
index 0000000..1986bac
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -0,0 +1,1275 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.AddReplicaCmd;
+import org.apache.solr.cloud.Assign;
+import org.apache.solr.cloud.CreateCollectionCmd;
+import org.apache.solr.cloud.CreateShardCmd;
+import org.apache.solr.cloud.SplitShardCmd;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.ZkWriteCommand;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocRouter;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ReplicaPosition;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.CreateMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.params.CommonParams.NAME;
+
+/**
+ * Simulated {@link ClusterStateProvider}.
+ * <p>The following behaviors are supported:</p>
+ *   <ul>
+ *     <li>using autoscaling policy for replica placements</li>
+ *     <li>maintaining and up-to-date list of /live_nodes and nodeAdded / nodeLost markers</li>
+ *     <li>running a simulated leader election on collection changes (with throttling), when needed</li>
+ *     <li>maintaining an up-to-date /clusterstate.json (single file format), which also tracks replica states,
+ *     leader election changes, replica property changes, etc. Note: this file is only written,
+ *     but never read by the framework!</li>
+ *     <li>maintaining an up-to-date /clusterprops.json. Note: this file is only written, but never read by the
+ *     framework!</li>
+ *   </ul>
+ */
+public class SimClusterStateProvider implements ClusterStateProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, List<ReplicaInfo>> nodeReplicaMap = new ConcurrentHashMap<>();
+  private final LiveNodesSet liveNodes;
+  private final SimDistribStateManager stateManager;
+  private final SimCloudManager cloudManager;
+
+  private final Map<String, Object> clusterProperties = new ConcurrentHashMap<>();
+  private final Map<String, Map<String, Object>> collProperties = new ConcurrentHashMap<>();
+  private final Map<String, Map<String, Map<String, Object>>> sliceProperties = new ConcurrentHashMap<>();
+
+  private final ReentrantLock lock = new ReentrantLock();
+
+  private final ActionThrottle leaderThrottle;
+
+  // default map of: operation -> delay
+  private final Map<String, Long> defaultOpDelays = new HashMap<>();
+  // per-collection map of: collection -> op -> delay
+  private final Map<String, Map<String, Long>> opDelays = new ConcurrentHashMap<>();
+
+
+  private volatile int clusterStateVersion = -1;
+  private Map<String, Object> lastSavedProperties = null;
+
+  private AtomicReference<Map<String, DocCollection>> collectionsStatesRef = new AtomicReference<>();
+  private AtomicBoolean saveClusterState = new AtomicBoolean();
+
+  /**
+   * The instance needs to be initialized using the <code>sim*</code> methods in order
+   * to ensure proper behavior, otherwise it will behave as a cluster with zero replicas.
+   */
+  public SimClusterStateProvider(LiveNodesSet liveNodes, SimCloudManager cloudManager) throws Exception {
+    this.liveNodes = liveNodes;
+    for (String nodeId : liveNodes.get()) {
+      createEphemeralLiveNode(nodeId);
+    }
+    this.cloudManager = cloudManager;
+    this.stateManager = cloudManager.getSimDistribStateManager();
+    this.leaderThrottle = new ActionThrottle("leader", 5000, cloudManager.getTimeSource());
+    // names are CollectionAction operation names, delays are in ms (simulated time)
+    defaultOpDelays.put(CollectionParams.CollectionAction.MOVEREPLICA.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.DELETEREPLICA.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.ADDREPLICA.name(), 500L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.SPLITSHARD.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.CREATESHARD.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.DELETESHARD.name(), 5000L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.CREATE.name(), 500L);
+    defaultOpDelays.put(CollectionParams.CollectionAction.DELETE.name(), 5000L);
+  }
+
+  // ============== SIMULATOR SETUP METHODS ====================
+
+  /**
+   * Initialize from an existing cluster state
+   * @param initialState initial cluster state
+   */
+  public void simSetClusterState(ClusterState initialState) throws Exception {
+    lock.lock();
+    try {
+      collProperties.clear();
+      sliceProperties.clear();
+      nodeReplicaMap.clear();
+      liveNodes.clear();
+      for (String nodeId : stateManager.listData(ZkStateReader.LIVE_NODES_ZKNODE)) {
+        if (stateManager.hasData(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeId)) {
+          stateManager.removeData(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeId, -1);
+        }
+        if (stateManager.hasData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeId)) {
+          stateManager.removeData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeId, -1);
+        }
+      }
+      liveNodes.addAll(initialState.getLiveNodes());
+      for (String nodeId : liveNodes.get()) {
+        createEphemeralLiveNode(nodeId);
+      }
+      initialState.forEachCollection(dc -> {
+        collProperties.computeIfAbsent(dc.getName(), name -> new ConcurrentHashMap<>()).putAll(dc.getProperties());
+        opDelays.computeIfAbsent(dc.getName(), c -> new HashMap<>()).putAll(defaultOpDelays);
+        dc.getSlices().forEach(s -> {
+          sliceProperties.computeIfAbsent(dc.getName(), name -> new ConcurrentHashMap<>())
+              .computeIfAbsent(s.getName(), name -> new HashMap<>()).putAll(s.getProperties());
+          s.getReplicas().forEach(r -> {
+            ReplicaInfo ri = new ReplicaInfo(r.getName(), r.getCoreName(), dc.getName(), s.getName(), r.getType(), r.getNodeName(), r.getProperties());
+            if (liveNodes.get().contains(r.getNodeName())) {
+              nodeReplicaMap.computeIfAbsent(r.getNodeName(), rn -> new ArrayList<>()).add(ri);
+            }
+          });
+        });
+      });
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Reset the leader election throttle.
+   */
+  public void simResetLeaderThrottle() {
+    leaderThrottle.reset();
+  }
+
+  /**
+   * Get random node id.
+   * @param random instance of random.
+   * @return one of the live nodes
+   */
+  public String simGetRandomNode(Random random) {
+    if (liveNodes.isEmpty()) {
+      return null;
+    }
+    List<String> nodes = new ArrayList<>(liveNodes.get());
+    return nodes.get(random.nextInt(nodes.size()));
+  }
+
+  // todo: maybe hook up DistribStateManager /clusterstate.json watchers?
+
+  /**
+   * Add a new node to the cluster.
+   * @param nodeId unique node id
+   */
+  public void simAddNode(String nodeId) throws Exception {
+    if (liveNodes.contains(nodeId)) {
+      throw new Exception("Node " + nodeId + " already exists");
+    }
+    liveNodes.add(nodeId);
+    createEphemeralLiveNode(nodeId);
+    nodeReplicaMap.putIfAbsent(nodeId, new ArrayList<>());
+  }
+
+  // utility class to run leader election in a separate thread and with throttling
+  // Note: leader election is a no-op if a shard leader already exists for each shard
+  private class LeaderElection implements Callable<Boolean> {
+    Collection<String> collections;
+    boolean saveClusterState;
+
+    LeaderElection(Collection<String> collections, boolean saveClusterState) {
+      this.collections = collections;
+      this.saveClusterState = saveClusterState;
+    }
+
+    @Override
+    public Boolean call() {
+      leaderThrottle.minimumWaitBetweenActions();
+      leaderThrottle.markAttemptingAction();
+      try {
+        simRunLeaderElection(collections, saveClusterState);
+      } catch (Exception e) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Remove node from a cluster. This is equivalent to a situation when a node is lost.
+   * All replicas that were assigned to this node are marked as DOWN.
+   * @param nodeId node id
+   * @return true if a node existed and was removed
+   */
+  public boolean simRemoveNode(String nodeId) throws Exception {
+    lock.lock();
+    try {
+      Set<String> collections = new HashSet<>();
+      // mark every replica on that node as down
+      setReplicaStates(nodeId, Replica.State.DOWN, collections);
+      boolean res = liveNodes.remove(nodeId);
+      if (!collections.isEmpty()) {
+        collectionsStatesRef.set(null);
+      }
+      // remove ephemeral nodes
+      stateManager.getRoot().removeEphemeralChildren(nodeId);
+      // create a nodeLost marker if needed
+      AutoScalingConfig cfg = stateManager.getAutoScalingConfig(null);
+      if (cfg.hasTriggerForEvents(TriggerEventType.NODELOST)) {
+        stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + nodeId);
+      }
+      if (!collections.isEmpty()) {
+        cloudManager.submit(new LeaderElection(collections, true));
+      }
+      return res;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  // this method needs to be called under a lock
+  private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
+    List<ReplicaInfo> replicas = nodeReplicaMap.get(nodeId);
+    if (replicas != null) {
+      replicas.forEach(r -> {
+        r.getVariables().put(ZkStateReader.STATE_PROP, state.toString());
+        changedCollections.add(r.getCollection());
+      });
+    }
+  }
+
+  // this method needs to be called under a lock
+  private void createEphemeralLiveNode(String nodeId) throws Exception {
+    DistribStateManager mgr = stateManager.withEphemeralId(nodeId);
+    mgr.makePath(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeId, null, CreateMode.EPHEMERAL, true);
+    AutoScalingConfig cfg = stateManager.getAutoScalingConfig(null);
+    if (cfg.hasTriggerForEvents(TriggerEventType.NODEADDED)) {
+      mgr.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeId, null, CreateMode.EPHEMERAL, true);
+    }
+  }
+
+  /**
+   * Restore a previously removed node. This also simulates a short replica recovery state.
+   * @param nodeId node id to restore
+   * @return true when this operation restored any replicas, false otherwise (empty node).
+   */
+  public boolean simRestoreNode(String nodeId) throws Exception {
+    liveNodes.add(nodeId);
+    createEphemeralLiveNode(nodeId);
+    Set<String> collections = new HashSet<>();
+    lock.lock();
+    try {
+      setReplicaStates(nodeId, Replica.State.RECOVERING, collections);
+    } finally {
+      lock.unlock();
+    }
+    cloudManager.getTimeSource().sleep(1000);
+    lock.lock();
+    try {
+      setReplicaStates(nodeId, Replica.State.ACTIVE, collections);
+    } finally {
+      lock.unlock();
+    }
+    if (!collections.isEmpty()) {
+      collectionsStatesRef.set(null);
+      cloudManager.submit(new LeaderElection(collections, true));
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Add a new replica. Note that if any details of a replica (node, coreNodeName, SolrCore name, etc)
+   * are missing they will be filled in using the policy framework.
+   * @param message replica details
+   * @param results result of the operation
+   */
+  public void simAddReplica(ZkNodeProps message, NamedList results) throws Exception {
+    ClusterState clusterState = getClusterState();
+    DocCollection coll = clusterState.getCollection(message.getStr(ZkStateReader.COLLECTION_PROP));
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+    message = AddReplicaCmd.assignReplicaDetails(cloudManager, clusterState, message, sessionWrapper);
+    if (sessionWrapper.get() != null) {
+      sessionWrapper.get().release();
+    }
+    if (message.getStr(CoreAdminParams.CORE_NODE_NAME) == null) {
+      message = message.plus(CoreAdminParams.CORE_NODE_NAME, Assign.assignCoreNodeName(stateManager, coll));
+    }
+    ReplicaInfo ri = new ReplicaInfo(
+        message.getStr(CoreAdminParams.CORE_NODE_NAME),
+        message.getStr(CoreAdminParams.NAME),
+        message.getStr(ZkStateReader.COLLECTION_PROP),
+        message.getStr(ZkStateReader.SHARD_ID_PROP),
+        Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT)),
+        message.getStr(CoreAdminParams.NODE),
+        message.getProperties()
+    );
+    simAddReplica(message.getStr(CoreAdminParams.NODE), ri, true);
+    results.add("success", "");
+  }
+
+  /**
+   * Add a replica. Note that all details of the replica must be present here, including
+   * node, coreNodeName and SolrCore name.
+   * @param nodeId node id where the replica will be added
+   * @param replicaInfo replica info
+   * @param runLeaderElection if true then run a leader election after adding the replica.
+   */
+  public void simAddReplica(String nodeId, ReplicaInfo replicaInfo, boolean runLeaderElection) throws Exception {
+    // make sure coreNodeName is unique across cluster
+    for (Map.Entry<String, List<ReplicaInfo>> e : nodeReplicaMap.entrySet()) {
+      for (ReplicaInfo ri : e.getValue()) {
+        if (ri.getCore().equals(replicaInfo.getCore())) {
+          throw new Exception("Duplicate SolrCore name for existing=" + ri + " on node " + e.getKey() + " and new=" + replicaInfo);
+        }
+        if (ri.getName().equals(replicaInfo.getName())) {
+          throw new Exception("Duplicate coreNode name for existing=" + ri + " on node " + e.getKey() + " and new=" + replicaInfo);
+        }
+      }
+    }
+    if (!liveNodes.contains(nodeId)) {
+      throw new Exception("Target node " + nodeId + " is not live: " + liveNodes);
+    }
+    // verify info
+    if (replicaInfo.getCore() == null) {
+      throw new Exception("Missing core: " + replicaInfo);
+    }
+    // XXX replica info is not supposed to have this as a variable
+    replicaInfo.getVariables().remove(ZkStateReader.SHARD_ID_PROP);
+    if (replicaInfo.getName() == null) {
+      throw new Exception("Missing name: " + replicaInfo);
+    }
+    if (replicaInfo.getNode() == null) {
+      throw new Exception("Missing node: " + replicaInfo);
+    }
+    if (!replicaInfo.getNode().equals(nodeId)) {
+      throw new Exception("Wrong node (not " + nodeId + "): " + replicaInfo);
+    }
+
+    lock.lock();
+    try {
+
+      opDelay(replicaInfo.getCollection(), CollectionParams.CollectionAction.ADDREPLICA.name());
+
+      List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent(nodeId, n -> new ArrayList<>());
+      // mark replica as active
+      replicaInfo.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
+      // add a property expected in tests
+      replicaInfo.getVariables().put(Suggestion.coreidxsize, 123450000);
+
+      replicas.add(replicaInfo);
+      // at this point nuke our cached DocCollection state
+      collectionsStatesRef.set(null);
+      LOG.trace("-- simAddReplica {}", replicaInfo);
+
+      Map<String, Object> values = cloudManager.getSimNodeStateProvider().simGetAllNodeValues()
+          .computeIfAbsent(nodeId, id -> new ConcurrentHashMap<>(SimCloudManager.createNodeValues(id)));
+      // update the number of cores and freedisk in node values
+      Integer cores = (Integer)values.get(ImplicitSnitch.CORES);
+      if (cores == null) {
+        cores = 0;
+      }
+      cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.CORES, cores + 1);
+      Integer disk = (Integer)values.get(ImplicitSnitch.DISK);
+      if (disk == null) {
+        disk = 1000;
+      }
+      cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk - 10);
+      if (runLeaderElection) {
+        cloudManager.submit(new LeaderElection(Collections.singleton(replicaInfo.getCollection()), true));
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Remove replica.
+   * @param nodeId node id
+   * @param coreNodeName coreNodeName
+   */
+  public void simRemoveReplica(String nodeId, String coreNodeName) throws Exception {
+    List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent(nodeId, n -> new ArrayList<>());
+    lock.lock();
+    try {
+      for (int i = 0; i < replicas.size(); i++) {
+        if (coreNodeName.equals(replicas.get(i).getName())) {
+          ReplicaInfo ri = replicas.remove(i);
+          collectionsStatesRef.set(null);
+
+          opDelay(ri.getCollection(), CollectionParams.CollectionAction.DELETEREPLICA.name());
+
+          // update the number of cores in node values, if node is live
+          if (liveNodes.contains(nodeId)) {
+            Integer cores = (Integer)cloudManager.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.CORES);
+            if (cores == null || cores == 0) {
+              throw new Exception("Unexpected value of 'cores' (" + cores + ") on node: " + nodeId);
+            }
+            cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.CORES, cores - 1);
+            Integer disk = (Integer)cloudManager.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.DISK);
+            if (disk == null || disk == 0) {
+              throw new Exception("Unexpected value of 'freedisk' (" + disk + ") on node: " + nodeId);
+            }
+            cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk + 10);
+          }
+          LOG.trace("-- simRemoveReplica {}", ri);
+          cloudManager.submit(new LeaderElection(Collections.singleton(ri.getCollection()), true));
+          return;
+        }
+      }
+      throw new Exception("Replica " + coreNodeName + " not found on node " + nodeId);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Save clusterstate.json to {@link DistribStateManager}.
+   * @return saved state
+   */
+  private ClusterState saveClusterState(ClusterState state) throws IOException {
+    byte[] data = Utils.toJSON(state);
+    try {
+      VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_STATE);
+      int version = oldData != null ? oldData.getVersion() : -1;
+      stateManager.setData(ZkStateReader.CLUSTER_STATE, data, version);
+      LOG.trace("-- saved cluster state version=" + clusterStateVersion +
+          ", zkVersion=" + (version + 1) + ", {}", state);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return state;
+  }
+
+  /**
+   * Delay an operation by a configured amount.
+   * @param collection collection name
+   * @param op operation name.
+   */
+  private void opDelay(String collection, String op) throws InterruptedException {
+    Map<String, Long> delays = opDelays.get(collection);
+    if (delays == null || delays.isEmpty() || !delays.containsKey(op)) {
+      return;
+    }
+    cloudManager.getTimeSource().sleep(delays.get(op));
+  }
+
+  /**
+   * Simulate running a shard leader election. This operation is a no-op if a leader already exists.
+   * If a new leader is elected the cluster state is saved.
+   * @param collections list of affected collections
+   * @param saveClusterState if true then save cluster state regardless of changes.
+   */
+  private synchronized void simRunLeaderElection(Collection<String> collections, boolean saveClusterState) throws Exception {
+    ClusterState state = getClusterState();
+    AtomicBoolean stateChanged = new AtomicBoolean(Boolean.FALSE);
+
+    state.forEachCollection(dc -> {
+      if (!collections.contains(dc.getName())) {
+        return;
+      }
+      dc.getSlices().forEach(s -> {
+        Replica leader = s.getLeader();
+        if (leader == null || !liveNodes.contains(leader.getNodeName())) {
+          LOG.trace("Running leader election for " + dc.getName() + " / " + s.getName());
+          if (s.getReplicas().isEmpty()) { // no replicas - punt
+            return;
+          }
+          // mark all replicas as non-leader (probably not necessary) and collect all active and live
+          List<ReplicaInfo> active = new ArrayList<>();
+          s.getReplicas().forEach(r -> {
+            AtomicReference<ReplicaInfo> riRef = new AtomicReference<>();
+            // find our ReplicaInfo for this replica
+            nodeReplicaMap.get(r.getNodeName()).forEach(info -> {
+              if (info.getName().equals(r.getName())) {
+                riRef.set(info);
+              }
+            });
+            ReplicaInfo ri = riRef.get();
+            if (ri == null) {
+              throw new IllegalStateException("-- could not find ReplicaInfo for replica " + r);
+            }
+            synchronized (ri) {
+              if (ri.getVariables().remove(ZkStateReader.LEADER_PROP) != null) {
+                stateChanged.set(true);
+              }
+              if (r.isActive(liveNodes.get())) {
+                active.add(ri);
+              } else { // if it's on a node that is not live mark it down
+                if (!liveNodes.contains(r.getNodeName())) {
+                  ri.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
+                }
+              }
+            }
+          });
+          if (active.isEmpty()) {
+            LOG.warn("-- can't find any active replicas for " + dc.getName() + " / " + s.getName());
+            return;
+          }
+          // pick first active one
+          ReplicaInfo ri = null;
+          for (ReplicaInfo a : active) {
+            if (!a.getType().equals(Replica.Type.PULL)) {
+              ri = a;
+              break;
+            }
+          }
+          if (ri == null) {
+            LOG.warn("-- can't find any suitable replica type for " + dc.getName() + " / " + s.getName());
+            return;
+          }
+          synchronized (ri) {
+            ri.getVariables().put(ZkStateReader.LEADER_PROP, "true");
+          }
+          stateChanged.set(true);
+          LOG.debug("-- elected new leader for " + dc.getName() + " / " + s.getName() + ": " + ri);
+        } else {
+          LOG.trace("-- already has leader for {} / {}", dc.getName(), s.getName());
+        }
+      });
+    });
+    if (saveClusterState || stateChanged.get()) {
+      collectionsStatesRef.set(null);
+    }
+  }
+
+  /**
+   * Create a new collection. This operation uses policy framework for node and replica assignments.
+   * @param props collection details
+   * @param results results of the operation.
+   */
+  public void simCreateCollection(ZkNodeProps props, NamedList results) throws Exception {
+    if (props.getStr(CommonAdminParams.ASYNC) != null) {
+      results.add(CoreAdminParams.REQUESTID, props.getStr(CommonAdminParams.ASYNC));
+    }
+    boolean waitForFinalState = props.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
+    List<String> nodeList = new ArrayList<>();
+    List<String> shardNames = new ArrayList<>();
+    final String collectionName = props.getStr(NAME);
+    ClusterState clusterState = getClusterState();
+    ZkWriteCommand cmd = new ClusterStateMutator(cloudManager).createCollection(clusterState, props);
+    if (cmd.noop) {
+      LOG.warn("Collection {} already exists. exit", collectionName);
+      results.add("success", "no-op");
+      return;
+    }
+    opDelays.computeIfAbsent(collectionName, c -> new HashMap<>()).putAll(defaultOpDelays);
+
+    opDelay(collectionName, CollectionParams.CollectionAction.CREATE.name());
+
+    AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+    List<ReplicaPosition> replicaPositions = CreateCollectionCmd.buildReplicaPositions(cloudManager, getClusterState(), props,
+        nodeList, shardNames, sessionWrapper);
+    if (sessionWrapper.get() != null) {
+      sessionWrapper.get().release();
+    }
+    final CountDownLatch finalStateLatch = new CountDownLatch(replicaPositions.size());
+    AtomicInteger replicaNum = new AtomicInteger(1);
+    replicaPositions.forEach(pos -> {
+      Map<String, Object> replicaProps = new HashMap<>();
+      replicaProps.put(ZkStateReader.NODE_NAME_PROP, pos.node);
+      replicaProps.put(ZkStateReader.REPLICA_TYPE, pos.type.toString());
+      String coreName = String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, pos.shard, pos.type.name().substring(0,1).toLowerCase(Locale.ROOT),
+          replicaNum.getAndIncrement());
+      try {
+        replicaProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+        ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+            coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps);
+        cloudManager.submit(() -> {
+          simAddReplica(pos.node, ri, false);
+          finalStateLatch.countDown();
+          return true;
+        });
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    });
+    // add collection props
+    DocCollection coll = cmd.collection;
+    collProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()).putAll(coll.getProperties());
+    // add slice props
+    coll.getSlices().forEach(s -> {
+      Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll.getName(), c -> new ConcurrentHashMap<>())
+          .computeIfAbsent(s.getName(), slice -> new ConcurrentHashMap<>());
+      s.getProperties().forEach((k, v) -> {
+        if (k != null && v != null) {
+          sliceProps.put(k, v);
+        }
+      });
+    });
+    cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+    if (waitForFinalState) {
+      boolean finished = finalStateLatch.await(cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, 60, TimeUnit.MILLISECONDS),
+          TimeUnit.MILLISECONDS);
+      if (!finished) {
+        results.add("failure", "Timeout waiting for all replicas to become active.");
+        return;
+      }
+    }
+    results.add("success", "");
+  }
+
+  /**
+   * Delete a collection
+   * @param collection collection name
+   * @param async async id
+   * @param results results of the operation
+   */
+  public void simDeleteCollection(String collection, String async, NamedList results) throws IOException {
+    if (async != null) {
+      results.add(CoreAdminParams.REQUESTID, async);
+    }
+    lock.lock();
+    try {
+      collProperties.remove(collection);
+      sliceProperties.remove(collection);
+
+      opDelay(collection, CollectionParams.CollectionAction.DELETE.name());
+
+      opDelays.remove(collection);
+      nodeReplicaMap.forEach((n, replicas) -> {
+        for (Iterator<ReplicaInfo> it = replicas.iterator(); it.hasNext(); ) {
+          ReplicaInfo ri = it.next();
+          if (ri.getCollection().equals(collection)) {
+            it.remove();
+            // update the number of cores in node values
+            Integer cores = (Integer) cloudManager.getSimNodeStateProvider().simGetNodeValue(n, "cores");
+            if (cores != null) { // node is still up
+              if (cores == 0) {
+                throw new RuntimeException("Unexpected value of 'cores' (" + cores + ") on node: " + n);
+              }
+              cloudManager.getSimNodeStateProvider().simSetNodeValue(n, "cores", cores - 1);
+            }
+          }
+        }
+      });
+      collectionsStatesRef.set(null);
+      results.add("success", "");
+    } catch (Exception e) {
+      LOG.warn("Exception", e);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Remove all collections.
+   */
+  public void simDeleteAllCollections() throws Exception {
+    lock.lock();
+    try {
+      nodeReplicaMap.clear();
+      collProperties.clear();
+      sliceProperties.clear();
+      cloudManager.getSimNodeStateProvider().simGetAllNodeValues().forEach((n, values) -> {
+        values.put("cores", 0);
+      });
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Move replica. This uses a similar algorithm as {@link org.apache.solr.cloud.MoveReplicaCmd#moveNormalReplica(ClusterState, NamedList, String, String, DocCollection, Replica, Slice, int, boolean)}.
+   * @param message operation details
+   * @param results operation results.
+   */
+  public void simMoveReplica(ZkNodeProps message, NamedList results) throws Exception {
+    if (message.getStr(CommonAdminParams.ASYNC) != null) {
+      results.add(CoreAdminParams.REQUESTID, message.getStr(CommonAdminParams.ASYNC));
+    }
+    String collection = message.getStr(COLLECTION_PROP);
+    String targetNode = message.getStr("targetNode");
+    ClusterState clusterState = getClusterState();
+    DocCollection coll = clusterState.getCollection(collection);
+    if (coll == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
+    }
+    String replicaName = message.getStr(REPLICA_PROP);
+    Replica replica = coll.getReplica(replicaName);
+    if (replica == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Collection: " + collection + " replica: " + replicaName + " does not exist");
+    }
+    Slice slice = null;
+    for (Slice s : coll.getSlices()) {
+      if (s.getReplicas().contains(replica)) {
+        slice = s;
+      }
+    }
+    if (slice == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Replica has no 'slice' property! : " + replica);
+    }
+
+    opDelay(collection, CollectionParams.CollectionAction.MOVEREPLICA.name());
+
+    // TODO: for now simulate moveNormalReplica sequence, where we first add new replica and then delete the old one
+
+    String newSolrCoreName = Assign.buildSolrCoreName(stateManager, coll, slice.getName(), replica.getType());
+    String coreNodeName = Assign.assignCoreNodeName(stateManager, coll);
+    ReplicaInfo newReplica = new ReplicaInfo(coreNodeName, newSolrCoreName, collection, slice.getName(), replica.getType(), targetNode, null);
+    LOG.debug("-- new replica: " + newReplica);
+    // xxx should run leader election here already?
+    simAddReplica(targetNode, newReplica, false);
+    // this will trigger leader election
+    simRemoveReplica(replica.getNodeName(), replica.getName());
+    results.add("success", "");
+  }
+
+  /**
+   * Create a new shard. This uses a similar algorithm as {@link CreateShardCmd}.
+   * @param message operation details
+   * @param results operation results
+   */
+  public void simCreateShard(ZkNodeProps message, NamedList results) throws Exception {
+    if (message.getStr(CommonAdminParams.ASYNC) != null) {
+      results.add(CoreAdminParams.REQUESTID, message.getStr(CommonAdminParams.ASYNC));
+    }
+    String collectionName = message.getStr(COLLECTION_PROP);
+    String sliceName = message.getStr(SHARD_ID_PROP);
+    ClusterState clusterState = getClusterState();
+    lock.lock();
+    try {
+      ZkWriteCommand cmd = new CollectionMutator(cloudManager).createShard(clusterState, message);
+      if (cmd.noop) {
+        results.add("success", "no-op");
+        return;
+      }
+
+      opDelay(collectionName, CollectionParams.CollectionAction.CREATESHARD.name());
+
+      // copy shard properties -- our equivalent of creating an empty shard in cluster state
+      DocCollection collection = cmd.collection;
+      Slice slice = collection.getSlice(sliceName);
+      Map<String, Object> props = sliceProperties.computeIfAbsent(collection.getName(), c -> new ConcurrentHashMap<>())
+          .computeIfAbsent(sliceName, s -> new ConcurrentHashMap<>());
+      props.clear();
+      slice.getProperties().entrySet().stream()
+          .filter(e -> !e.getKey().equals("range"))
+          .filter(e -> !e.getKey().equals("replicas"))
+          .forEach(e -> props.put(e.getKey(), e.getValue()));
+      // 2. create new replicas
+      AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
+      List<ReplicaPosition> positions = CreateShardCmd.buildReplicaPositions(cloudManager, clusterState, collectionName,
+          message, sessionWrapper);
+      if (sessionWrapper.get() != null) {
+        sessionWrapper.get().release();
+      }
+      AtomicInteger replicaNum = new AtomicInteger(1);
+      positions.forEach(pos -> {
+        Map<String, Object> replicaProps = new HashMap<>();
+        replicaProps.put(ZkStateReader.SHARD_ID_PROP, pos.shard);
+        replicaProps.put(ZkStateReader.NODE_NAME_PROP, pos.node);
+        replicaProps.put(ZkStateReader.REPLICA_TYPE, pos.type.toString());
+        replicaProps.put(ZkStateReader.BASE_URL_PROP, Utils.getBaseUrlForNodeName(pos.node, "http"));
+        String coreName = String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, pos.shard, pos.type.name().substring(0,1).toLowerCase(Locale.ROOT),
+            replicaNum.getAndIncrement());
+        try {
+          replicaProps.put(ZkStateReader.CORE_NAME_PROP, coreName);
+          ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+              coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps);
+          simAddReplica(pos.node, ri, false);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      });
+      Map<String, Object> colProps = collProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>());
+
+      cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+      results.add("success", "");
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Split a shard. This uses a similar algorithm as {@link SplitShardCmd}, including simulating its
+   * quirks, and leaving the original parent slice in place.
+   * @param message operation details
+   * @param results operation results.
+   */
+  public void simSplitShard(ZkNodeProps message, NamedList results) throws Exception {
+    String collectionName = message.getStr(COLLECTION_PROP);
+    AtomicReference<String> sliceName = new AtomicReference<>();
+    sliceName.set(message.getStr(SHARD_ID_PROP));
+    String splitKey = message.getStr("split.key");
+    ClusterState clusterState = getClusterState();
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice parentSlice = SplitShardCmd.getParentSlice(clusterState, collectionName, sliceName, splitKey);
+    List<DocRouter.Range> subRanges = new ArrayList<>();
+    List<String> subSlices = new ArrayList<>();
+    List<String> subShardNames = new ArrayList<>();
+
+    opDelay(collectionName, CollectionParams.CollectionAction.SPLITSHARD.name());
+
+    SplitShardCmd.fillRanges(cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames);
+    // mark the old slice as inactive
+    sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>())
+        .computeIfAbsent(sliceName.get(), s -> new ConcurrentHashMap<>())
+        .put(ZkStateReader.SHARD_STATE_PROP, Slice.State.INACTIVE.toString());
+    // add slice props
+    for (int i = 0; i < subRanges.size(); i++) {
+      String subSlice = subSlices.get(i);
+      DocRouter.Range range = subRanges.get(i);
+      Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>())
+          .computeIfAbsent(subSlice, ss -> new ConcurrentHashMap<>());
+      sliceProps.put(Slice.RANGE, range);
+      sliceProps.put(Slice.PARENT, sliceName.get());
+      sliceProps.put(ZkStateReader.SHARD_STATE_PROP, Slice.State.ACTIVE.toString());
+    }
+    // add replicas for new subShards
+    int repFactor = parentSlice.getReplicas().size();
+    List<ReplicaPosition> replicaPositions = Assign.identifyNodes(cloudManager,
+        clusterState,
+        new ArrayList<>(clusterState.getLiveNodes()),
+        collectionName,
+        new ZkNodeProps(collection.getProperties()),
+        // reproduce the bug
+        subSlices, repFactor, 0, 0);
+    PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
+    if (sessionWrapper != null) sessionWrapper.release();
+
+    for (ReplicaPosition replicaPosition : replicaPositions) {
+      String subSliceName = replicaPosition.shard;
+      String subShardNodeName = replicaPosition.node;
+      String solrCoreName = collectionName + "_" + subSliceName + "_replica" + (replicaPosition.index);
+      Map<String, Object> replicaProps = new HashMap<>();
+      replicaProps.put(ZkStateReader.SHARD_ID_PROP, replicaPosition.shard);
+      replicaProps.put(ZkStateReader.NODE_NAME_PROP, replicaPosition.node);
+      replicaProps.put(ZkStateReader.REPLICA_TYPE, replicaPosition.type.toString());
+      replicaProps.put(ZkStateReader.BASE_URL_PROP, Utils.getBaseUrlForNodeName(subShardNodeName, "http"));
+
+      ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+          solrCoreName, collectionName, replicaPosition.shard, replicaPosition.type, subShardNodeName, replicaProps);
+      simAddReplica(replicaPosition.node, ri, false);
+    }
+    cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+    results.add("success", "");
+
+  }
+
+  /**
+   * Delete a shard. This uses a similar algorithm as {@link org.apache.solr.cloud.DeleteShardCmd}
+   * @param message operation details
+   * @param results operation results
+   */
+  public void simDeleteShard(ZkNodeProps message, NamedList results) throws Exception {
+    String collectionName = message.getStr(COLLECTION_PROP);
+    String sliceName = message.getStr(SHARD_ID_PROP);
+    ClusterState clusterState = getClusterState();
+    DocCollection collection = clusterState.getCollection(collectionName);
+    if (collection == null) {
+      throw new Exception("Collection " + collectionName + " doesn't exist");
+    }
+    Slice slice = collection.getSlice(sliceName);
+    if (slice == null) {
+      throw new Exception(" Collection " + collectionName + " slice " + sliceName + " doesn't exist.");
+    }
+
+    opDelay(collectionName, CollectionParams.CollectionAction.DELETESHARD.name());
+
+    lock.lock();
+    try {
+      sliceProperties.computeIfAbsent(collectionName, coll -> new ConcurrentHashMap<>()).remove(sliceName);
+      nodeReplicaMap.forEach((n, replicas) -> {
+        Iterator<ReplicaInfo> it = replicas.iterator();
+        while (it.hasNext()) {
+          ReplicaInfo ri = it.next();
+          if (ri.getCollection().equals(collectionName) && ri.getShard().equals(sliceName)) {
+            it.remove();
+          }
+        }
+      });
+      collectionsStatesRef.set(null);
+      results.add("success", "");
+    } catch (Exception e) {
+      results.add("failure", e.toString());
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Saves cluster properties to clusterprops.json.
+   * @return current properties
+   */
+  private synchronized Map<String, Object> saveClusterProperties() throws Exception {
+    if (lastSavedProperties != null && lastSavedProperties.equals(clusterProperties)) {
+      return lastSavedProperties;
+    }
+    byte[] data = Utils.toJSON(clusterProperties);
+    VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_PROPS);
+    int version = oldData != null ? oldData.getVersion() : -1;
+    stateManager.setData(ZkStateReader.CLUSTER_PROPS, data, version);
+    lastSavedProperties = (Map)Utils.fromJSON(data);
+    return lastSavedProperties;
+  }
+
+  /**
+   * Set all cluster properties. This also updates the clusterprops.json data in
+   * {@link DistribStateManager}
+   * @param properties properties to set
+   */
+  public void simSetClusterProperties(Map<String, Object> properties) throws Exception {
+    lock.lock();
+    try {
+      clusterProperties.clear();
+      if (properties != null) {
+        this.clusterProperties.putAll(properties);
+      }
+      saveClusterProperties();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set a cluster property. This also updates the clusterprops.json data in
+   * {@link DistribStateManager}
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetClusterProperty(String key, Object value) throws Exception {
+    lock.lock();
+    try {
+      if (value != null) {
+        clusterProperties.put(key, value);
+      } else {
+        clusterProperties.remove(key);
+      }
+      saveClusterProperties();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set collection properties.
+   * @param coll collection name
+   * @param properties properties
+   */
+  public void simSetCollectionProperties(String coll, Map<String, Object> properties) throws Exception {
+    lock.lock();
+    try {
+      if (properties == null) {
+        collProperties.remove(coll);
+      } else {
+        Map<String, Object> props = collProperties.computeIfAbsent(coll, c -> new HashMap<>());
+        props.clear();
+        props.putAll(properties);
+      }
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set collection property.
+   * @param coll collection name
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetCollectionProperty(String coll, String key, String value) throws Exception {
+    Map<String, Object> props = collProperties.computeIfAbsent(coll, c -> new HashMap<>());
+    lock.lock();
+    try {
+      if (value == null) {
+        props.remove(key);
+      } else {
+        props.put(key, value);
+      }
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set slice properties.
+   * @param coll collection name
+   * @param slice slice name
+   * @param properties slice properties
+   */
+  public void simSetSliceProperties(String coll, String slice, Map<String, Object> properties) throws Exception {
+    Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll, c -> new HashMap<>()).computeIfAbsent(slice, s -> new HashMap<>());
+    lock.lock();
+    try {
+      sliceProps.clear();
+      if (properties != null) {
+        sliceProps.putAll(properties);
+      }
+      collectionsStatesRef.set(null);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica.
+   * @param collection collection name
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetCollectionValue(String collection, String key, Object value) throws Exception {
+    simSetCollectionValue(collection, key, value, false);
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica.
+   * @param collection collection name
+   * @param key property name
+   * @param value property value
+   * @param divide if the value is a {@link Number} and this param is true, then the value will be evenly
+   *               divided by the number of replicas.
+   */
+  public void simSetCollectionValue(String collection, String key, Object value, boolean divide) throws Exception {
+    simSetShardValue(collection, null, key, value, divide);
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica in a selected shard.
+   * @param collection collection name
+   * @param shard shard name. If null then all shards will be affected.
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetShardValue(String collection, String shard, String key, Object value) throws Exception {
+    simSetShardValue(collection, shard, key, value, false);
+  }
+
+  /**
+   * Set per-collection value (eg. a metric). This value will be applied to each replica in a selected shard.
+   * @param collection collection name
+   * @param shard shard name. If null then all shards will be affected.
+   * @param key property name
+   * @param value property value
+   * @param divide if the value is a {@link Number} and this is true, then the value will be evenly
+   *               divided by the number of replicas.
+   */
+  public void simSetShardValue(String collection, String shard, String key, Object value, boolean divide) throws Exception {
+    List<ReplicaInfo> infos = new ArrayList<>();
+    nodeReplicaMap.forEach((n, replicas) -> {
+      replicas.forEach(r -> {
+        if (r.getCollection().equals(collection)) {
+          if (shard != null && !shard.equals(r.getShard())) {
+            return;
+          }
+          infos.add(r);
+        }
+      });
+    });
+    if (infos.isEmpty()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection " + collection + " doesn't exist.");
+    }
+    if (divide && value != null && (value instanceof Number)) {
+      value = ((Number)value).doubleValue() / infos.size();
+    }
+    for (ReplicaInfo r : infos) {
+      synchronized (r) {
+        if (value == null) {
+          r.getVariables().remove(key);
+        } else {
+          r.getVariables().put(key, value);
+        }
+      }
+    }
+  }
+
+  /**
+   * Return all replica infos for a node.
+   * @param node node id
+   * @return list of replicas on that node, or empty list if none
+   */
+  public List<ReplicaInfo> simGetReplicaInfos(String node) {
+    List<ReplicaInfo> replicas = nodeReplicaMap.get(node);
+    if (replicas == null) {
+      return Collections.emptyList();
+    } else {
+      return replicas;
+    }
+  }
+
+  /**
+   * List collections.
+   * @return list of existing collections.
+   */
+  public List<String> simListCollections() {
+    final Set<String> collections = new HashSet<>();
+    lock.lock();
+    try {
+      nodeReplicaMap.forEach((n, replicas) -> {
+        replicas.forEach(ri -> collections.add(ri.getCollection()));
+      });
+      return new ArrayList<>(collections);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  // interface methods
+
+  @Override
+  public ClusterState.CollectionRef getState(String collection) {
+    try {
+      return getClusterState().getCollectionRef(collection);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public Set<String> getLiveNodes() {
+    return liveNodes.get();
+  }
+
+  @Override
+  public List<String> resolveAlias(String alias) {
+    throw new UnsupportedOperationException("resolveAlias not implemented");
+  }
+
+  @Override
+  public ClusterState getClusterState() throws IOException {
+    Map<String, DocCollection> states = getCollectionStates();
+    ClusterState state = new ClusterState(clusterStateVersion, liveNodes.get(), states);
+    if (saveClusterState.getAndSet(false)) {
+      saveClusterState(state);
+    }
+    return state;
+  }
+
+  private Map<String, DocCollection> getCollectionStates() {
+    Map<String, DocCollection> collectionStates = collectionsStatesRef.get();
+    if (collectionStates != null) {
+      return collectionStates;
+    }
+    lock.lock();
+    collectionsStatesRef.set(null);
+    clusterStateVersion++;
+    saveClusterState.set(true);
+    try {
+      Map<String, Map<String, Map<String, Replica>>> collMap = new HashMap<>();
+      nodeReplicaMap.forEach((n, replicas) -> {
+        replicas.forEach(ri -> {
+          Map<String, Object> props;
+          synchronized (ri) {
+            props = new HashMap<>(ri.getVariables());
+          }
+          props.put(ZkStateReader.NODE_NAME_PROP, n);
+          props.put(ZkStateReader.CORE_NAME_PROP, ri.getCore());
+          props.put(ZkStateReader.REPLICA_TYPE, ri.getType().toString());
+          props.put(ZkStateReader.STATE_PROP, ri.getState().toString());
+          Replica r = new Replica(ri.getName(), props);
+          collMap.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
+              .computeIfAbsent(ri.getShard(), s -> new HashMap<>())
+              .put(ri.getName(), r);
+        });
+      });
+
+      // add empty slices
+      sliceProperties.forEach((c, perSliceProps) -> {
+        perSliceProps.forEach((slice, props) -> {
+          collMap.computeIfAbsent(c, co -> new ConcurrentHashMap<>()).computeIfAbsent(slice, s -> new ConcurrentHashMap<>());
+        });
+      });
+      // add empty collections
+      collProperties.keySet().forEach(c -> {
+        collMap.computeIfAbsent(c, co -> new ConcurrentHashMap<>());
+      });
+
+      Map<String, DocCollection> res = new HashMap<>();
+      collMap.forEach((coll, shards) -> {
+        Map<String, Slice> slices = new HashMap<>();
+        shards.forEach((s, replicas) -> {
+          Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>()).computeIfAbsent(s, sl -> new ConcurrentHashMap<>());
+          Slice slice = new Slice(s, replicas, sliceProps);
+          slices.put(s, slice);
+        });
+        Map<String, Object> collProps = collProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>());
+        DocCollection dc = new DocCollection(coll, slices, collProps, DocRouter.DEFAULT, clusterStateVersion, ZkStateReader.CLUSTER_STATE);
+        res.put(coll, dc);
+      });
+      collectionsStatesRef.set(res);
+      return res;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public Map<String, Object> getClusterProperties() {
+    return clusterProperties;
+  }
+
+  @Override
+  public String getPolicyNameByCollection(String coll) {
+    Map<String, Object> props = collProperties.computeIfAbsent(coll, c -> new HashMap<>());
+    return (String)props.get("policy");
+  }
+
+  @Override
+  public void connect() {
+
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
new file mode 100644
index 0000000..f9f17a0
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
@@ -0,0 +1,580 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.jute.Record;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.AutoScalingParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.IdUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.CheckVersionRequest;
+import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.DeleteRequest;
+import org.apache.zookeeper.proto.SetDataRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simulated {@link DistribStateManager} that keeps all data locally in a static structure. Instances of this
+ * class are identified by their id in order to simulate the deletion of ephemeral nodes when {@link #close()} is
+ * invoked.
+ */
+public class SimDistribStateManager implements DistribStateManager {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final class Node {
+    ReentrantLock dataLock = new ReentrantLock();
+    private int version = -1;
+    private int seq = 0;
+    private final CreateMode mode;
+    private final String clientId;
+    private final String path;
+    private final String name;
+    private final Node parent;
+    private byte[] data = null;
+    private Map<String, Node> children = new ConcurrentHashMap<>();
+    Set<Watcher> dataWatches = ConcurrentHashMap.newKeySet();
+    Set<Watcher> childrenWatches = ConcurrentHashMap.newKeySet();
+
+    Node(Node parent, String name, String path, CreateMode mode, String clientId) {
+      this.parent = parent;
+      this.name = name;
+      this.path = path;
+      this.mode = mode;
+      this.clientId = clientId;
+
+    }
+
+    public void clear() {
+      dataLock.lock();
+      try {
+        children.clear();
+        version = 0;
+        seq = 0;
+        dataWatches.clear();
+        childrenWatches.clear();
+        data = null;
+      } finally {
+        dataLock.unlock();
+      }
+    }
+
+    public void setData(byte[] data, int version) throws BadVersionException, IOException {
+      Set<Watcher> currentWatchers = new HashSet<>(dataWatches);
+      dataLock.lock();
+      try {
+        if (version != -1 && version != this.version) {
+          throw new BadVersionException(version, path);
+        }
+        if (data != null) {
+          this.data = Arrays.copyOf(data, data.length);
+        } else {
+          this.data = null;
+        }
+        this.version++;
+        dataWatches.clear();
+      } finally {
+        dataLock.unlock();
+      }
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeDataChanged, Watcher.Event.KeeperState.SyncConnected, path));
+      }
+    }
+
+    public VersionedData getData(Watcher w) {
+      dataLock.lock();
+      try {
+        VersionedData res = new VersionedData(version, data, clientId);
+        if (w != null && !dataWatches.contains(w)) {
+          dataWatches.add(w);
+        }
+        return res;
+      } finally {
+        dataLock.unlock();
+      }
+    }
+
+    public void setChild(String name, Node child) {
+      assert child.name.equals(name);
+      Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
+      dataLock.lock();
+      try {
+        children.put(name, child);
+        childrenWatches.clear();
+      } finally {
+        dataLock.unlock();
+      }
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
+      }
+    }
+
+    public void removeChild(String name, int version) throws NoSuchElementException, BadVersionException, IOException {
+      Node n = children.get(name);
+      if (n == null) {
+        throw new NoSuchElementException(path + "/" + name);
+      }
+      if (version != -1 && version != n.version) {
+        throw new BadVersionException(version, path);
+      }
+      children.remove(name);
+      Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
+      childrenWatches.clear();
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
+      }
+      currentWatchers = new HashSet<>(n.dataWatches);
+      n.dataWatches.clear();
+      for (Watcher w : currentWatchers) {
+        w.process(new WatchedEvent(Watcher.Event.EventType.NodeDeleted, Watcher.Event.KeeperState.SyncConnected, n.path));
+      }
+      // TODO: not sure if it's correct to recurse and fire watches???
+      Set<String> kids = new HashSet<>(n.children.keySet());
+      for (String kid : kids) {
+        n.removeChild(kid, -1);
+      }
+    }
+
+    public void removeEphemeralChildren(String id) throws NoSuchElementException, BadVersionException, IOException {
+      Set<String> kids = new HashSet<>(children.keySet());
+      for (String kid : kids) {
+        Node n = children.get(kid);
+        if (n == null) {
+          continue;
+        }
+        if ((CreateMode.EPHEMERAL == n.mode || CreateMode.EPHEMERAL_SEQUENTIAL == n.mode) &&
+            id.equals(n.clientId)) {
+          removeChild(n.name, -1);
+        } else {
+          n.removeEphemeralChildren(id);
+        }
+      }
+    }
+
+  }
+
+  private final ReentrantLock multiLock = new ReentrantLock();
+
+  public static Node createNewRootNode() {
+    return new Node(null, "", "/", CreateMode.PERSISTENT, "__root__");
+  }
+
+  private final ExecutorService watchersPool;
+
+  private final AtomicReference<ActionThrottle> throttleRef = new AtomicReference<>();
+  private final AtomicReference<ActionError> errorRef = new AtomicReference<>();
+  private final String id;
+  private final Node root;
+
+  public SimDistribStateManager() {
+    this(null);
+  }
+
+  /**
+   * Construct new state manager that uses provided root node for storing data.
+   * @param root if null then a new root node will be created.
+   */
+  public SimDistribStateManager(Node root) {
+    this.id = IdUtils.timeRandomId();
+    this.root = root != null ? root : createNewRootNode();
+    watchersPool = ExecutorUtil.newMDCAwareFixedThreadPool(10, new DefaultSolrThreadFactory("sim-watchers"));
+  }
+
+  public SimDistribStateManager(ActionThrottle actionThrottle, ActionError actionError) {
+    this(null, actionThrottle, actionError);
+  }
+
+  public SimDistribStateManager(Node root, ActionThrottle actionThrottle, ActionError actionError) {
+    this(root);
+    this.throttleRef.set(actionThrottle);
+    this.errorRef.set(actionError);
+  }
+
+  private SimDistribStateManager(String id, ExecutorService watchersPool, Node root, ActionThrottle actionThrottle,
+                                 ActionError actionError) {
+    this.id = id;
+    this.watchersPool = watchersPool;
+    this.root = root;
+    this.throttleRef.set(actionThrottle);
+    this.errorRef.set(actionError);
+  }
+
+  /**
+   * Create a copy of this instance using a specified ephemeral owner id. This is useful when performing
+   * node operations that require using a specific id. Note: this instance should never be closed, it can
+   * be just discarded after use.
+   * @param id ephemeral owner id
+   */
+  public SimDistribStateManager withEphemeralId(String id) {
+    return new SimDistribStateManager(id, watchersPool, root, throttleRef.get(), errorRef.get()) {
+      @Override
+      public void close() {
+        throw new UnsupportedOperationException("this instance should never be closed - instead close the parent instance.");
+      }
+    };
+  }
+
+  /**
+   * Get the root node of the tree used by this instance. It could be a static shared root node.
+   */
+  public Node getRoot() {
+    return root;
+  }
+
+  /**
+   * Clear this instance. All nodes, watchers and data is deleted.
+   */
+  public void clear() {
+    root.clear();
+  }
+
+  private void throttleOrError(String path) throws IOException {
+    ActionError err = errorRef.get();
+    if (err != null && err.shouldFail(path)) {
+      throw new IOException("Simulated error, path=" + path);
+    }
+    ActionThrottle throttle = throttleRef.get();
+    if (throttle != null) {
+      throttle.minimumWaitBetweenActions();
+      throttle.markAttemptingAction();
+    }
+  }
+
+  // this method should always be invoked under lock
+  private Node traverse(String path, boolean create, CreateMode mode) throws IOException {
+    if (path == null || path.isEmpty()) {
+      return null;
+    }
+    throttleOrError(path);
+    if (path.charAt(0) == '/') {
+      path = path.substring(1);
+    }
+    StringBuilder currentPath = new StringBuilder();
+    String[] elements = path.split("/");
+    Node parentNode = root;
+    Node n = null;
+    for (int i = 0; i < elements.length; i++) {
+      String currentName = elements[i];
+      currentPath.append('/');
+      LOG.info(" - parentNode=" + parentNode + ", currentName=" + currentName);
+      n = parentNode.children != null ? parentNode.children.get(currentName) : null;
+      if (n == null) {
+        if (create) {
+          if ((parentNode.mode == CreateMode.EPHEMERAL || parentNode.mode == CreateMode.EPHEMERAL_SEQUENTIAL) &&
+              (mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL)) {
+            throw new IOException("NoChildrenEphemerals for " + parentNode.path);
+          }
+          if (CreateMode.PERSISTENT_SEQUENTIAL == mode || CreateMode.EPHEMERAL_SEQUENTIAL == mode) {
+            currentName = currentName + String.format(Locale.ROOT, "%010d", parentNode.seq);
+            parentNode.seq++;
+          }
+          currentPath.append(currentName);
+          n = new Node(parentNode, currentName, currentPath.toString(), mode, id);
+          parentNode.setChild(currentName, n);
+        } else {
+          break;
+        }
+      } else {
+        currentPath.append(currentName);
+      }
+      parentNode = n;
+    }
+    return n;
+  }
+
+  @Override
+  public void close() throws IOException {
+    multiLock.lock();
+    try {
+      // remove all my ephemeral nodes
+      root.removeEphemeralChildren(id);
+    } catch (BadVersionException e) {
+      // not happening
+    } finally {
+      multiLock.unlock();
+    }
+
+  }
+
+  @Override
+  public boolean hasData(String path) throws IOException {
+    multiLock.lock();
+    try {
+      return traverse(path, false, CreateMode.PERSISTENT) != null;
+    } finally {
+      multiLock.unlock();
+    }
+  }
+
+  @Override
+  public List<String> listData(String path) throws NoSuchElementException, IOException {
+    multiLock.lock();
+    try {
+      Node n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+      List<String> res = new ArrayList<>(n.children.keySet());
+      Collections.sort(res);
+      return res;
+    } finally {
+      multiLock.unlock();
+    }
+  }
+
+  @Override
+  public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException {
+    Node n;
+    List<String> res;
+    multiLock.lock();
+    try {
+      n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+      res = new ArrayList<>(n.children.keySet());
+      Collections.sort(res);
+    } finally {
+      multiLock.unlock();
+    }
+    if (watcher != null) {
+      n.dataWatches.add(watcher);
+      n.childrenWatches.add(watcher);
+    }
+    return res;
+  }
+
+  @Override
+  public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException {
+    Node n = null;
+    multiLock.lock();
+    try {
+      n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+    } finally {
+      multiLock.unlock();
+    }
+    return n.getData(watcher);
+  }
+
+  @Override
+  public void makePath(String path) throws IOException {
+    multiLock.lock();
+    try {
+      traverse(path, true, CreateMode.PERSISTENT);
+    } finally {
+      multiLock.unlock();
+    }
+  }
+
+  @Override
+  public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    Node n = null;
+    multiLock.lock();
+    try {
+      if (failOnExists && hasData(path)) {
+        throw new AlreadyExistsException(path);
+      }
+      n = traverse(path, true, createMode);
+    } finally {
+      multiLock.unlock();
+    }
+    try {
+      n.setData(data, -1);
+    } catch (BadVersionException e) {
+      throw new IOException("should not happen!", e);
+    }
+  }
+
+  @Override
+  public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, NoSuchElementException, IOException {
+    if ((CreateMode.EPHEMERAL == mode || CreateMode.PERSISTENT == mode) && hasData(path)) {
+      throw new AlreadyExistsException(path);
+    }
+    // check if parent exists
+    String relPath = path.charAt(0) == '/' ? path.substring(1) : path;
+    if (relPath.length() > 0) { // non-root path - check if parent exists
+      String[] elements = relPath.split("/");
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < elements.length - 1; i++) {
+        sb.append('/');
+        sb.append(elements[i]);
+      }
+      if (!hasData(sb.toString())) {
+        throw new NoSuchElementException(sb.toString());
+      }
+    }
+    Node n = null;
+    multiLock.lock();
+    try {
+      n = traverse(path, true, mode);
+    } finally {
+      multiLock.unlock();
+    }
+    try {
+      n.setData(data, -1);
+      return n.path;
+    } catch (BadVersionException e) {
+      // not happening
+      return null;
+    }
+  }
+
+  @Override
+  public void removeData(String path, int version) throws NoSuchElementException, BadVersionException, IOException {
+    multiLock.lock();
+    try {
+      Node n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+      Node parent = n.parent;
+      if (parent == null) {
+        throw new IOException("Cannot remove root node");
+      }
+      parent.removeChild(n.name, version);
+    } finally {
+      multiLock.unlock();
+    }
+
+  }
+
+  @Override
+  public void setData(String path, byte[] data, int version) throws NoSuchElementException, BadVersionException, IOException {
+    multiLock.lock();
+    Node n = null;
+    try {
+      n = traverse(path, false, CreateMode.PERSISTENT);
+      if (n == null) {
+        throw new NoSuchElementException(path);
+      }
+    } finally {
+      multiLock.unlock();
+    }
+    n.setData(data, version);
+  }
+
+  @Override
+  public List<OpResult> multi(Iterable<Op> ops) throws BadVersionException, NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    multiLock.lock();
+    List<OpResult> res = new ArrayList<>();
+    try {
+      for (Op op : ops) {
+        Record r = op.toRequestRecord();
+        try {
+          if (op instanceof Op.Check) {
+            CheckVersionRequest rr = (CheckVersionRequest)r;
+            Node n = traverse(rr.getPath(), false, CreateMode.PERSISTENT);
+            if (n == null) {
+              throw new NoSuchElementException(rr.getPath());
+            }
+            if (rr.getVersion() != -1 && n.version != rr.getVersion()) {
+              throw new Exception("version mismatch");
+            }
+            // everything ok
+            res.add(new OpResult.CheckResult());
+          } else if (op instanceof Op.Create) {
+            CreateRequest rr = (CreateRequest)r;
+            createData(rr.getPath(), rr.getData(), CreateMode.fromFlag(rr.getFlags()));
+            res.add(new OpResult.CreateResult(rr.getPath()));
+          } else if (op instanceof Op.Delete) {
+            DeleteRequest rr = (DeleteRequest)r;
+            removeData(rr.getPath(), rr.getVersion());
+            res.add(new OpResult.DeleteResult());
+          } else if (op instanceof Op.SetData) {
+            SetDataRequest rr = (SetDataRequest)r;
+            setData(rr.getPath(), rr.getData(), rr.getVersion());
+            VersionedData vd = getData(rr.getPath());
+            Stat s = new Stat();
+            s.setVersion(vd.getVersion());
+            res.add(new OpResult.SetDataResult(s));
+          } else {
+            throw new Exception("Unknown Op: " + op);
+          }
+        } catch (Exception e) {
+          res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue()));
+        }
+      }
+    } finally {
+      multiLock.unlock();
+    }
+    return res;
+  }
+
+  @Override
+  public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws InterruptedException, IOException {
+    Map<String, Object> map = new HashMap<>();
+    int version = -1;
+    try {
+      VersionedData data = getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, watcher);
+      if (data != null && data.getData() != null && data.getData().length > 0) {
+        map = (Map<String, Object>) Utils.fromJSON(data.getData());
+        version = data.getVersion();
+      }
+    } catch (NoSuchElementException e) {
+      // ignore
+    }
+    map.put(AutoScalingParams.ZK_VERSION, version);
+    return new AutoScalingConfig(map);
+  }
+
+  // ------------ simulator methods --------------
+
+  public void simSetAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
+    try {
+      makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    } catch (Exception e) {
+      // ignore
+    }
+    setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(cfg), -1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
new file mode 100644
index 0000000..e9616f0
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistributedQueueFactory.java
@@ -0,0 +1,284 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Predicate;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Preconditions;
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.cloud.Stats;
+import org.apache.solr.common.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simulated {@link DistributedQueueFactory} that keeps all data in memory. Unlike
+ * the {@link GenericDistributedQueueFactory} this queue implementation data is not
+ * exposed anywhere.
+ */
+public class SimDistributedQueueFactory implements DistributedQueueFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  Map<String, SimDistributedQueue> queues = new ConcurrentHashMap<>();
+
+  public SimDistributedQueueFactory() {
+  }
+
+  @Override
+  public DistributedQueue makeQueue(final String path) throws IOException {
+    return queues.computeIfAbsent(path, p -> new SimDistributedQueue(path));
+  }
+
+  @Override
+  public void removeQueue(String path) throws IOException {
+    queues.remove(path);
+  }
+
+  public static class SimDistributedQueue implements DistributedQueue {
+    private final Queue<Pair<String, byte[]>> queue = new ConcurrentLinkedQueue<>();
+    private final ReentrantLock updateLock = new ReentrantLock();
+    private final Condition changed = updateLock.newCondition();
+    private final Stats stats = new Stats();
+    private final String dir;
+    private int seq = 0;
+
+    public SimDistributedQueue(String dir) {
+      this.dir = dir;
+    }
+
+    @Override
+    public byte[] peek() throws Exception {
+      Timer.Context time = stats.time(dir + "_peek");
+      try {
+        Pair<String, byte[]> pair = queue.peek();
+        return pair != null ? pair.second() : null;
+      } finally {
+        time.stop();
+      }
+    }
+
+    @Override
+    public byte[] peek(boolean block) throws Exception {
+      return block ? peek(Long.MAX_VALUE) : peek();
+    }
+
+    @Override
+    public byte[] peek(long wait) throws Exception {
+      Timer.Context time;
+      if (wait == Long.MAX_VALUE) {
+        time = stats.time(dir + "_peek_wait_forever");
+      } else {
+        time = stats.time(dir + "_peek_wait" + wait);
+      }
+      try {
+        Pair<String, byte[]> pair = peekInternal(wait);
+        return pair != null ? pair.second() : null;
+      } finally {
+        time.stop();
+      }
+    }
+
+    private Pair<String, byte[]> peekInternal(long wait) throws Exception {
+      Preconditions.checkArgument(wait > 0);
+      long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
+      updateLock.lockInterruptibly();
+      try {
+        while (waitNanos > 0) {
+          Pair<String, byte[]> pair = queue.peek();
+          if (pair != null) {
+            return pair;
+          }
+          waitNanos = changed.awaitNanos(waitNanos);
+          if (waitNanos < 0) { // timed out
+            return null;
+          }
+        }
+      } finally {
+        updateLock.unlock();
+      }
+      return null;
+    }
+
+    @Override
+    public byte[] poll() throws Exception {
+      Timer.Context time = stats.time(dir + "_poll");
+      updateLock.lockInterruptibly();
+      try {
+        Pair<String, byte[]>  pair = queue.poll();
+        if (pair != null) {
+          changed.signalAll();
+          return pair.second();
+        } else {
+          return null;
+        }
+      } finally {
+        updateLock.unlock();
+        time.stop();
+      }
+    }
+
+    @Override
+    public byte[] remove() throws Exception {
+      Timer.Context time = stats.time(dir + "_remove");
+      updateLock.lockInterruptibly();
+      try {
+        byte[] res = queue.remove().second();
+        changed.signalAll();
+        return res;
+      } finally {
+        updateLock.unlock();
+        time.stop();
+      }
+    }
+
+    @Override
+    public byte[] take() throws Exception {
+      Timer.Context timer = stats.time(dir + "_take");
+      updateLock.lockInterruptibly();
+      try {
+        while (true) {
+          byte[] result = poll();
+          if (result != null) {
+            return result;
+          }
+          changed.await();
+        }
+      } finally {
+        updateLock.unlock();
+        timer.stop();
+      }
+    }
+
+    @Override
+    public void offer(byte[] data) throws Exception {
+      Timer.Context time = stats.time(dir + "_offer");
+      updateLock.lockInterruptibly();
+      try {
+        queue.offer(new Pair(String.format(Locale.ROOT, "qn-%010d", seq), data));
+        seq++;
+        LOG.trace("=== offer " + System.nanoTime());
+        changed.signalAll();
+      } finally {
+        updateLock.unlock();
+        time.stop();
+      }
+    }
+
+    @Override
+    public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
+      updateLock.lockInterruptibly();
+      try {
+        List<Pair<String, byte[]>> res = new LinkedList<>();
+        final int maximum = max < 0 ? Integer.MAX_VALUE : max;
+        final AtomicReference<Pair<String, byte[]>> pairRef = new AtomicReference<>();
+        queue.forEach(pair -> {
+          if (acceptFilter != null && !acceptFilter.test(pair.first())) {
+            return;
+          }
+          if (res.size() < maximum) {
+            pairRef.set(pair);
+            res.add(pair);
+          }
+        });
+        if (res.size() < maximum && waitMillis > 0) {
+          long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
+          waitNanos = changed.awaitNanos(waitNanos);
+          if (waitNanos < 0) {
+            return res;
+          }
+          AtomicBoolean seen = new AtomicBoolean(false);
+          queue.forEach(pair -> {
+            if (!seen.get()) {
+              if (pairRef.get() == null) {
+                seen.set(true);
+              } else {
+                if (pairRef.get().first().equals(pair.first())) {
+                  seen.set(true);
+                  return;
+                }
+              }
+            }
+            if (!seen.get()) {
+              return;
+            }
+            if (!acceptFilter.test(pair.first())) {
+              return;
+            }
+            if (res.size() < maximum) {
+              res.add(pair);
+              pairRef.set(pair);
+            } else {
+              return;
+            }
+          });
+        }
+        return res;
+      } finally {
+        updateLock.unlock();
+      }
+    }
+
+    public Stats getZkStats() {
+      return stats;
+    }
+
+    @Override
+    public Map<String, Object> getStats() {
+      if (stats == null) {
+        return Collections.emptyMap();
+      }
+      Map<String, Object> res = new HashMap<>();
+      res.put("queueLength", stats.getQueueLength());
+      final Map<String, Object> statsMap = new HashMap<>();
+      res.put("stats", statsMap);
+      stats.getStats().forEach((op, stat) -> {
+        final Map<String, Object> statMap = new HashMap<>();
+        statMap.put("success", stat.success.get());
+        statMap.put("errors", stat.errors.get());
+        final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
+        statMap.put("failureDetails", failed);
+        stat.failureDetails.forEach(failedOp -> {
+          Map<String, Object> fo = new HashMap<>();
+          fo.put("req", failedOp.req);
+          fo.put("resp", failedOp.resp);
+        });
+        statsMap.put(op, statMap);
+      });
+      return res;
+    }
+  }
+}


[39/54] [abbrv] lucene-solr:jira/solr-11702: Update doap files with 7.2.0 release.

Posted by da...@apache.org.
Update doap files with 7.2.0 release.


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

Branch: refs/heads/jira/solr-11702
Commit: 63cba4093157e32fcc399948fcc5b1a201f35d8e
Parents: 69e9f17
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Dec 21 17:58:57 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Dec 21 17:58:57 2017 +0100

----------------------------------------------------------------------
 dev-tools/doap/lucene.rdf | 8 ++++++++
 dev-tools/doap/solr.rdf   | 7 +++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/63cba409/dev-tools/doap/lucene.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/lucene.rdf b/dev-tools/doap/lucene.rdf
index 8e7d7a1..101e094 100644
--- a/dev-tools/doap/lucene.rdf
+++ b/dev-tools/doap/lucene.rdf
@@ -69,6 +69,14 @@
     <!-- NOTE: please insert releases in numeric order, NOT chronologically. -->
     <release>
       <Version>
+        <name>lucene-7.2.0</name>
+        <created>2017-12-21</created>
+        <revision>7.2.0</revision>
+      </Version>
+    </release>
+    <release>
+    <release>
+      <Version>
         <name>lucene-7.1.0</name>
         <created>2017-10-17</created>
         <revision>7.1.0</revision>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/63cba409/dev-tools/doap/solr.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/solr.rdf b/dev-tools/doap/solr.rdf
index 5af1491..fd2f96a 100644
--- a/dev-tools/doap/solr.rdf
+++ b/dev-tools/doap/solr.rdf
@@ -69,6 +69,13 @@
     <!-- NOTE: please insert releases in numeric order, NOT chronologically. -->
     <release>
       <Version>
+        <name>solr-7.2.0</name>
+        <created>2017-12-21</created>
+        <revision>7.2.0</revision>
+      </Version>
+    </release>
+    <release>
+      <Version>
         <name>solr-7.1.0</name>
         <created>2017-10-17</created>
         <revision>7.1.0</revision>


[40/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11754: remove AbstractSolrTestCase; use SolrTestCaseJ4 instead. Also removed SolrExampleTestBase in Solrj-tests.

Posted by da...@apache.org.
SOLR-11754: remove AbstractSolrTestCase; use SolrTestCaseJ4 instead.
Also removed SolrExampleTestBase in Solrj-tests.


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

Branch: refs/heads/jira/solr-11702
Commit: f354a278f810fb702048e1f656c3e79a1a8888c2
Parents: 63cba40
Author: David Smiley <ds...@apache.org>
Authored: Thu Dec 21 13:20:22 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Dec 21 13:20:22 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/BasicFunctionalityTest.java |   2 +-
 .../org/apache/solr/cloud/ZkSolrClientTest.java |   4 +-
 .../org/apache/solr/core/PluginInfoTest.java    |  10 +-
 .../apache/solr/core/TestJmxIntegration.java    |   4 +-
 .../apache/solr/core/TestXIncludeConfig.java    |   4 +-
 .../apache/solr/handler/SearchHandlerTest.java  |   4 +-
 .../handler/admin/LukeRequestHandlerTest.java   |   4 +-
 .../admin/SegmentsInfoRequestHandlerTest.java   |   4 +-
 .../handler/component/StatsComponentTest.java   |   4 +-
 .../solr/highlight/HighlighterConfigTest.java   |   4 +-
 .../org/apache/solr/request/TestWriterPerf.java |   4 +-
 .../solr/response/TestBinaryResponseWriter.java |   4 +-
 .../search/TestComplexPhraseQParserPlugin.java  |   4 +-
 .../solr/search/TestMaxScoreQueryParser.java    |   4 +-
 .../org/apache/solr/search/TestQueryTypes.java  |   4 +-
 .../org/apache/solr/search/TestQueryUtils.java  |   4 +-
 .../org/apache/solr/search/TestSearchPerf.java  |   4 +-
 .../solr/search/TestSurroundQueryParser.java    |   4 +-
 .../search/function/SortByFunctionTest.java     |   4 +-
 .../solr/search/function/TestFunctionQuery.java |   2 +-
 .../solr/security/BasicAuthStandaloneTest.java  |   6 +-
 .../solr/servlet/DirectSolrConnectionTest.java  |   4 +-
 .../update/DirectUpdateHandlerOptimizeTest.java |   4 +-
 .../apache/solr/update/SoftAutoCommitTest.java  |  14 +-
 .../solr/update/TestIndexingPerformance.java    |   4 +-
 .../apache/solr/update/UpdateParamsTest.java    |   4 +-
 .../UpdateRequestProcessorFactoryTest.java      |   4 +-
 .../solrj/MergeIndexesExampleTestBase.java      |  20 +--
 .../solr/client/solrj/SolrExampleTestBase.java  |  64 ---------
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   6 +-
 .../apache/solr/util/AbstractSolrTestCase.java  | 135 -------------------
 32 files changed, 72 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index fabe30a..80d0d11 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -74,6 +74,8 @@ Other Changes
 * SOLR-11757: In tests, fix race condition on SolrException.ignoreException.
   Also ensure we register "ignore_exception" in @BeforeClass (previously only @AfterClass)  (David Smiley)
 
+* SOLR-11754: Remove AbstractSolrTestCase which has long been supplanted by SolrTestCaseJ4. (David Smiley)
+
 ==================  7.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
index 84280ee..2321188 100644
--- a/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
+++ b/solr/core/src/test/org/apache/solr/BasicFunctionalityTest.java
@@ -57,7 +57,7 @@ import org.junit.Test;
 
 /**
  * Tests some basic functionality of Solr while demonstrating good
- * Best Practices for using AbstractSolrTestCase
+ * Best Practices for using SolrTestCaseJ4
  */
 public class BasicFunctionalityTest extends SolrTestCaseJ4 {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
index c5ce2c9..fc40395 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
@@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import junit.framework.Assert;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -31,7 +31,7 @@ import org.apache.zookeeper.Watcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-public class ZkSolrClientTest extends AbstractSolrTestCase {
+public class ZkSolrClientTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/core/PluginInfoTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/PluginInfoTest.java b/solr/core/src/test/org/apache/solr/core/PluginInfoTest.java
index 2353226..c0cfe56 100644
--- a/solr/core/src/test/org/apache/solr/core/PluginInfoTest.java
+++ b/solr/core/src/test/org/apache/solr/core/PluginInfoTest.java
@@ -18,7 +18,7 @@ package org.apache.solr.core;
 
 import java.util.List;
 
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.util.DOMUtilTestBase;
 import org.junit.Test;
 import org.w3c.dom.Node;
@@ -63,14 +63,14 @@ public class PluginInfoTest extends DOMUtilTestBase {
   @Test
   public void testNameRequired() throws Exception {
     Node nodeWithNoName = getNode("<plugin></plugin>", "plugin");
-    AbstractSolrTestCase.ignoreException("missing mandatory attribute");
+    SolrTestCaseJ4.ignoreException("missing mandatory attribute");
     try {
       PluginInfo pi = new PluginInfo(nodeWithNoName, "Node with No name", true, false);
       fail("Exception should have been thrown");
     } catch (RuntimeException e) {
       assertTrue(e.getMessage().contains("missing mandatory attribute"));
     } finally {
-      AbstractSolrTestCase.resetExceptionIgnores();
+      SolrTestCaseJ4.resetExceptionIgnores();
     }
     Node nodeWithAName = getNode("<plugin name=\"myName\" />", "plugin");
     PluginInfo pi2 = new PluginInfo(nodeWithAName, "Node with a Name", true, false);
@@ -80,7 +80,7 @@ public class PluginInfoTest extends DOMUtilTestBase {
   @Test
   public void testClassRequired() throws Exception {
     Node nodeWithNoClass = getNode("<plugin></plugin>", "plugin");
-    AbstractSolrTestCase.ignoreException("missing mandatory attribute");
+    SolrTestCaseJ4.ignoreException("missing mandatory attribute");
     try {
       @SuppressWarnings("unused")
       PluginInfo pi = new PluginInfo(nodeWithNoClass, "Node with No Class", false, true);
@@ -88,7 +88,7 @@ public class PluginInfoTest extends DOMUtilTestBase {
     } catch (RuntimeException e) {
       assertTrue(e.getMessage().contains("missing mandatory attribute"));
     } finally {
-      AbstractSolrTestCase.resetExceptionIgnores();
+      SolrTestCaseJ4.resetExceptionIgnores();
     }
     Node nodeWithAClass = getNode("<plugin class=\"myName\" />", "plugin");
     PluginInfo pi2 = new PluginInfo(nodeWithAClass, "Node with a Class", false, true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java b/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
index 31e5718..044b300 100644
--- a/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
+++ b/solr/core/src/test/org/apache/solr/core/TestJmxIntegration.java
@@ -20,7 +20,7 @@ import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
 import org.apache.solr.metrics.reporters.jmx.JmxObjectNameFactory;
 import org.apache.solr.metrics.reporters.SolrJmxReporter;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,7 +44,7 @@ import java.util.Set;
  *
  * @since solr 1.3
  */
-public class TestJmxIntegration extends AbstractSolrTestCase {
+public class TestJmxIntegration extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/core/TestXIncludeConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestXIncludeConfig.java b/solr/core/src/test/org/apache/solr/core/TestXIncludeConfig.java
index 32be46e..0a80795 100644
--- a/solr/core/src/test/org/apache/solr/core/TestXIncludeConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestXIncludeConfig.java
@@ -21,14 +21,14 @@ import javax.xml.parsers.DocumentBuilderFactory;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.update.processor.RegexReplaceProcessorFactory;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.Assume;
 import org.junit.BeforeClass;
 
 /** 
  * Test both XInclude as well as more old school "entity includes"
  */
-public class TestXIncludeConfig extends AbstractSolrTestCase {
+public class TestXIncludeConfig extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/handler/SearchHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/SearchHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/SearchHandlerTest.java
index 4fa5f10..ebf4294 100644
--- a/solr/core/src/test/org/apache/solr/handler/SearchHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/SearchHandlerTest.java
@@ -16,13 +16,13 @@
  */
 package org.apache.solr.handler;
 
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 /**
  * Most of the tests for {@link org.apache.solr.handler.component.SearchHandler} are in {@link org.apache.solr.ConvertedLegacyTest}.
  */
-public class SearchHandlerTest extends AbstractSolrTestCase {
+public class SearchHandlerTest extends SolrTestCaseJ4 {
   
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
index d253ef2..df043c1 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
@@ -23,7 +23,7 @@ import org.apache.solr.common.luke.FieldFlag;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.CustomAnalyzerStrField; // jdoc
 import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.util.TestHarness;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -32,7 +32,7 @@ import org.junit.Test;
 /**
  * :TODO: currently only tests some of the utilities in the LukeRequestHandler
  */
-public class LukeRequestHandlerTest extends AbstractSolrTestCase {
+public class LukeRequestHandlerTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
index 6de7835..3173c12 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
@@ -18,7 +18,7 @@ package org.apache.solr.handler.admin;
 
 import org.apache.lucene.util.Version;
 import org.apache.solr.index.LogDocMergePolicyFactory;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -26,7 +26,7 @@ import org.junit.Test;
 /**
  * Tests for SegmentsInfoRequestHandler. Plugin entry, returning data of created segment.
  */
-public class SegmentsInfoRequestHandlerTest extends AbstractSolrTestCase {
+public class SegmentsInfoRequestHandlerTest extends SolrTestCaseJ4 {
   private static final int DOC_COUNT = 5;
   
   private static final int DEL_COUNT = 1;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
index cee1ab5..d5c7472 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
@@ -50,7 +50,7 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 
 import org.apache.commons.math3.util.Combinations;
 import com.tdunning.math.stats.AVLTreeDigest;
@@ -62,7 +62,7 @@ import org.junit.BeforeClass;
 /**
  * Statistics Component Test
  */
-public class StatsComponentTest extends AbstractSolrTestCase {
+public class StatsComponentTest extends SolrTestCaseJ4 {
 
   final static String XPRE = "/response/lst[@name='stats']/";
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/highlight/HighlighterConfigTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/HighlighterConfigTest.java b/solr/core/src/test/org/apache/solr/highlight/HighlighterConfigTest.java
index 5832132..eb1e2b9 100644
--- a/solr/core/src/test/org/apache/solr/highlight/HighlighterConfigTest.java
+++ b/solr/core/src/test/org/apache/solr/highlight/HighlighterConfigTest.java
@@ -20,13 +20,13 @@ import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
 
 import org.apache.solr.handler.component.HighlightComponent;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.util.TestHarness;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HighlighterConfigTest extends AbstractSolrTestCase {
+public class HighlighterConfigTest extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java b/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
index 1e4663f..201f9cb 100644
--- a/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
+++ b/solr/core/src/test/org/apache/solr/request/TestWriterPerf.java
@@ -30,14 +30,14 @@ import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.response.BinaryQueryResponseWriter;
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.util.RTimer;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-public class TestWriterPerf extends AbstractSolrTestCase {
+public class TestWriterPerf extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/response/TestBinaryResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/TestBinaryResponseWriter.java b/solr/core/src/test/org/apache/solr/response/TestBinaryResponseWriter.java
index ae85d52..5db49ca 100644
--- a/solr/core/src/test/org/apache/solr/response/TestBinaryResponseWriter.java
+++ b/solr/core/src/test/org/apache/solr/response/TestBinaryResponseWriter.java
@@ -30,7 +30,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.BinaryResponseWriter.Resolver;
 import org.apache.solr.search.SolrReturnFields;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 /**
@@ -39,7 +39,7 @@ import org.junit.BeforeClass;
  *
  * @since solr 1.4
  */
-public class TestBinaryResponseWriter extends AbstractSolrTestCase {
+public class TestBinaryResponseWriter extends SolrTestCaseJ4 {
 
   
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/TestComplexPhraseQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestComplexPhraseQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestComplexPhraseQParserPlugin.java
index 4964d5b..1b5f5ee 100644
--- a/solr/core/src/test/org/apache/solr/search/TestComplexPhraseQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestComplexPhraseQParserPlugin.java
@@ -19,14 +19,14 @@ package org.apache.solr.search;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.HighlightParams;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.util.TestHarness;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.util.HashMap;
 
-public class TestComplexPhraseQParserPlugin extends AbstractSolrTestCase {
+public class TestComplexPhraseQParserPlugin extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/TestMaxScoreQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestMaxScoreQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestMaxScoreQueryParser.java
index 6c2694d..1800ddb 100644
--- a/solr/core/src/test/org/apache/solr/search/TestMaxScoreQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestMaxScoreQueryParser.java
@@ -21,7 +21,7 @@ import org.apache.solr.legacy.LegacyNumericRangeQuery;
 import org.apache.lucene.search.*;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -30,7 +30,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Collections;
 
-public class TestMaxScoreQueryParser extends AbstractSolrTestCase {
+public class TestMaxScoreQueryParser extends SolrTestCaseJ4 {
   Query q;
   BooleanClause[] clauses;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/TestQueryTypes.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestQueryTypes.java b/solr/core/src/test/org/apache/solr/search/TestQueryTypes.java
index ef976a3..aa94fab 100644
--- a/solr/core/src/test/org/apache/solr/search/TestQueryTypes.java
+++ b/solr/core/src/test/org/apache/solr/search/TestQueryTypes.java
@@ -18,11 +18,11 @@ package org.apache.solr.search;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class TestQueryTypes extends AbstractSolrTestCase {
+public class TestQueryTypes extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/TestQueryUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestQueryUtils.java b/solr/core/src/test/org/apache/solr/search/TestQueryUtils.java
index adc8058..8f6e72d 100644
--- a/solr/core/src/test/org/apache/solr/search/TestQueryUtils.java
+++ b/solr/core/src/test/org/apache/solr/search/TestQueryUtils.java
@@ -21,7 +21,7 @@ import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.index.Term;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 import java.util.Collection;
@@ -29,7 +29,7 @@ import java.util.Collection;
 /**
  *
  */
-public class TestQueryUtils extends AbstractSolrTestCase {
+public class TestQueryUtils extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java b/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java
index 4c63309..08167b6 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSearchPerf.java
@@ -18,7 +18,7 @@ package org.apache.solr.search;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
@@ -34,7 +34,7 @@ import java.io.IOException;
 /**
  *
  */
-public class TestSearchPerf extends AbstractSolrTestCase {
+public class TestSearchPerf extends SolrTestCaseJ4 {
 
   
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/TestSurroundQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSurroundQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSurroundQueryParser.java
index 7b01a73..8a6d7e2 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSurroundQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSurroundQueryParser.java
@@ -16,10 +16,10 @@
  */
 package org.apache.solr.search;
 
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
-public class TestSurroundQueryParser extends AbstractSolrTestCase {
+public class TestSurroundQueryParser extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/function/SortByFunctionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/SortByFunctionTest.java b/solr/core/src/test/org/apache/solr/search/function/SortByFunctionTest.java
index ada1208..658a257 100644
--- a/solr/core/src/test/org/apache/solr/search/function/SortByFunctionTest.java
+++ b/solr/core/src/test/org/apache/solr/search/function/SortByFunctionTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 package org.apache.solr.search.function;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 
@@ -23,7 +23,7 @@ import org.junit.BeforeClass;
  *
  * @see TestSortByMinMaxFunction
  **/
-public class SortByFunctionTest extends AbstractSolrTestCase {
+public class SortByFunctionTest extends SolrTestCaseJ4 {
 
   
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
index 30b33da..53e1ea7 100644
--- a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
+++ b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
@@ -35,7 +35,7 @@ import org.junit.Test;
 
 /**
  * Tests some basic functionality of Solr while demonstrating good
- * Best Practices for using AbstractSolrTestCase
+ * Best Practices for using SolrTestCaseJ4
  */
 public class TestFunctionQuery extends SolrTestCaseJ4 {
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
index 98cdcd2..b382342 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
@@ -41,7 +41,7 @@ import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.util.LogLevel;
 import org.junit.After;
 import org.junit.Before;
@@ -54,10 +54,10 @@ import static org.apache.solr.security.BasicAuthIntegrationTest.NOT_NULL_PREDICA
 import static org.apache.solr.security.BasicAuthIntegrationTest.STD_CONF;
 import static org.apache.solr.security.BasicAuthIntegrationTest.verifySecurityStatus;
 
-public class BasicAuthStandaloneTest extends AbstractSolrTestCase {
+public class BasicAuthStandaloneTest extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private Path ROOT_DIR = Paths.get(getSolrHome());
+  private Path ROOT_DIR = Paths.get(TEST_HOME());
   private Path CONF_DIR = ROOT_DIR.resolve("configsets").resolve("configset-2").resolve("conf");
 
   SecurityConfHandlerLocalForTesting securityConfHandler;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java b/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
index 8c7217f..9b70155 100644
--- a/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
+++ b/solr/core/src/test/org/apache/solr/servlet/DirectSolrConnectionTest.java
@@ -19,12 +19,12 @@ package org.apache.solr.servlet;
 import java.net.URLEncoder;
 
 import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 
 
-public class DirectSolrConnectionTest extends AbstractSolrTestCase 
+public class DirectSolrConnectionTest extends SolrTestCaseJ4
 {
 
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java
index 01e44fd..21a39bd 100644
--- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java
+++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java
@@ -21,7 +21,7 @@ import java.io.FileFilter;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 
@@ -29,7 +29,7 @@ import org.junit.BeforeClass;
  *
  *
  **/
-public class DirectUpdateHandlerOptimizeTest extends AbstractSolrTestCase {
+public class DirectUpdateHandlerOptimizeTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java b/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
index a5814d3..93af37a 100644
--- a/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
@@ -31,7 +31,8 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrEventListener;
 import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.util.TestHarness;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
@@ -57,7 +58,7 @@ import org.slf4j.LoggerFactory;
  * </ul>
  */
 @Slow
-public class SoftAutoCommitTest extends AbstractSolrTestCase {
+public class SoftAutoCommitTest extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   @BeforeClass
@@ -529,6 +530,15 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase {
     };
     public abstract int useValue(final int softCommitWaitMillis, final int hardCommitWaitMillis);
   }
+
+  public String delI(String id, String... args) {
+    return TestHarness.deleteById(id, args);
+  }
+
+  public String adoc(int commitWithin, String... fieldsAndValues) {
+    XmlDoc d = doc(fieldsAndValues);
+    return add(d, "commitWithin", String.valueOf(commitWithin));
+  }
 }
 
 class MockEventListener implements SolrEventListener {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java b/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java
index 6aee5bd..802090b 100644
--- a/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java
+++ b/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java
@@ -18,7 +18,7 @@ package org.apache.solr.update;
 
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.util.RTimer;
 import org.junit.AfterClass;
@@ -36,7 +36,7 @@ import java.util.Arrays;
  * 
  * $ ant test -Dtestcase=TestIndexingPerformance -Dargs="-server -Diter=100000"; grep throughput build/test-results/*TestIndexingPerformance.xml
  */
-public class TestIndexingPerformance extends AbstractSolrTestCase {
+public class TestIndexingPerformance extends SolrTestCaseJ4 {
   
   // TODO: fix this test to not require FSDirectory
   static String savedFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/update/UpdateParamsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/UpdateParamsTest.java b/solr/core/src/test/org/apache/solr/update/UpdateParamsTest.java
index b90a940..114ff6a 100644
--- a/solr/core/src/test/org/apache/solr/update/UpdateParamsTest.java
+++ b/solr/core/src/test/org/apache/solr/update/UpdateParamsTest.java
@@ -24,12 +24,12 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.UpdateRequestHandler;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 
 
 
-public class UpdateParamsTest extends AbstractSolrTestCase {
+public class UpdateParamsTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/core/src/test/org/apache/solr/update/processor/UpdateRequestProcessorFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/UpdateRequestProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/UpdateRequestProcessorFactoryTest.java
index dc7aba1..65f3eca 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/UpdateRequestProcessorFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/UpdateRequestProcessorFactoryTest.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,7 +34,7 @@ import org.slf4j.LoggerFactory;
 /**
  * 
  */
-public class UpdateRequestProcessorFactoryTest extends AbstractSolrTestCase {
+public class UpdateRequestProcessorFactoryTest extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
index 2c8ebe9..76c4e5b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
@@ -40,7 +40,7 @@ import java.util.Arrays;
  * @since solr 1.4
  *
  */
-public abstract class MergeIndexesExampleTestBase extends SolrExampleTestBase {
+public abstract class MergeIndexesExampleTestBase extends SolrTestCaseJ4 {
 
   protected CoreContainer cores;
   private String saveProp;
@@ -49,16 +49,10 @@ public abstract class MergeIndexesExampleTestBase extends SolrExampleTestBase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  @Override
-  public String getSolrHome() {
+  static String getSolrHome() {
     return SolrTestCaseJ4.getFile("solrj/solr/multicore").getAbsolutePath();
   }
 
-  @BeforeClass
-  public static void beforeClass2() throws Exception {
-
-  }
-
   protected void setupCoreContainer() {
     cores = new CoreContainer(getSolrHome());
     cores.load();
@@ -93,16 +87,6 @@ public abstract class MergeIndexesExampleTestBase extends SolrExampleTestBase {
     else System.setProperty("solr.directoryFactory", saveProp);
   }
 
-  @Override
-  protected final SolrClient getSolrClient() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  protected final SolrClient createNewSolrClient() {
-    throw new UnsupportedOperationException();
-  }
-
   protected abstract SolrClient getSolrCore0();
 
   protected abstract SolrClient getSolrCore1();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
deleted file mode 100644
index 114d8ed..0000000
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
+++ /dev/null
@@ -1,64 +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.client.solrj;
-
-import org.apache.solr.util.AbstractSolrTestCase;
-import org.junit.BeforeClass;
-
-/**
- * This should include tests against the example solr config
- * 
- * This lets us try various SolrServer implementations with the same tests.
- * 
- */
-abstract public class SolrExampleTestBase extends AbstractSolrTestCase {
-  @Override
-  public String getSolrHome() {
-    return "../../../example/solr/";
-  }
-  
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    
-  }
-  
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    
-    // this sets the property for jetty starting SolrDispatchFilter
-    System.setProperty("solr.solr.home", this.getSolrHome());
-    System.setProperty("solr.data.dir", this.initCoreDataDir.getCanonicalPath());
-  }
-  
-  @Override
-  public void tearDown() throws Exception {
-    System.clearProperty("solr.solr.home");
-    System.clearProperty("solr.data.dir");
-    super.tearDown();
-  }
-  
-  /**
-   * Subclasses need to initialize the server impl
-   */
-  protected abstract SolrClient getSolrClient();
-  
-  /**
-   * Create a new solr server
-   */
-  protected abstract SolrClient createNewSolrClient();
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index c089b24..706f1eb 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -118,7 +118,6 @@ import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.servlet.DirectSolrConnection;
-import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.RandomizeSSL;
 import org.apache.solr.util.RandomizeSSL.SSLRandomizer;
@@ -149,10 +148,9 @@ import static org.apache.solr.update.processor.DistributedUpdateProcessor.Distri
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
 /**
- * A junit4 Solr test harness that extends LuceneTestCaseJ4. To change which core is used when loading the schema and solrconfig.xml, simply
+ * A junit4 Solr test harness that extends LuceneTestCaseJ4.
+ * To change which core is used when loading the schema and solrconfig.xml, simply
  * invoke the {@link #initCore(String, String, String, String)} method.
- * 
- * Unlike {@link AbstractSolrTestCase}, a new core is not created for each test method.
  */
 @ThreadLeakFilters(defaultFilters = true, filters = {
     SolrIgnoredThreadsFilter.class,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f354a278/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java b/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java
deleted file mode 100644
index dcbdb49..0000000
--- a/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java
+++ /dev/null
@@ -1,135 +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.util;
-
-
-import java.io.File;
-import java.lang.invoke.MethodHandles;
-import java.util.HashSet;
-
-import org.apache.lucene.util.QuickPatchThreadsFilter;
-import org.apache.solr.SolrIgnoredThreadsFilter;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.core.SolrConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
-
-/**
- * An Abstract base class that makes writing Solr JUnit tests "easier"
- *
- * <p>
- * Test classes that subclass this need only specify the path to the
- * schema.xml file (:TODO: the solrconfig.xml as well) and write some
- * testMethods.  This class takes care of creating/destroying the index,
- * and provides several assert methods to assist you.
- * </p>
- *
- * @see #setUp
- * @see #tearDown
- */
-@ThreadLeakFilters(defaultFilters = true, filters = {
-    SolrIgnoredThreadsFilter.class,
-    QuickPatchThreadsFilter.class
-})
-public abstract class AbstractSolrTestCase extends SolrTestCaseJ4 {
-  protected SolrConfig solrConfig;
-
-
-  /**
-   * Subclasses can override this to change a test's solr home
-   * (default is in test-files)
-   */
-  public String getSolrHome() {
-    return SolrTestCaseJ4.TEST_HOME();
-  }
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-
-    /** Causes an exception matching the regex pattern to not be logged. */
-  public static void ignoreException(String pattern) {
-    if (SolrException.ignorePatterns == null)
-      SolrException.ignorePatterns = new HashSet<>();
-    SolrException.ignorePatterns.add(pattern);
-  }
-
-  public static void resetExceptionIgnores() {
-    SolrException.ignorePatterns = null;
-    ignoreException("ignore_exception");  // always ignore "ignore_exception"
-  }
-
-  /** Subclasses that override setUp can optionally call this method
-   * to log the fact that their setUp process has ended.
-   */
-  @Override
-  public void postSetUp() {
-    log.info("####POSTSETUP " + getTestName());
-  }
-
-
-  /** Subclasses that override tearDown can optionally call this method
-   * to log the fact that the tearDown process has started.  This is necessary
-   * since subclasses will want to call super.tearDown() at the *end* of their
-   * tearDown method.
-   */
-  @Override
-  public void preTearDown() {
-    log.info("####PRETEARDOWN " + getTestName());      
-  }
-
-
-  /**
-   * Generates a simple &lt;add&gt;&lt;doc&gt;... XML String with the
-   * commitWithin attribute.
-   *
-   * @param commitWithin the value of the commitWithin attribute 
-   * @param fieldsAndValues 0th and Even numbered args are fields names odds are field values.
-   * @see #add
-   * @see #doc
-   */
-  public String adoc(int commitWithin, String... fieldsAndValues) {
-    XmlDoc d = doc(fieldsAndValues);
-    return add(d, "commitWithin", String.valueOf(commitWithin));
-  }
-
-
-  /**
-   * Generates a &lt;delete&gt;... XML string for an ID
-   *
-   * @see TestHarness#deleteById
-   */
-  public String delI(String id, String... args) {
-    return TestHarness.deleteById(id, args);
-  }
-  
-  /**
-   * Generates a &lt;delete&gt;... XML string for an query
-   *
-   * @see TestHarness#deleteByQuery
-   */
-  public String delQ(String q, String... args) {
-    return TestHarness.deleteByQuery(q, args);
-  }
-
-  /** @see SolrTestCaseJ4#getFile */
-  public static File getFile(String name) {
-    return SolrTestCaseJ4.getFile(name);
-  }
-}


[43/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11730: Add simulated tests for nodeAdded / nodeLost dynamic in a large cluster. Plus some other fixes: * Fix leader election throttle and cluster state versioning in the simulator. * PolicyHelper was stil

Posted by da...@apache.org.
SOLR-11730: Add simulated tests for nodeAdded / nodeLost dynamic in a large cluster.
Plus some other fixes:
* Fix leader election throttle and cluster state versioning in the simulator.
* PolicyHelper was still using a static ThreadLocal field, use ObjectCache isntead.


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

Branch: refs/heads/jira/solr-11702
Commit: 091f45dd7b4c6685b1e787283ecc029994641f3e
Parents: 67e1b4a
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Fri Dec 22 12:54:48 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Fri Dec 22 12:58:21 2017 +0100

----------------------------------------------------------------------
 .../org/apache/solr/cloud/ActionThrottle.java   |   2 +-
 .../cloud/autoscaling/ExecutePlanAction.java    |   6 +-
 .../solr/cloud/autoscaling/NodeLostTrigger.java |   2 +-
 .../cloud/autoscaling/sim/LiveNodesSet.java     |   4 +
 .../cloud/autoscaling/sim/SimCloudManager.java  |  25 ++-
 .../sim/SimClusterStateProvider.java            | 209 +++++++++---------
 .../autoscaling/sim/SimNodeStateProvider.java   |  31 +++
 .../autoscaling/sim/SimSolrCloudTestCase.java   |  96 ++++++++-
 .../autoscaling/sim/TestComputePlanAction.java  |  13 +-
 .../autoscaling/sim/TestExecutePlanAction.java  |  14 --
 .../cloud/autoscaling/sim/TestLargeCluster.java | 213 ++++++++++++++++++-
 .../autoscaling/sim/TestNodeAddedTrigger.java   |   2 +-
 .../autoscaling/sim/TestNodeLostTrigger.java    |   8 +-
 .../solrj/cloud/autoscaling/PolicyHelper.java   |   9 +-
 14 files changed, 479 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java b/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
index 520a269..5e1d06a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ActionThrottle.java
@@ -70,7 +70,7 @@ public class ActionThrottle {
     long diff = timeSource.getTime() - lastActionStartedAt;
     int diffMs = (int) TimeUnit.MILLISECONDS.convert(diff, TimeUnit.NANOSECONDS);
     long minNsBetweenActions = TimeUnit.NANOSECONDS.convert(minMsBetweenActions, TimeUnit.MILLISECONDS);
-    log.info("The last {} attempt started {}ms ago.", name, diffMs);
+    log.debug("The last {} attempt started {}ms ago.", name, diffMs);
     int sleep = 0;
     
     if (diffMs > 0 && diff < minNsBetweenActions) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
index 47b3440..bce0806 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
@@ -74,7 +74,7 @@ public class ExecutePlanAction extends TriggerActionBase {
             req.setWaitForFinalState(true);
             String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
             String znode = saveAsyncId(cloudManager.getDistribStateManager(), event, asyncId);
-            log.debug("Saved requestId: {} in znode: {}", asyncId, znode);
+            log.trace("Saved requestId: {} in znode: {}", asyncId, znode);
             // TODO: find a better way of using async calls using dataProvider API !!!
             req.setAsyncId(asyncId);
             SolrResponse asyncResponse = cloudManager.request(req);
@@ -132,7 +132,7 @@ public class ExecutePlanAction extends TriggerActionBase {
         statusResponse = (CollectionAdminRequest.RequestStatusResponse)cloudManager.request(CollectionAdminRequest.requestStatus(requestId));
         state = statusResponse.getRequestStatus();
         if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
-          log.debug("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
+          log.trace("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
           cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
           return statusResponse;
         } else if (state == RequestStatusState.NOT_FOUND) {
@@ -156,7 +156,7 @@ public class ExecutePlanAction extends TriggerActionBase {
         throw e;
       }
       if (i > 0 && i % 5 == 0) {
-        log.debug("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
+        log.trace("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
       }
       cloudManager.getTimeSource().sleep(5000);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
index 57c76c0..aaba1bd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
@@ -163,7 +163,7 @@ public class NodeLostTrigger extends TriggerBase {
               removeMarker(n);
             });
           } else  {
-            log.debug("NodeLostTrigger listener for lost nodes: {} is not ready, will try later", nodeNames);
+            log.debug("NodeLostTrigger processor for lost nodes: {} is not ready, will try later", nodeNames);
           }
         } else  {
           nodeNames.forEach(n -> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
index 45cd66b..ca4ed71 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
@@ -37,6 +37,10 @@ public class LiveNodesSet {
     return Collections.unmodifiableSet(set);
   }
 
+  public int size() {
+    return set.size();
+  }
+
   public void registerLiveNodesListener(LiveNodesListener listener) {
     listeners.add(listener);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index bde4b41..cd9f177 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -27,7 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
@@ -96,10 +96,10 @@ public class SimCloudManager implements SolrCloudManager {
   private TimeSource timeSource;
 
   private final List<SolrInputDocument> systemColl = Collections.synchronizedList(new ArrayList<>());
-  private final ExecutorService simCloudManagerPool;
-  private final Map<String, AtomicLong> opCounts = new ConcurrentHashMap<>();
+  private final Map<String, AtomicLong> opCounts = new ConcurrentSkipListMap<>();
 
 
+  private ExecutorService simCloudManagerPool;
   private Overseer.OverseerThread triggerThread;
   private ThreadGroup triggerThreadGroup;
   private SolrResourceLoader loader;
@@ -327,7 +327,8 @@ public class SimCloudManager implements SolrCloudManager {
 
   /**
    * Simulate the effect of restarting Overseer leader - in this case this means restarting the
-   * OverseerTriggerThread and optionally killing a node.
+   * OverseerTriggerThread and optionally killing a node. All background tasks currently in progress
+   * will be interrupted.
    * @param killNodeId optional nodeId to kill. If null then don't kill any node, just restart the thread
    */
   public void simRestartOverseer(String killNodeId) throws Exception {
@@ -335,9 +336,17 @@ public class SimCloudManager implements SolrCloudManager {
     triggerThread.interrupt();
     IOUtils.closeQuietly(triggerThread);
     if (killNodeId != null) {
-      simRemoveNode(killNodeId, true);
+      simRemoveNode(killNodeId, false);
     }
     objectCache.clear();
+
+    try {
+      simCloudManagerPool.shutdownNow();
+    } catch (Exception e) {
+      // ignore
+    }
+    simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
+
     OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
         new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
     triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
@@ -378,6 +387,10 @@ public class SimCloudManager implements SolrCloudManager {
     return opCounts;
   }
 
+  public void simResetOpCounts() {
+    opCounts.clear();
+  }
+
   /**
    * Get the number of processed operations of a specified type.
    * @param op operation name, eg. MOVEREPLICA
@@ -497,7 +510,7 @@ public class SimCloudManager implements SolrCloudManager {
       if (action == null) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
       }
-      LOG.debug("Invoking Collection Action :{} with params {}", action.toLower(), req.getParams().toQueryString());
+      LOG.trace("Invoking Collection Action :{} with params {}", action.toLower(), req.getParams().toQueryString());
       NamedList results = new NamedList();
       rsp.setResponse(results);
       incrementCount(action.name());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/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 1986bac..22f9fb9 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
@@ -30,7 +30,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
@@ -72,12 +71,17 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
+import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CommonParams.NAME;
 
 /**
@@ -108,7 +112,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
 
   private final ReentrantLock lock = new ReentrantLock();
 
-  private final ActionThrottle leaderThrottle;
+  private final Map<String, Map<String, ActionThrottle>> leaderThrottles = new ConcurrentHashMap<>();
 
   // default map of: operation -> delay
   private final Map<String, Long> defaultOpDelays = new HashMap<>();
@@ -116,7 +120,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   private final Map<String, Map<String, Long>> opDelays = new ConcurrentHashMap<>();
 
 
-  private volatile int clusterStateVersion = -1;
+  private volatile int clusterStateVersion = 0;
   private Map<String, Object> lastSavedProperties = null;
 
   private AtomicReference<Map<String, DocCollection>> collectionsStatesRef = new AtomicReference<>();
@@ -133,7 +137,6 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
     this.cloudManager = cloudManager;
     this.stateManager = cloudManager.getSimDistribStateManager();
-    this.leaderThrottle = new ActionThrottle("leader", 5000, cloudManager.getTimeSource());
     // names are CollectionAction operation names, delays are in ms (simulated time)
     defaultOpDelays.put(CollectionParams.CollectionAction.MOVEREPLICA.name(), 5000L);
     defaultOpDelays.put(CollectionParams.CollectionAction.DELETEREPLICA.name(), 5000L);
@@ -191,10 +194,15 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   }
 
   /**
-   * Reset the leader election throttle.
+   * Reset the leader election throttles.
    */
-  public void simResetLeaderThrottle() {
-    leaderThrottle.reset();
+  public void simResetLeaderThrottles() {
+    leaderThrottles.clear();
+  }
+
+  private ActionThrottle getThrottle(String collection, String shard) {
+    return leaderThrottles.computeIfAbsent(collection, coll -> new ConcurrentHashMap<>())
+        .computeIfAbsent(shard, s -> new ActionThrottle("leader", 5000, cloudManager.getTimeSource()));
   }
 
   /**
@@ -225,30 +233,6 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     nodeReplicaMap.putIfAbsent(nodeId, new ArrayList<>());
   }
 
-  // utility class to run leader election in a separate thread and with throttling
-  // Note: leader election is a no-op if a shard leader already exists for each shard
-  private class LeaderElection implements Callable<Boolean> {
-    Collection<String> collections;
-    boolean saveClusterState;
-
-    LeaderElection(Collection<String> collections, boolean saveClusterState) {
-      this.collections = collections;
-      this.saveClusterState = saveClusterState;
-    }
-
-    @Override
-    public Boolean call() {
-      leaderThrottle.minimumWaitBetweenActions();
-      leaderThrottle.markAttemptingAction();
-      try {
-        simRunLeaderElection(collections, saveClusterState);
-      } catch (Exception e) {
-        return false;
-      }
-      return true;
-    }
-  }
-
   /**
    * Remove node from a cluster. This is equivalent to a situation when a node is lost.
    * All replicas that were assigned to this node are marked as DOWN.
@@ -273,7 +257,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + nodeId);
       }
       if (!collections.isEmpty()) {
-        cloudManager.submit(new LeaderElection(collections, true));
+        simRunLeaderElection(collections, true);
       }
       return res;
     } finally {
@@ -326,7 +310,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
     if (!collections.isEmpty()) {
       collectionsStatesRef.set(null);
-      cloudManager.submit(new LeaderElection(collections, true));
+      simRunLeaderElection(collections, true);
       return true;
     } else {
       return false;
@@ -431,7 +415,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       }
       cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk - 10);
       if (runLeaderElection) {
-        cloudManager.submit(new LeaderElection(Collections.singleton(replicaInfo.getCollection()), true));
+        simRunLeaderElection(Collections.singleton(replicaInfo.getCollection()), true);
       }
     } finally {
       lock.unlock();
@@ -468,7 +452,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk + 10);
           }
           LOG.trace("-- simRemoveReplica {}", ri);
-          cloudManager.submit(new LeaderElection(Collections.singleton(ri.getCollection()), true));
+          simRunLeaderElection(Collections.singleton(ri.getCollection()), true);
           return;
         }
       }
@@ -482,14 +466,14 @@ public class SimClusterStateProvider implements ClusterStateProvider {
    * Save clusterstate.json to {@link DistribStateManager}.
    * @return saved state
    */
-  private ClusterState saveClusterState(ClusterState state) throws IOException {
+  private synchronized ClusterState saveClusterState(ClusterState state) throws IOException {
     byte[] data = Utils.toJSON(state);
     try {
       VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_STATE);
       int version = oldData != null ? oldData.getVersion() : -1;
+      Assert.assertEquals(clusterStateVersion, version + 1);
       stateManager.setData(ZkStateReader.CLUSTER_STATE, data, version);
-      LOG.trace("-- saved cluster state version=" + clusterStateVersion +
-          ", zkVersion=" + (version + 1) + ", {}", state);
+      clusterStateVersion++;
     } catch (Exception e) {
       throw new IOException(e);
     }
@@ -515,75 +499,89 @@ public class SimClusterStateProvider implements ClusterStateProvider {
    * @param collections list of affected collections
    * @param saveClusterState if true then save cluster state regardless of changes.
    */
-  private synchronized void simRunLeaderElection(Collection<String> collections, boolean saveClusterState) throws Exception {
+  private void simRunLeaderElection(Collection<String> collections, boolean saveClusterState) throws Exception {
     ClusterState state = getClusterState();
-    AtomicBoolean stateChanged = new AtomicBoolean(Boolean.FALSE);
-
+    if (saveClusterState) {
+      collectionsStatesRef.set(null);
+    }
     state.forEachCollection(dc -> {
       if (!collections.contains(dc.getName())) {
         return;
       }
-      dc.getSlices().forEach(s -> {
-        Replica leader = s.getLeader();
-        if (leader == null || !liveNodes.contains(leader.getNodeName())) {
-          LOG.trace("Running leader election for " + dc.getName() + " / " + s.getName());
-          if (s.getReplicas().isEmpty()) { // no replicas - punt
-            return;
+      dc.getSlices().forEach(s ->
+        cloudManager.submit(() -> {
+          simRunLeaderElection(dc.getName(), s, saveClusterState);
+          return true;
+        })
+      );
+    });
+  }
+
+  private void simRunLeaderElection(String collection, Slice s, boolean saveClusterState) throws Exception {
+    AtomicBoolean stateChanged = new AtomicBoolean(Boolean.FALSE);
+    Replica leader = s.getLeader();
+    if (leader == null || !liveNodes.contains(leader.getNodeName())) {
+      LOG.trace("Running leader election for " + collection + " / " + s.getName());
+      if (s.getReplicas().isEmpty()) { // no replicas - punt
+        return;
+      }
+      ActionThrottle lt = getThrottle(collection, s.getName());
+      lt.minimumWaitBetweenActions();
+      lt.markAttemptingAction();
+
+      // mark all replicas as non-leader (probably not necessary) and collect all active and live
+      List<ReplicaInfo> active = new ArrayList<>();
+      s.getReplicas().forEach(r -> {
+        AtomicReference<ReplicaInfo> riRef = new AtomicReference<>();
+        // find our ReplicaInfo for this replica
+        nodeReplicaMap.get(r.getNodeName()).forEach(info -> {
+          if (info.getName().equals(r.getName())) {
+            riRef.set(info);
           }
-          // mark all replicas as non-leader (probably not necessary) and collect all active and live
-          List<ReplicaInfo> active = new ArrayList<>();
-          s.getReplicas().forEach(r -> {
-            AtomicReference<ReplicaInfo> riRef = new AtomicReference<>();
-            // find our ReplicaInfo for this replica
-            nodeReplicaMap.get(r.getNodeName()).forEach(info -> {
-              if (info.getName().equals(r.getName())) {
-                riRef.set(info);
-              }
-            });
-            ReplicaInfo ri = riRef.get();
-            if (ri == null) {
-              throw new IllegalStateException("-- could not find ReplicaInfo for replica " + r);
-            }
-            synchronized (ri) {
-              if (ri.getVariables().remove(ZkStateReader.LEADER_PROP) != null) {
-                stateChanged.set(true);
-              }
-              if (r.isActive(liveNodes.get())) {
-                active.add(ri);
-              } else { // if it's on a node that is not live mark it down
-                if (!liveNodes.contains(r.getNodeName())) {
-                  ri.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
-                }
-              }
-            }
-          });
-          if (active.isEmpty()) {
-            LOG.warn("-- can't find any active replicas for " + dc.getName() + " / " + s.getName());
-            return;
+        });
+        ReplicaInfo ri = riRef.get();
+        if (ri == null) {
+          throw new IllegalStateException("-- could not find ReplicaInfo for replica " + r);
+        }
+        synchronized (ri) {
+          if (ri.getVariables().remove(ZkStateReader.LEADER_PROP) != null) {
+            stateChanged.set(true);
           }
-          // pick first active one
-          ReplicaInfo ri = null;
-          for (ReplicaInfo a : active) {
-            if (!a.getType().equals(Replica.Type.PULL)) {
-              ri = a;
-              break;
+          if (r.isActive(liveNodes.get())) {
+            active.add(ri);
+          } else { // if it's on a node that is not live mark it down
+            if (!liveNodes.contains(r.getNodeName())) {
+              ri.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
+              stateChanged.set(true);
             }
           }
-          if (ri == null) {
-            LOG.warn("-- can't find any suitable replica type for " + dc.getName() + " / " + s.getName());
-            return;
-          }
-          synchronized (ri) {
-            ri.getVariables().put(ZkStateReader.LEADER_PROP, "true");
-          }
-          stateChanged.set(true);
-          LOG.debug("-- elected new leader for " + dc.getName() + " / " + s.getName() + ": " + ri);
-        } else {
-          LOG.trace("-- already has leader for {} / {}", dc.getName(), s.getName());
         }
       });
-    });
-    if (saveClusterState || stateChanged.get()) {
+      if (active.isEmpty()) {
+        LOG.warn("-- can't find any active replicas for " + collection + " / " + s.getName());
+        return;
+      }
+      // pick first active one
+      ReplicaInfo ri = null;
+      for (ReplicaInfo a : active) {
+        if (!a.getType().equals(Replica.Type.PULL)) {
+          ri = a;
+          break;
+        }
+      }
+      if (ri == null) {
+        LOG.warn("-- can't find any suitable replica type for " + collection + " / " + s.getName());
+        return;
+      }
+      synchronized (ri) {
+        ri.getVariables().put(ZkStateReader.LEADER_PROP, "true");
+      }
+      stateChanged.set(true);
+      LOG.debug("-- elected new leader for " + collection + " / " + s.getName() + ": " + ri);
+    } else {
+      LOG.trace("-- already has leader for {} / {}", collection, s.getName());
+    }
+    if (stateChanged.get()) {
       collectionsStatesRef.set(null);
     }
   }
@@ -618,6 +616,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     if (sessionWrapper.get() != null) {
       sessionWrapper.get().release();
     }
+    // calculate expected number of positions
+    int numTlogReplicas = props.getInt(TLOG_REPLICAS, 0);
+    int numNrtReplicas = props.getInt(NRT_REPLICAS, props.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
+    int numPullReplicas = props.getInt(PULL_REPLICAS, 0);
+    int totalReplicas = shardNames.size() * (numNrtReplicas + numPullReplicas + numTlogReplicas);
+    Assert.assertEquals("unexpected number of replica positions", totalReplicas, replicaPositions.size());
     final CountDownLatch finalStateLatch = new CountDownLatch(replicaPositions.size());
     AtomicInteger replicaNum = new AtomicInteger(1);
     replicaPositions.forEach(pos -> {
@@ -652,7 +656,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         }
       });
     });
-    cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+    simRunLeaderElection(Collections.singleton(collectionName), true);
     if (waitForFinalState) {
       boolean finished = finalStateLatch.await(cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, 60, TimeUnit.MILLISECONDS),
           TimeUnit.MILLISECONDS);
@@ -678,6 +682,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     try {
       collProperties.remove(collection);
       sliceProperties.remove(collection);
+      leaderThrottles.remove(collection);
 
       opDelay(collection, CollectionParams.CollectionAction.DELETE.name());
 
@@ -716,9 +721,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       nodeReplicaMap.clear();
       collProperties.clear();
       sliceProperties.clear();
-      cloudManager.getSimNodeStateProvider().simGetAllNodeValues().forEach((n, values) -> {
-        values.put("cores", 0);
-      });
+      leaderThrottles.clear();
+      cloudManager.getSimNodeStateProvider().simGetAllNodeValues().forEach((n, values) -> values.put("cores", 0));
       collectionsStatesRef.set(null);
     } finally {
       lock.unlock();
@@ -831,7 +835,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       });
       Map<String, Object> colProps = collProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>());
 
-      cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+      simRunLeaderElection(Collections.singleton(collectionName), true);
       results.add("success", "");
     } finally {
       lock.unlock();
@@ -899,7 +903,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           solrCoreName, collectionName, replicaPosition.shard, replicaPosition.type, subShardNodeName, replicaProps);
       simAddReplica(replicaPosition.node, ri, false);
     }
-    cloudManager.submit(new LeaderElection(Collections.singleton(collectionName), true));
+    simRunLeaderElection(Collections.singleton(collectionName), true);
     results.add("success", "");
 
   }
@@ -1201,7 +1205,6 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
     lock.lock();
     collectionsStatesRef.set(null);
-    clusterStateVersion++;
     saveClusterState.set(true);
     try {
       Map<String, Map<String, Map<String, Replica>>> collMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
index a96a1d5..8310836 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
@@ -26,7 +26,9 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
 import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
@@ -150,6 +152,35 @@ public class SimNodeStateProvider implements NodeStateProvider {
   }
 
   /**
+   * Remove values that correspond to dead nodes. If values contained a 'nodeRole'
+   * key then /roles.json is updated.
+   */
+  public void simRemoveDeadNodes() {
+    Set<String> myNodes = new HashSet<>(nodeValues.keySet());
+    myNodes.removeAll(liveNodesSet.get());
+    AtomicBoolean updateRoles = new AtomicBoolean(false);
+    myNodes.forEach(n -> {
+      LOG.debug("- removing dead node values: " + n);
+      Map<String, Object> vals = nodeValues.remove(n);
+      if (vals.containsKey("nodeRole")) {
+        updateRoles.set(true);
+      }
+    });
+    if (updateRoles.get()) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Return a set of nodes that are not live but their values are still present.
+   */
+  public Set<String> simGetDeadNodes() {
+    Set<String> myNodes = new TreeSet<>(nodeValues.keySet());
+    myNodes.removeAll(liveNodesSet.get());
+    return myNodes;
+  }
+
+  /**
    * Get all node values.
    */
   public Map<String, Map<String, Object>> simGetAllNodeValues() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
index be7209b..77ddbc0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
@@ -20,15 +20,22 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
+import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.CollectionStatePredicate;
 import org.apache.solr.common.cloud.DocCollection;
@@ -63,9 +70,11 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
 
   /** The cluster. */
   protected static SimCloudManager cluster;
+  protected static int clusterNodeCount = 0;
 
   protected static void configureCluster(int nodeCount, TimeSource timeSource) throws Exception {
     cluster = SimCloudManager.createCluster(nodeCount, timeSource);
+    clusterNodeCount = nodeCount;
   }
 
   @AfterClass
@@ -77,11 +86,90 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
   }
 
   @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    if (cluster != null) {
+      log.info("\n");
+      log.info("#############################################");
+      log.info("############ FINAL CLUSTER STATS ############");
+      log.info("#############################################\n");
+      log.info("## Live nodes:\t\t" + cluster.getLiveNodesSet().size());
+      int emptyNodes = 0;
+      int maxReplicas = 0;
+      int minReplicas = Integer.MAX_VALUE;
+      Map<String, Map<Replica.State, AtomicInteger>> replicaStates = new TreeMap<>();
+      int numReplicas = 0;
+      for (String node : cluster.getLiveNodesSet().get()) {
+        List<ReplicaInfo> replicas = cluster.getSimClusterStateProvider().simGetReplicaInfos(node);
+        numReplicas += replicas.size();
+        if (replicas.size() > maxReplicas) {
+          maxReplicas = replicas.size();
+        }
+        if (minReplicas > replicas.size()) {
+          minReplicas = replicas.size();
+        }
+        for (ReplicaInfo ri : replicas) {
+          replicaStates.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>())
+              .computeIfAbsent(ri.getState(), s -> new AtomicInteger())
+              .incrementAndGet();
+        }
+        if (replicas.isEmpty()) {
+          emptyNodes++;
+        }
+      }
+      if (minReplicas == Integer.MAX_VALUE) {
+        minReplicas = 0;
+      }
+      log.info("## Empty nodes:\t" + emptyNodes);
+      Set<String> deadNodes = cluster.getSimNodeStateProvider().simGetDeadNodes();
+      log.info("## Dead nodes:\t\t" + deadNodes.size());
+      deadNodes.forEach(n -> log.info("##\t\t" + n));
+      log.info("## Collections:\t" + cluster.getSimClusterStateProvider().simListCollections());
+      log.info("## Max replicas per node:\t" + maxReplicas);
+      log.info("## Min replicas per node:\t" + minReplicas);
+      log.info("## Total replicas:\t\t" + numReplicas);
+      replicaStates.forEach((c, map) -> {
+        AtomicInteger repCnt = new AtomicInteger();
+        map.forEach((s, cnt) -> repCnt.addAndGet(cnt.get()));
+        log.info("## * " + c + "\t\t" + repCnt.get());
+        map.forEach((s, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-12s  %4d", s, cnt.get())));
+      });
+      log.info("######### Final Solr op counts ##########");
+      cluster.simGetOpCounts().forEach((k, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-14s  %4d", k, cnt.get())));
+      log.info("######### Autoscaling event counts ###########");
+      TreeMap<String, Map<String, AtomicInteger>> counts = new TreeMap<>();
+      for (SolrInputDocument d : cluster.simGetSystemCollection()) {
+        if (!"autoscaling_event".equals(d.getFieldValue("type"))) {
+          continue;
+        }
+        counts.computeIfAbsent((String)d.getFieldValue("event.source_s"), s -> new TreeMap<>())
+            .computeIfAbsent((String)d.getFieldValue("stage_s"), s -> new AtomicInteger())
+            .incrementAndGet();
+      }
+      counts.forEach((trigger, map) -> {
+        log.info("## * Trigger: " + trigger);
+        map.forEach((s, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-11s  %4d", s, cnt.get())));
+      });
+    }
+  }
+
+  @Override
   public void setUp() throws Exception {
     super.setUp();
     if (cluster != null) {
-      // clear any persisted auto scaling configuration
+      // clear any persisted configuration
       cluster.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), -1);
+      cluster.getDistribStateManager().setData(ZkStateReader.ROLES, Utils.toJSON(new HashMap<>()), -1);
+      // restore the expected number of nodes
+      int currentSize = cluster.getLiveNodesSet().size();
+      if (currentSize < clusterNodeCount) {
+        int addCnt = clusterNodeCount - currentSize;
+        while (addCnt-- > 0) {
+          cluster.simAddNode();
+        }
+      } else if (currentSize > clusterNodeCount) {
+        cluster.simRemoveRandomNodes(currentSize - clusterNodeCount, true, random());
+      }
       // clean any persisted trigger state or events
       removeChildren(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
       removeChildren(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
@@ -89,8 +177,12 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
       removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
       cluster.getSimClusterStateProvider().simDeleteAllCollections();
       cluster.simClearSystemCollection();
-      cluster.getSimClusterStateProvider().simResetLeaderThrottle();
+      // clear any dead nodes
+      cluster.getSimNodeStateProvider().simRemoveDeadNodes();
+      cluster.getSimClusterStateProvider().simResetLeaderThrottles();
       cluster.simRestartOverseer(null);
+      cluster.getTimeSource().sleep(5000);
+      cluster.simResetOpCounts();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
index b7053d7d..306d35f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
@@ -81,15 +81,6 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
     triggerFiredLatch = new CountDownLatch(1);
     actionContextPropsRef.set(null);
 
-    if (cluster.getClusterStateProvider().getLiveNodes().size() > NODE_COUNT) {
-      // stop some to get to original state
-      int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
-      for (int i = 0; i < numJetties - NODE_COUNT; i++) {
-        String node = cluster.getSimClusterStateProvider().simGetRandomNode(random());
-        cluster.getSimClusterStateProvider().simRemoveNode(node);
-      }
-    }
-
     String setClusterPolicyCommand = "{" +
         " 'set-cluster-policy': [" +
         "      {'cores':'<10', 'node':'#ANY'}," +
@@ -202,7 +193,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
         "'waitFor' : '1s'," +
         "'enabled' : true," +
         "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
-        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "{'name':'test','class':'" + AssertingTriggerAction.class.getName() + "'}]" +
         "}}";
     SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
     NamedList<Object> response = solrClient.request(req);
@@ -245,7 +236,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
     Map context = actionContextPropsRef.get();
     assertNotNull(context);
     List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
-    assertNotNull("The operations computed by ComputePlanAction should not be null " + actionContextPropsRef.get(), operations);
+    assertNotNull("The operations computed by ComputePlanAction should not be null " + actionContextPropsRef.get() + "\nevent: " + eventRef.get(), operations);
     operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
     assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
index 18d76dc..bb03fc5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
@@ -46,7 +46,6 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.common.util.TimeSource;
 import org.junit.After;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -66,19 +65,6 @@ public class TestExecutePlanAction extends SimSolrCloudTestCase {
     configureCluster(NODE_COUNT, TimeSource.get("simTime:50"));
   }
 
-  @Before
-  public void setUp() throws Exception  {
-    super.setUp();
-
-    if (cluster.getClusterStateProvider().getLiveNodes().size() < NODE_COUNT) {
-      // start some to get to original state
-      int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
-      for (int i = 0; i < NODE_COUNT - numJetties; i++) {
-        cluster.simAddNode();
-      }
-    }
-  }
-
   @After
   public void printState() throws Exception {
     log.info("-------------_ FINAL STATE --------------");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
index 034a039..6758987 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
@@ -42,6 +42,7 @@ import org.apache.solr.cloud.autoscaling.TriggerActionBase;
 import org.apache.solr.cloud.autoscaling.TriggerEvent;
 import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
 import org.apache.solr.cloud.autoscaling.CapturedEvent;
+import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
@@ -79,15 +80,10 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
   @Before
   public void setupTest() throws Exception {
 
-    waitForSeconds = 1 + random().nextInt(3);
+    waitForSeconds = 5;
     triggerFiredCount.set(0);
     triggerFiredLatch = new CountDownLatch(1);
     listenerEvents.clear();
-    while (cluster.getClusterStateProvider().getLiveNodes().size() < NUM_NODES) {
-      // perhaps a test stopped a node but didn't start it back
-      // lets start a node
-      cluster.simAddNode();
-    }
   }
 
   public static class TestTriggerListener extends TriggerListenerBase {
@@ -163,6 +159,7 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
         "conf", 5, 5, 5, 5);
     create.setMaxShardsPerNode(1);
+    create.setAutoAddReplicas(false);
     create.setCreateNodeSet(String.join(",", nodes));
     create.process(solrClient);
 
@@ -196,7 +193,6 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
     }
 
     log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
-    log.info("OP COUNTS: " + cluster.simGetOpCounts());
     long newMoveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
     log.info("==== Flaky replicas: {}. Additional MOVEREPLICA count: {}", flakyReplicas, (newMoveReplicaOps - moveReplicaOps));
     // flaky nodes lead to a number of MOVEREPLICA that is non-zero but lower than the number of flaky replicas
@@ -206,6 +202,208 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
   }
 
   @Test
+  public void testAddNode() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // create a collection with more than 1 replica per node
+    String collectionName = "testNodeAdded";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", NUM_NODES / 10, NUM_NODES / 10, NUM_NODES / 10, NUM_NODES / 10);
+    create.setMaxShardsPerNode(5);
+    create.setAutoAddReplicas(false);
+    create.process(solrClient);
+
+    log.info("Ready after " + waitForState(collectionName, 20 * NUM_NODES, TimeUnit.SECONDS, clusterShape(NUM_NODES / 10, NUM_NODES / 10 * 3)) + " ms");
+
+    int numAddNode = NUM_NODES / 5;
+    List<String> addNodesList = new ArrayList<>(numAddNode);
+    for (int i = 0; i < numAddNode; i++) {
+      addNodesList.add(cluster.simAddNode());
+      cluster.getTimeSource().sleep(5000);
+    }
+    List<SolrInputDocument> systemColl = cluster.simGetSystemCollection();
+    int startedEventPos = -1;
+    for (int i = 0; i < systemColl.size(); i++) {
+      SolrInputDocument d = systemColl.get(i);
+      if (!"node_added_trigger".equals(d.getFieldValue("event.source_s"))) {
+        continue;
+      }
+      if ("NODEADDED".equals(d.getFieldValue("event.type_s")) &&
+          "STARTED".equals(d.getFieldValue("stage_s"))) {
+        startedEventPos = i;
+        break;
+      }
+    }
+    assertTrue("no STARTED event", startedEventPos > -1);
+    SolrInputDocument startedEvent = systemColl.get(startedEventPos);
+    int ignored = 0;
+    int lastIgnoredPos = startedEventPos;
+    for (int i = startedEventPos + 1; i < systemColl.size(); i++) {
+      SolrInputDocument d = systemColl.get(i);
+      if (!"node_added_trigger".equals(d.getFieldValue("event.source_s"))) {
+        continue;
+      }
+      if ("NODEADDED".equals(d.getFieldValue("event.type_s"))) {
+        if ("IGNORED".equals(d.getFieldValue("stage_s"))) {
+          ignored++;
+          lastIgnoredPos = i;
+        }
+      }
+    }
+    assertTrue("no IGNORED events", ignored > 0);
+    // make sure some replicas have been moved
+    assertTrue("no MOVEREPLICA ops?", cluster.simGetOpCount("MOVEREPLICA") > 0);
+
+    log.info("Ready after " + waitForState(collectionName, 20 * NUM_NODES, TimeUnit.SECONDS, clusterShape(NUM_NODES / 10, NUM_NODES / 10 * 3)) + " ms");
+
+    int count = 50;
+    SolrInputDocument finishedEvent = null;
+    long lastNumOps = cluster.simGetOpCount("MOVEREPLICA");
+    while (count-- > 0) {
+      cluster.getTimeSource().sleep(150000);
+      long currentNumOps = cluster.simGetOpCount("MOVEREPLICA");
+      if (currentNumOps == lastNumOps) {
+        int size = systemColl.size() - 1;
+        for (int i = size; i > lastIgnoredPos; i--) {
+          SolrInputDocument d = systemColl.get(i);
+          if (!"node_added_trigger".equals(d.getFieldValue("event.source_s"))) {
+            continue;
+          }
+          if ("SUCCEEDED".equals(d.getFieldValue("stage_s"))) {
+            finishedEvent = d;
+            break;
+          }
+        }
+        break;
+      } else {
+        lastNumOps = currentNumOps;
+      }
+    }
+
+    assertTrue("did not finish processing changes", finishedEvent != null);
+    long delta = (Long)finishedEvent.getFieldValue("event.time_l") - (Long)startedEvent.getFieldValue("event.time_l");
+    log.info("#### System stabilized after " + TimeUnit.NANOSECONDS.toMillis(delta) + " ms");
+    assertTrue("unexpected number of MOVEREPLICA ops", cluster.simGetOpCount("MOVEREPLICA") > 1);
+  }
+
+  @Test
+  public void testNodeLost() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // create a collection with 1 replica per node
+    String collectionName = "testNodeLost";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", NUM_NODES / 5, NUM_NODES / 10);
+    create.setMaxShardsPerNode(5);
+    create.setAutoAddReplicas(false);
+    create.process(solrClient);
+
+    log.info("Ready after " + waitForState(collectionName, 20 * NUM_NODES, TimeUnit.SECONDS, clusterShape(NUM_NODES / 5, NUM_NODES / 10)) + " ms");
+
+    // start killing nodes
+    int numNodes = NUM_NODES / 5;
+    List<String> nodes = new ArrayList<>(cluster.getLiveNodesSet().get());
+    for (int i = 0; i < numNodes; i++) {
+      // this may also select a node where a replica is moved to, so the total number of
+      // MOVEREPLICA may vary
+      cluster.simRemoveNode(nodes.get(i), false);
+      cluster.getTimeSource().sleep(4000);
+    }
+    List<SolrInputDocument> systemColl = cluster.simGetSystemCollection();
+    int startedEventPos = -1;
+    for (int i = 0; i < systemColl.size(); i++) {
+      SolrInputDocument d = systemColl.get(i);
+      if (!"node_lost_trigger".equals(d.getFieldValue("event.source_s"))) {
+        continue;
+      }
+      if ("NODELOST".equals(d.getFieldValue("event.type_s")) &&
+          "STARTED".equals(d.getFieldValue("stage_s"))) {
+        startedEventPos = i;
+        break;
+      }
+    }
+    assertTrue("no STARTED event: " + systemColl, startedEventPos > -1);
+    SolrInputDocument startedEvent = systemColl.get(startedEventPos);
+    int ignored = 0;
+    int lastIgnoredPos = startedEventPos;
+    for (int i = startedEventPos + 1; i < systemColl.size(); i++) {
+      SolrInputDocument d = systemColl.get(i);
+      if (!"node_lost_trigger".equals(d.getFieldValue("event.source_s"))) {
+        continue;
+      }
+      if ("NODELOST".equals(d.getFieldValue("event.type_s"))) {
+        if ("IGNORED".equals(d.getFieldValue("stage_s"))) {
+          ignored++;
+          lastIgnoredPos = i;
+        }
+      }
+    }
+    assertTrue("no IGNORED events", ignored > 0);
+    // make sure some replicas have been moved
+    assertTrue("no MOVEREPLICA ops?", cluster.simGetOpCount("MOVEREPLICA") > 0);
+
+    log.info("Ready after " + waitForState(collectionName, 20 * NUM_NODES, TimeUnit.SECONDS, clusterShape(NUM_NODES / 5, NUM_NODES / 10)) + " ms");
+
+    int count = 50;
+    SolrInputDocument finishedEvent = null;
+    long lastNumOps = cluster.simGetOpCount("MOVEREPLICA");
+    while (count-- > 0) {
+      cluster.getTimeSource().sleep(150000);
+      long currentNumOps = cluster.simGetOpCount("MOVEREPLICA");
+      if (currentNumOps == lastNumOps) {
+        int size = systemColl.size() - 1;
+        for (int i = size; i > lastIgnoredPos; i--) {
+          SolrInputDocument d = systemColl.get(i);
+          if (!"node_lost_trigger".equals(d.getFieldValue("event.source_s"))) {
+            continue;
+          }
+          if ("SUCCEEDED".equals(d.getFieldValue("stage_s"))) {
+            finishedEvent = d;
+            break;
+          }
+        }
+        break;
+      } else {
+        lastNumOps = currentNumOps;
+      }
+    }
+
+    assertTrue("did not finish processing changes", finishedEvent != null);
+    long delta = (Long)finishedEvent.getFieldValue("event.time_l") - (Long)startedEvent.getFieldValue("event.time_l");
+    log.info("#### System stabilized after " + TimeUnit.NANOSECONDS.toMillis(delta) + " ms");
+    long ops = cluster.simGetOpCount("MOVEREPLICA");
+    assertTrue("unexpected number of MOVEREPLICA ops: " + ops, ops >= 40);
+  }
+
+  @Test
   @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
   public void testSearchRate() throws Exception {
     SolrClient solrClient = cluster.simGetSolrClient();
@@ -255,7 +453,6 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
     // simulate search traffic
     cluster.getSimClusterStateProvider().simSetShardValue(collectionName, "shard1", metricName, 40, true);
 
-    Thread.sleep(1000000000);
 //    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
 //    assertTrue("The trigger did not fire at all", await);
     // wait for listener to capture the SUCCEEDED stage

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
index 9b4e2bc..f938d5c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeAddedTrigger.java
@@ -137,7 +137,7 @@ public class TestNodeAddedTrigger extends SimSolrCloudTestCase {
         return true;
       });
       trigger.run(); // first run should detect the new node
-      cluster.simRemoveNode(newNode, true);
+      cluster.simRemoveNode(newNode, false);
       int counter = 0;
       do {
         trigger.run();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
index 109cee3..1474c69 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestNodeLostTrigger.java
@@ -81,8 +81,8 @@ public class TestNodeLostTrigger extends SimSolrCloudTestCase {
       Iterator<String> it = cluster.getLiveNodesSet().get().iterator();
       String lostNodeName1 = it.next();
       String lostNodeName2 = it.next();
-      cluster.simRemoveNode(lostNodeName1, true);
-      cluster.simRemoveNode(lostNodeName2, true);
+      cluster.simRemoveNode(lostNodeName1, false);
+      cluster.simRemoveNode(lostNodeName2, false);
       timeSource.sleep(1000);
 
       AtomicBoolean fired = new AtomicBoolean(false);
@@ -223,7 +223,7 @@ public class TestNodeLostTrigger extends SimSolrCloudTestCase {
       trigger.run(); // starts tracking live nodes
 
       // stop the newly created node
-      cluster.simRemoveNode(newNode, true);
+      cluster.simRemoveNode(newNode, false);
 
       AtomicInteger callCount = new AtomicInteger(0);
       AtomicBoolean fired = new AtomicBoolean(false);
@@ -263,7 +263,7 @@ public class TestNodeLostTrigger extends SimSolrCloudTestCase {
     trigger.run();
 
     // stop the newly created node
-    cluster.simRemoveNode(newNode, true);
+    cluster.simRemoveNode(newNode, false);
 
     trigger.run(); // this run should detect the lost node
     trigger.close(); // close the old trigger

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/091f45dd/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
index 40ca619..489a9f9 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
@@ -47,9 +47,15 @@ import static org.apache.solr.common.util.Utils.time;
 import static org.apache.solr.common.util.Utils.timeElapsed;
 
 public class PolicyHelper {
-  private static ThreadLocal<Map<String, String>> policyMapping = new ThreadLocal<>();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private static final String POLICY_MAPPING_KEY = "PolicyHelper.policyMapping";
+
+  private static ThreadLocal<Map<String, String>> getPolicyMapping(SolrCloudManager cloudManager) {
+    return (ThreadLocal<Map<String, String>>)cloudManager.getObjectCache()
+        .computeIfAbsent(POLICY_MAPPING_KEY, k -> new ThreadLocal<>());
+  }
+
   public static List<ReplicaPosition> getReplicaLocations(String collName, AutoScalingConfig autoScalingConfig,
                                                           SolrCloudManager cloudManager,
                                                           Map<String, String> optionalPolicyMapping,
@@ -59,6 +65,7 @@ public class PolicyHelper {
                                                           int pullReplicas,
                                                           List<String> nodesList) {
     List<ReplicaPosition> positions = new ArrayList<>();
+    ThreadLocal<Map<String, String>> policyMapping = getPolicyMapping(cloudManager);
     ClusterStateProvider stateProvider = new DelegatingClusterStateProvider(cloudManager.getClusterStateProvider()) {
       @Override
       public String getPolicyNameByCollection(String coll) {


[52/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11783: Rename core in solr standalone mode is not persisted

Posted by da...@apache.org.
SOLR-11783: Rename core in solr standalone mode is not persisted


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

Branch: refs/heads/jira/solr-11702
Commit: 137675ace7fe0ca3264d18fd11e9e61cb535515a
Parents: e3f9038
Author: Erick Erickson <er...@apache.org>
Authored: Tue Dec 26 22:16:52 2017 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Tue Dec 26 22:16:52 2017 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  5 +++
 .../apache/solr/core/CorePropertiesLocator.java |  6 ++++
 .../handler/admin/CoreAdminHandlerTest.java     | 35 ++++++++++++++++++++
 3 files changed, 46 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/137675ac/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1d4ac2a..68003cb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -63,6 +63,11 @@ New Features
 * SOLR-11201: Implement autoscaling trigger for arbitrary metrics that creates events when
   a given metric breaches a threshold (shalin)
 
+Bug Fixes
+----------------------
+
+* SOLR-11783: Rename core in solr standalone mode is not persisted (Erick Erickson)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/137675ac/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java b/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
index 99c101b..70acd84 100644
--- a/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
+++ b/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
@@ -116,6 +116,12 @@ public class CorePropertiesLocator implements CoresLocator {
 
   @Override
   public void rename(CoreContainer cc, CoreDescriptor oldCD, CoreDescriptor newCD) {
+    String oldName = newCD.getPersistableStandardProperties().getProperty(CoreDescriptor.CORE_NAME);
+    String newName = newCD.coreProperties.getProperty(CoreDescriptor.CORE_NAME);
+    if (oldName == null ||
+        (newName != null && oldName.equals(newName) == false)) {
+      newCD.getPersistableStandardProperties().put(CoreDescriptor.CORE_NAME, newName);
+    }
     persist(cc, newCD);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/137675ac/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
index b9a282a..e9a0e31 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
@@ -17,11 +17,14 @@
 package org.apache.solr.handler.admin;
 
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Map;
+import java.util.Properties;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.commons.io.FileUtils;
@@ -256,6 +259,13 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     copySolrHomeToTemp(solrHomeDirectory, "corex");
     File corex = new File(solrHomeDirectory, "corex");
     FileUtils.write(new File(corex, "core.properties"), "", StandardCharsets.UTF_8);
+
+    copySolrHomeToTemp(solrHomeDirectory, "corerename");
+
+    File coreRename = new File(solrHomeDirectory, "corerename");
+    File renamePropFile = new File(coreRename, "core.properties");
+    FileUtils.write(renamePropFile, "", StandardCharsets.UTF_8);
+
     JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
     runner.start();
 
@@ -273,11 +283,36 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
       req.process(client);
     }
 
+    // Make sure a renamed core
+    // 1> has the property persisted (SOLR-11783)
+    // 2> is deleted after rename properly.
+
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
+      CoreAdminRequest.renameCore("corerename", "brand_new_core_name", client);
+      Properties props = new Properties();
+      try (InputStreamReader is = new InputStreamReader(new FileInputStream(renamePropFile), StandardCharsets.UTF_8)) {
+        props.load(is);
+      }
+      assertEquals("Name should have been persisted!", "brand_new_core_name", props.getProperty("name"));
+    }
+
+
+    try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
+      CoreAdminRequest.Unload req = new CoreAdminRequest.Unload(false);
+      req.setDeleteInstanceDir(true);
+      req.setCoreName("brand_new_core_name");
+      req.process(client);
+    }
+
+
     runner.stop();
 
     assertFalse("Instance directory exists after core unload with deleteInstanceDir=true : " + corex,
         corex.exists());
 
+    assertFalse("Instance directory exists after core unload with deleteInstanceDir=true : " + coreRename,
+        coreRename.exists());
+
   }
 
   @Test


[51/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8097: Implement maxScore() on disjunctions.

Posted by da...@apache.org.
LUCENE-8097: Implement maxScore() on disjunctions.


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

Branch: refs/heads/jira/solr-11702
Commit: e3f90385b4928e3639ee09a907df323e452c74de
Parents: 01023a9
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Dec 26 14:19:26 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Dec 26 14:19:26 2017 +0100

----------------------------------------------------------------------
 .../lucene/search/DisjunctionMaxQuery.java      | 18 +++++---
 .../lucene/search/DisjunctionMaxScorer.java     | 46 ++++++++++++++++----
 .../lucene/search/DisjunctionSumScorer.java     | 20 +++++++--
 .../org/apache/lucene/search/WANDScorer.java    | 14 +++++-
 .../java/org/apache/lucene/util/MathUtil.java   | 15 +++++++
 .../queryparser/xml/DisjunctionMaxQuery.xml     |  4 +-
 .../lucene/queryparser/xml/TestCoreParser.java  |  2 +-
 7 files changed, 98 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
index 97c02a6..3285baf 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
@@ -62,6 +62,9 @@ public final class DisjunctionMaxQuery extends Query implements Iterable<Query>
    */
   public DisjunctionMaxQuery(Collection<Query> disjuncts, float tieBreakerMultiplier) {
     Objects.requireNonNull(disjuncts, "Collection of Querys must not be null");
+    if (tieBreakerMultiplier < 0 || tieBreakerMultiplier > 1) {
+      throw new IllegalArgumentException("tieBreakerMultiplier must be in [0, 1]");
+    }
     this.tieBreakerMultiplier = tieBreakerMultiplier;
     this.disjuncts = disjuncts.toArray(new Query[disjuncts.size()]);
   }
@@ -156,20 +159,25 @@ public final class DisjunctionMaxQuery extends Query implements Iterable<Query>
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
       boolean match = false;
-      float max = Float.NEGATIVE_INFINITY;
-      double sum = 0;
+      float max = 0;
+      double otherSum = 0;
       List<Explanation> subs = new ArrayList<>();
       for (Weight wt : weights) {
         Explanation e = wt.explain(context, doc);
         if (e.isMatch()) {
           match = true;
           subs.add(e);
-          sum += e.getValue();
-          max = Math.max(max, e.getValue());
+          float score = e.getValue();
+          if (score >= max) {
+            otherSum += max;
+            max = score;
+          } else {
+            otherSum += score;
+          }
         }
       }
       if (match) {
-        final float score = (float) (max + (sum - max) * tieBreakerMultiplier);
+        final float score = (float) (max + otherSum * tieBreakerMultiplier);
         final String desc = tieBreakerMultiplier == 0.0f ? "max of:" : "max plus " + tieBreakerMultiplier + " times others of:";
         return Explanation.match(score, desc, subs);
       } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
index 084de66..c5c3640 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
@@ -19,6 +19,8 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.lucene.util.MathUtil;
+
 /**
  * The Scorer for DisjunctionMaxQuery.  The union of all documents generated by the the subquery scorers
  * is generated in document number order.  The score for each document is the maximum of the scores computed
@@ -28,6 +30,7 @@ import java.util.List;
 final class DisjunctionMaxScorer extends DisjunctionScorer {
   /* Multiplier applied to non-maximum-scoring subqueries for a document as they are summed into the result. */
   private final float tieBreakerMultiplier;
+  private final float maxScore;
 
   /**
    * Creates a new instance of DisjunctionMaxScorer
@@ -43,25 +46,52 @@ final class DisjunctionMaxScorer extends DisjunctionScorer {
   DisjunctionMaxScorer(Weight weight, float tieBreakerMultiplier, List<Scorer> subScorers, boolean needsScores) {
     super(weight, subScorers, needsScores);
     this.tieBreakerMultiplier = tieBreakerMultiplier;
+    if (tieBreakerMultiplier < 0 || tieBreakerMultiplier > 1) {
+      throw new IllegalArgumentException("tieBreakerMultiplier must be in [0, 1]");
+    }
+
+    float scoreMax = 0;
+    double otherScoreSum = 0;
+    for (Scorer scorer : subScorers) {
+      float subScore = scorer.maxScore();
+      if (subScore >= scoreMax) {
+        otherScoreSum += scoreMax;
+        scoreMax = subScore;
+      } else {
+        otherScoreSum += subScore;
+      }
+    }
+
+    if (tieBreakerMultiplier == 0) {
+      this.maxScore = scoreMax;
+    } else {
+      // The error of sums depends on the order in which values are summed up. In
+      // order to avoid this issue, we compute an upper bound of the value that
+      // the sum may take. If the max relative error is b, then it means that two
+      // sums are always within 2*b of each other.
+      otherScoreSum *= (1 + 2 * MathUtil.sumRelativeErrorBound(subScorers.size() - 1));
+      this.maxScore = (float) (scoreMax + otherScoreSum * tieBreakerMultiplier);
+    }
   }
 
   @Override
   protected float score(DisiWrapper topList) throws IOException {
-    double scoreSum = 0;
-    float scoreMax = Float.NEGATIVE_INFINITY;
+    float scoreMax = 0;
+    double otherScoreSum = 0;
     for (DisiWrapper w = topList; w != null; w = w.next) {
-      final float subScore = w.scorer.score();
-      scoreSum += subScore;
-      if (subScore > scoreMax) {
+      float subScore = w.scorer.score();
+      if (subScore >= scoreMax) {
+        otherScoreSum += scoreMax;
         scoreMax = subScore;
+      } else {
+        otherScoreSum += subScore;
       }
     }
-    return (float) (scoreMax + (scoreSum - scoreMax) * tieBreakerMultiplier); 
+    return (float) (scoreMax + otherScoreSum * tieBreakerMultiplier);
   }
 
   @Override
   public float maxScore() {
-    // TODO: implement but be careful about floating-point errors.
-    return Float.POSITIVE_INFINITY;
+    return maxScore;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
index 729a298..7e22991 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
@@ -20,21 +20,36 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.lucene.util.MathUtil;
+
 /** A Scorer for OR like queries, counterpart of <code>ConjunctionScorer</code>.
  */
 final class DisjunctionSumScorer extends DisjunctionScorer {
-  
+
+  private final float maxScore;
+
   /** Construct a <code>DisjunctionScorer</code>.
    * @param weight The weight to be used.
    * @param subScorers Array of at least two subscorers.
    */
   DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, boolean needsScores) {
     super(weight, subScorers, needsScores);
+    double maxScore = 0;
+    for (Scorer scorer : subScorers) {
+      maxScore += scorer.maxScore();
+    }
+    // The error of sums depends on the order in which values are summed up. In
+    // order to avoid this issue, we compute an upper bound of the value that
+    // the sum may take. If the max relative error is b, then it means that two
+    // sums are always within 2*b of each other.
+    double maxScoreRelativeErrorBound = MathUtil.sumRelativeErrorBound(subScorers.size());
+    this.maxScore = (float) ((1.0 + 2 * maxScoreRelativeErrorBound) * maxScore);
   }
 
   @Override
   protected float score(DisiWrapper topList) throws IOException {
     double score = 0;
+
     for (DisiWrapper w = topList; w != null; w = w.next) {
       score += w.scorer.score();
     }
@@ -43,8 +58,7 @@ final class DisjunctionSumScorer extends DisjunctionScorer {
 
   @Override
   public float maxScore() {
-    // TODO: implement it but be careful with floating-point errors
-    return Float.POSITIVE_INFINITY;
+    return maxScore;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
index 2f3b600..f5f647e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
@@ -26,6 +26,8 @@ import java.util.Collection;
 import java.util.List;
 import java.util.OptionalInt;
 
+import org.apache.lucene.util.MathUtil;
+
 /**
  * This implements the WAND (Weak AND) algorithm for dynamic pruning
  * described in "Efficient Query Evaluation using a Two-Level Retrieval
@@ -120,6 +122,7 @@ final class WANDScorer extends Scorer {
   int tailSize;
 
   final long cost;
+  final float maxScore;
 
   WANDScorer(Weight weight, Collection<Scorer> scorers) {
     super(weight);
@@ -142,10 +145,12 @@ final class WANDScorer extends Scorer {
     // Use a scaling factor of 0 if all max scores are either 0 or +Infty
     this.scalingFactor = scalingFactor.orElse(0);
     
+    double maxScoreSum = 0;
     for (Scorer scorer : scorers) {
       DisiWrapper w = new DisiWrapper(scorer);
       float maxScore = scorer.maxScore();
       w.maxScore = scaleMaxScore(maxScore, this.scalingFactor);
+      maxScoreSum += maxScore;
       addLead(w);
     }
 
@@ -154,6 +159,12 @@ final class WANDScorer extends Scorer {
       cost += w.cost;
     }
     this.cost = cost;
+    // The error of sums depends on the order in which values are summed up. In
+    // order to avoid this issue, we compute an upper bound of the value that
+    // the sum may take. If the max relative error is b, then it means that two
+    // sums are always within 2*b of each other.
+    double maxScoreRelativeErrorBound = MathUtil.sumRelativeErrorBound(scorers.size());
+    this.maxScore = (float) ((1.0 + 2 * maxScoreRelativeErrorBound) * maxScoreSum);
   }
 
   // returns a boolean so that it can be called from assert
@@ -375,8 +386,7 @@ final class WANDScorer extends Scorer {
 
   @Override
   public float maxScore() {
-    // TODO: implement but be careful about floating-point errors.
-    return Float.POSITIVE_INFINITY;
+    return maxScore;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/core/src/java/org/apache/lucene/util/MathUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/MathUtil.java b/lucene/core/src/java/org/apache/lucene/util/MathUtil.java
index 09437fe..7430c5d 100644
--- a/lucene/core/src/java/org/apache/lucene/util/MathUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/MathUtil.java
@@ -149,5 +149,20 @@ public final class MathUtil {
     return mult * Math.log((1.0d + a) / (1.0d - a));
   }
 
+  /**
+   * Return a relative error bound for a sum of {@code numValues} positive doubles,
+   * computed using recursive summation, ie. sum = x1 + ... + xn.
+   * NOTE: This only works if all values are POSITIVE so that Σ |xi| == |Σ xi|.
+   * This uses formula 3.5 from Higham, Nicholas J. (1993),
+   * "The accuracy of floating point summation", SIAM Journal on Scientific Computing.
+   */
+  public static double sumRelativeErrorBound(int numValues) {
+    if (numValues <= 1) {
+      return 0;
+    }
+    // u = unit roundoff in the paper, also called machine precision or machine epsilon
+    double u = Math.scalb(1.0, -52);
+    return (numValues - 1) * u;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/DisjunctionMaxQuery.xml
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/DisjunctionMaxQuery.xml b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/DisjunctionMaxQuery.xml
index ebf1400..0c94b00 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/DisjunctionMaxQuery.xml
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/DisjunctionMaxQuery.xml
@@ -18,7 +18,7 @@
 
 <DisjunctionMaxQuery>
   <TermQuery fieldName="a">merger</TermQuery>
-  <DisjunctionMaxQuery tieBreaker="1.2">
+  <DisjunctionMaxQuery tieBreaker="0.3">
     <TermQuery fieldName="b">verger</TermQuery>
   </DisjunctionMaxQuery>
-</DisjunctionMaxQuery>
\ No newline at end of file
+</DisjunctionMaxQuery>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3f90385/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
index d97e2f6..b9e44c1 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
@@ -102,7 +102,7 @@ public class TestCoreParser extends LuceneTestCase {
     assertEquals(0.0f, d.getTieBreakerMultiplier(), 0.0001f);
     assertEquals(2, d.getDisjuncts().size());
     DisjunctionMaxQuery ndq = (DisjunctionMaxQuery) d.getDisjuncts().get(1);
-    assertEquals(1.2f, ndq.getTieBreakerMultiplier(), 0.0001f);
+    assertEquals(0.3f, ndq.getTieBreakerMultiplier(), 0.0001f);
     assertEquals(1, ndq.getDisjuncts().size());
   }
 


[27/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8099: Fix xmlqueryparser tests

Posted by da...@apache.org.
LUCENE-8099: Fix xmlqueryparser tests


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

Branch: refs/heads/jira/solr-11702
Commit: dfaf023d4a97f4356fc256a94951443b1893876f
Parents: 48c8947
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Dec 18 18:40:27 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Dec 18 18:40:27 2017 +0000

----------------------------------------------------------------------
 .../lucene/queryparser/xml/TestCorePlusQueriesParser.java       | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfaf023d/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCorePlusQueriesParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCorePlusQueriesParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCorePlusQueriesParser.java
index a91800f..0288d2c 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCorePlusQueriesParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCorePlusQueriesParser.java
@@ -31,9 +31,4 @@ public class TestCorePlusQueriesParser extends TestCoreParser {
     dumpResults("like this", q, 5);
   }
 
-  public void testBoostingQueryXML() throws Exception {
-    Query q = parse("BoostingQuery.xml");
-    dumpResults("boosting ", q, 5);
-  }
-
 }


[29/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11681: Add ttest and pairedTtest Stream Evaluators

Posted by da...@apache.org.
SOLR-11681: Add ttest and pairedTtest Stream Evaluators


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

Branch: refs/heads/jira/solr-11702
Commit: f8c93db9eb3626877a1a9c52c197ed0f0911ab73
Parents: 53a7f5b
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Dec 18 14:31:08 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Dec 18 14:31:47 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  2 +
 .../solrj/io/eval/PairedTTestEvaluator.java     | 75 ++++++++++++++++
 .../client/solrj/io/eval/TTestEvaluator.java    | 95 ++++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 43 ++++++++-
 4 files changed, 213 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8c93db9/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index e3491f6..8a83160 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -281,6 +281,8 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
         .withFunctionName("olsRegress", OLSRegressionEvaluator.class)
         .withFunctionName("derivative", DerivativeEvaluator.class)
         .withFunctionName("spline", SplineEvaluator.class)
+        .withFunctionName("ttest", TTestEvaluator.class)
+        .withFunctionName("pairedTtest", PairedTTestEvaluator.class)
 
 
         // Boolean Stream Evaluators

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8c93db9/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PairedTTestEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PairedTTestEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PairedTTestEvaluator.java
new file mode 100644
index 0000000..56c2dc9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PairedTTestEvaluator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.commons.math3.stat.inference.TTest;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class PairedTTestEvaluator extends RecursiveNumericListEvaluator implements TwoValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public PairedTTestEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(containedEvaluators.size() != 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two parameters but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object value1, Object value2) throws IOException {
+
+    TTest tTest = new TTest();
+    Map map = new HashMap();
+    Tuple tuple = new Tuple(map);
+    if(value1 instanceof List) {
+      List<Number> values1 = (List<Number>)value1;
+      double[] samples1 = new double[values1.size()];
+
+      for(int i=0; i< samples1.length; i++) {
+        samples1[i] = values1.get(i).doubleValue();
+      }
+
+      if(value2 instanceof List) {
+        List<Number> values2 = (List<Number>) value2;
+        double[] samples2 = new double[values2.size()];
+
+        for (int i = 0; i < samples2.length; i++) {
+          samples2[i] = values2.get(i).doubleValue();
+        }
+
+        double tstat = tTest.pairedT(samples1, samples2);
+        double pval = tTest.pairedTTest(samples1, samples2);
+        tuple.put("t-statistic", tstat);
+        tuple.put("p-value", pval);
+        return tuple;
+      } else {
+        throw new IOException("Second parameter for pairedTtest must be a double array");
+      }
+    } else {
+      throw new IOException("First parameter for pairedTtest must be a double array");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8c93db9/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TTestEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TTestEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TTestEvaluator.java
new file mode 100644
index 0000000..6273376
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TTestEvaluator.java
@@ -0,0 +1,95 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.commons.math3.stat.inference.TTest;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class TTestEvaluator extends RecursiveNumericEvaluator implements TwoValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public TTestEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(containedEvaluators.size() != 2){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two parameters but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object value1, Object value2) throws IOException {
+
+    TTest tTest = new TTest();
+    Map map = new HashMap();
+    Tuple tuple = new Tuple(map);
+    if(value1 instanceof Number) {
+      double mean = ((Number) value1).doubleValue();
+
+      if(value2 instanceof List) {
+        List<Number> values = (List<Number>) value2;
+        double[] samples = new double[values.size()];
+        for (int i = 0; i < samples.length; i++) {
+          samples[i] = values.get(i).doubleValue();
+        }
+
+        double tstat = tTest.t(mean, samples);
+        double pval = tTest.tTest(mean, samples);
+
+        tuple.put("t-statistic", tstat);
+        tuple.put("p-value", pval);
+        return tuple;
+      } else {
+        throw new IOException("Second parameter for ttest must be a double array");
+      }
+    } else if(value1 instanceof List) {
+      List<Number> values1 = (List<Number>)value1;
+
+      double[] samples1 = new double[values1.size()];
+
+      for(int i=0; i< samples1.length; i++) {
+        samples1[i] = values1.get(i).doubleValue();
+      }
+
+      if(value2 instanceof List) {
+        List<Number> values2 = (List<Number>) value2;
+        double[] samples2 = new double[values2.size()];
+
+        for (int i = 0; i < samples2.length; i++) {
+          samples2[i] = values2.get(i).doubleValue();
+        }
+
+        double tstat = tTest.t(samples1, samples2);
+        double pval = tTest.tTest(samples1, samples2);
+        tuple.put("t-statistic", tstat);
+        tuple.put("p-value", pval);
+        return tuple;
+      } else {
+        throw new IOException("Second parameter for ttest must be a double array");
+      }
+    } else {
+      throw new IOException("First parameter for ttest must be either a double our double array");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f8c93db9/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 4b368c1..c8fe2ff 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -6796,12 +6796,12 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     List<Map> listh = (List<Map>)tuples.get(0).get("h");
     Map maph = listh.get(0);
     double pcth = (double)maph.get("pct");
-    assertEquals(pcth, .5, .02 );
+    assertEquals(pcth, .5, .02);
 
     List<Map> listi = (List<Map>)tuples.get(0).get("i");
     Map mapi = listi.get(0);
     double pcti = (double)mapi.get("pct");
-    assertEquals(pcti, .8, .02 );
+    assertEquals(pcti, .8, .02);
   }
 
   @Test
@@ -7114,6 +7114,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     assertEquals(out.get(6).doubleValue(),21.0,0.01);
   }
 
+
   @Test
   public void testPolyfit() throws Exception {
     String cexpr = "let(echo=true," +
@@ -7142,6 +7143,44 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
 
   @Test
+  public void testTtest() throws Exception {
+    String cexpr = "let(echo=true," +
+                       "a=array(0,1,2,3,4,5,6,7,9,10,11,12), " +
+                       "b=array(0,1,2,3,4,5,6,7,1,1,1,1), " +
+                       "ttest=ttest(a, b)," +
+                       "sample2Mean=mean(b),"+
+                       "onesamplettest=ttest(sample2Mean, b)," +
+                       "pairedttest=pairedTtest(a,b))";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    Map testResult = (Map)tuples.get(0).get("ttest");
+    Number tstat = (Number)testResult.get("t-statistic");
+    Number pval = (Number)testResult.get("p-value");
+    assertEquals(tstat.doubleValue(), 2.3666107120397575, .0001);
+    assertEquals(pval.doubleValue(), 0.029680704317867967, .0001);
+
+    Map testResult2 = (Map)tuples.get(0).get("onesamplettest");
+    Number tstat2 = (Number)testResult2.get("t-statistic");
+    Number pval2 = (Number)testResult2.get("p-value");
+    assertEquals(tstat2.doubleValue(), 0, .0001);
+    assertEquals(pval2.doubleValue(), 1, .0001);
+
+    Map testResult3 = (Map)tuples.get(0).get("pairedttest");
+    Number tstat3 = (Number)testResult3.get("t-statistic");
+    Number pval3 = (Number)testResult3.get("p-value");
+    assertEquals(tstat3.doubleValue(), 2.321219442769799, .0001);
+    assertEquals(pval3.doubleValue(), 0.0404907407662755, .0001);
+  }
+
+
+  @Test
   public void testLoess() throws Exception {
     String cexpr = "let(echo=true," +
                    "    a=array(0,1,2,3,4,5,6,7)," +


[26/54] [abbrv] lucene-solr:jira/solr-11702: WIP

Posted by da...@apache.org.
WIP


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

Branch: refs/heads/jira/solr-11702
Commit: 48c8947c1775c7695fdd689049fc0ef0e487ebc7
Parents: 33a64af
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Dec 18 18:33:01 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Dec 18 18:33:01 2017 +0000

----------------------------------------------------------------------
 .../java/org/apache/lucene/search/Query.java    |  4 +--
 .../apache/lucene/search/RewriteContext.java    | 32 ++++++++++++++++++++
 .../org/apache/lucene/search/TermQuery.java     | 29 +++++++-----------
 3 files changed, 44 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48c8947c/lucene/core/src/java/org/apache/lucene/search/Query.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Query.java b/lucene/core/src/java/org/apache/lucene/search/Query.java
index 54de63f..7a3fc0d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Query.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Query.java
@@ -70,9 +70,7 @@ public abstract class Query {
    * a PrefixQuery will be rewritten into a BooleanQuery that consists
    * of TermQuerys.
    */
-  public Query rewrite(IndexReader reader) throws IOException {
-    return this;
-  }
+  public abstract Query rewrite(IndexReader reader, RewriteContext rewriteContext) throws IOException;
 
   /**
    * Override and implement query instance equivalence properly in a subclass. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48c8947c/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java b/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java
new file mode 100644
index 0000000..8738399
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+
+public class RewriteContext {
+
+  public TermContext buildTermContext(Term term, IndexReader reader) throws IOException {
+    return TermContext.build(reader.getContext(), term);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48c8947c/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index 925fe93..b13a9f4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -22,6 +22,7 @@ import java.util.Objects;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -203,25 +204,17 @@ public class TermQuery extends Query {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    final IndexReaderContext context = searcher.getTopReaderContext();
-    final TermContext termState;
-    if (perReaderTermState == null
-        || perReaderTermState.wasBuiltFor(context) == false) {
-      if (scoreMode.needsScores()) {
-        // make TermQuery single-pass if we don't have a PRTS or if the context
-        // differs!
-        termState = TermContext.build(context, term);
-      } else {
-        // do not compute the term state, this will help save seeks in the terms
-        // dict on segments that have a cache entry for this query
-        termState = null;
-      }
-    } else {
-      // PRTS was pre-build for this IS
-      termState = this.perReaderTermState;
-    }
+    if (perReaderTermState == null)
+      throw new IllegalStateException("TermQuery must be rewritten before use");
+    if (perReaderTermState.wasBuiltFor(searcher.getTopReaderContext()) == false)
+      throw new IllegalStateException("TermQuery was built against a different IndexReader");
+    return new TermWeight(searcher, scoreMode.needsScores(), boost, perReaderTermState);
+  }
 
-    return new TermWeight(searcher, scoreMode.needsScores(), boost, termState);
+  @Override
+  public Query rewrite(IndexReader reader, RewriteContext rewriteContext) throws IOException {
+    TermContext tc = rewriteContext.buildTermContext(term, reader);
+    return new TermQuery(term, tc);
   }
 
   /** Prints a user-readable version of this query. */


[24/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8099: Deprecate CustomScoreQuery, BoostedQuery, BoostingQuery

Posted by da...@apache.org.
LUCENE-8099: Deprecate CustomScoreQuery, BoostedQuery, BoostingQuery


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

Branch: refs/heads/jira/solr-11702
Commit: b01e6023e1cd3c62260b38c05c8d145ba143a2ac
Parents: e441a99
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Dec 14 12:09:51 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Dec 18 17:47:03 2017 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../org/apache/lucene/search/BoostQuery.java    |   3 +
 .../expressions/ExpressionValueSource.java      |   6 +-
 .../highlight/WeightedSpanTermExtractor.java    |   5 +-
 .../uhighlight/MultiTermHighlighting.java       |   6 +-
 .../search/vectorhighlight/FieldQuery.java      |   8 +-
 .../search/highlight/HighlighterTest.java       |  24 +++
 .../FastVectorHighlighterTest.java              |  32 ++++
 lucene/queries/build.xml                        |   8 +
 .../apache/lucene/queries/BoostingQuery.java    |  12 ++
 .../apache/lucene/queries/CustomScoreQuery.java |  12 ++
 .../lucene/queries/function/BoostedQuery.java   |  15 +-
 .../queries/function/FunctionScoreQuery.java    |   7 +
 .../function/TestFunctionScoreQuery.java        | 145 +++++--------------
 .../apache/solr/search/BoostQParserPlugin.java  |  23 ++-
 .../solr/search/ExtendedDismaxQParser.java      |   5 +-
 .../apache/solr/search/ValueSourceParser.java   |   4 +-
 17 files changed, 194 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index db8aaab..ac70c4e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,10 @@ API Changes
 * LUCENE-8051: LevensteinDistance renamed to LevenshteinDistance.
   (Pulak Ghosh via Adrien Grand)
 
+* LUCENE-8099: Deprecate CustomScoreQuery, BoostedQuery and BoostingQuery.
+  Users should instead use FunctionScoreQuery, possibly combined with
+  a lucene expression (Alan Woodward)
+
 New Features
 
 * LUCENE-2899: Add new module analysis/opennlp, with analysis components

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/core/src/java/org/apache/lucene/search/BoostQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BoostQuery.java b/lucene/core/src/java/org/apache/lucene/search/BoostQuery.java
index 2c2cb78..4e4649c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BoostQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BoostQuery.java
@@ -27,6 +27,9 @@ import org.apache.lucene.index.IndexReader;
  * Boost values that are less than one will give less importance to this
  * query compared to other ones while values that are greater than one will
  * give more importance to the scores returned by this query.
+ *
+ * More complex boosts can be applied by using FunctionScoreQuery in the
+ * lucene-queries module
  */
 public final class BoostQuery extends Query {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
index 7b30b11..2bfafab 100644
--- a/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
+++ b/lucene/expressions/src/java/org/apache/lucene/expressions/ExpressionValueSource.java
@@ -107,7 +107,7 @@ final class ExpressionValueSource extends DoubleValuesSource {
     final int prime = 31;
     int result = 1;
     result = prime * result
-        + ((expression == null) ? 0 : expression.hashCode());
+        + ((expression == null) ? 0 : expression.sourceText.hashCode());
     result = prime * result + (needsScores ? 1231 : 1237);
     result = prime * result + Arrays.hashCode(variables);
     return result;
@@ -129,7 +129,7 @@ final class ExpressionValueSource extends DoubleValuesSource {
       if (other.expression != null) {
         return false;
       }
-    } else if (!expression.equals(other.expression)) {
+    } else if (!expression.sourceText.equals(other.expression.sourceText)) {
       return false;
     }
     if (needsScores != other.needsScores) {
@@ -178,7 +178,7 @@ final class ExpressionValueSource extends DoubleValuesSource {
       changed |= (rewritten[i] == variables[i]);
     }
     if (changed) {
-      return new ExpressionValueSource(variables, expression, needsScores);
+      return new ExpressionValueSource(rewritten, expression, needsScores);
     }
     return this;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
index a05e9c6..8e6bf24 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
@@ -40,6 +40,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.queries.CommonTermsQuery;
 import org.apache.lucene.queries.CustomScoreQuery;
+import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
@@ -211,8 +212,10 @@ public class WeightedSpanTermExtractor {
       }
     } else if (query instanceof MatchAllDocsQuery) {
       //nothing
-    } else if (query instanceof CustomScoreQuery){
+    } else if (query instanceof CustomScoreQuery) {
       extract(((CustomScoreQuery) query).getSubQuery(), boost, terms);
+    } else if (query instanceof FunctionScoreQuery) {
+      extract(((FunctionScoreQuery) query).getWrappedQuery(), boost, terms);
     } else if (isQueryUnsupported(query.getClass())) {
       // nothing
     } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
index e745d78..57d5afa 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.function.Function;
 import java.util.function.Predicate;
 
+import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -80,8 +81,11 @@ class MultiTermHighlighting {
       list.addAll(Arrays.asList(extractAutomata(((ConstantScoreQuery) query).getQuery(), fieldMatcher, lookInSpan,
           preRewriteFunc)));
     } else if (query instanceof BoostQuery) {
-      list.addAll(Arrays.asList(extractAutomata(((BoostQuery)query).getQuery(), fieldMatcher, lookInSpan,
+      list.addAll(Arrays.asList(extractAutomata(((BoostQuery) query).getQuery(), fieldMatcher, lookInSpan,
           preRewriteFunc)));
+    } else if (query instanceof FunctionScoreQuery) {
+      list.addAll(Arrays.asList(extractAutomata(((FunctionScoreQuery) query).getWrappedQuery(), fieldMatcher,
+          lookInSpan, preRewriteFunc)));
     } else if (query instanceof DisjunctionMaxQuery) {
       for (Query sub : ((DisjunctionMaxQuery) query).getDisjuncts()) {
         list.addAll(Arrays.asList(extractAutomata(sub, fieldMatcher, lookInSpan, preRewriteFunc)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
index 0fd336a..6daa791 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
@@ -28,6 +28,7 @@ import java.util.Set;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.CustomScoreQuery;
+import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
@@ -143,7 +144,12 @@ public class FieldQuery {
     } else if (sourceQuery instanceof CustomScoreQuery) {
       final Query q = ((CustomScoreQuery) sourceQuery).getSubQuery();
       if (q != null) {
-        flatten( q, reader, flatQueries, boost);
+        flatten(q, reader, flatQueries, boost);
+      }
+    } else if (sourceQuery instanceof FunctionScoreQuery) {
+      final Query q = ((FunctionScoreQuery)sourceQuery).getWrappedQuery();
+      if (q != null) {
+        flatten(q, reader, flatQueries, boost);
       }
     } else if (sourceQuery instanceof ToParentBlockJoinQuery) {
       Query childQuery = ((ToParentBlockJoinQuery) sourceQuery).getChildQuery();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
index d03432e..cf14e27 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
@@ -59,10 +59,12 @@ import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.CommonTermsQuery;
 import org.apache.lucene.queries.CustomScoreQuery;
+import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.queries.payloads.SpanPayloadCheckQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.FuzzyQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MultiPhraseQuery;
@@ -166,6 +168,28 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
 
   }
 
+  public void testFunctionScoreQuery() throws Exception {
+    TermQuery termQuery = new TermQuery(new Term(FIELD_NAME, "very"));
+    FunctionScoreQuery query = new FunctionScoreQuery(termQuery, DoubleValuesSource.constant(1));
+
+    searcher = newSearcher(reader);
+    TopDocs hits = searcher.search(query, 10, new Sort(SortField.FIELD_DOC, SortField.FIELD_SCORE));
+    assertEquals(2, hits.totalHits);
+    QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
+    Highlighter highlighter = new Highlighter(scorer);
+
+    final int docId0 = hits.scoreDocs[0].doc;
+    Document doc = searcher.doc(docId0);
+    String storedField = doc.get(FIELD_NAME);
+
+    TokenStream stream = getAnyTokenStream(FIELD_NAME, docId0);
+    Fragmenter fragmenter = new SimpleSpanFragmenter(scorer);
+    highlighter.setTextFragmenter(fragmenter);
+    String fragment = highlighter.getBestFragment(stream, storedField);
+    assertEquals("Hello this is a piece of text that is <B>very</B> long and contains too much preamble and the meat is really here which says kennedy has been shot", fragment);
+
+  }
+
   public void testQueryScorerHits() throws Exception {
     PhraseQuery phraseQuery = new PhraseQuery(FIELD_NAME, "very", "long");
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
index 89f46af..732f62f 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
@@ -39,10 +39,12 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.CommonTermsQuery;
 import org.apache.lucene.queries.CustomScoreQuery;
+import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
@@ -119,6 +121,36 @@ public class FastVectorHighlighterTest extends LuceneTestCase {
     writer.close();
     dir.close();
   }
+
+  public void testFunctionScoreQueryHighlight() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    Document doc = new Document();
+    FieldType type = new FieldType(TextField.TYPE_STORED);
+    type.setStoreTermVectorOffsets(true);
+    type.setStoreTermVectorPositions(true);
+    type.setStoreTermVectors(true);
+    type.freeze();
+    Field field = new Field("field", "This is a test where foo is highlighed and should be highlighted", type);
+
+    doc.add(field);
+    writer.addDocument(doc);
+    FastVectorHighlighter highlighter = new FastVectorHighlighter();
+
+    IndexReader reader = DirectoryReader.open(writer);
+    int docId = 0;
+    FieldQuery fieldQuery  = highlighter.getFieldQuery( new FunctionScoreQuery(new TermQuery(new Term("field", "foo")), DoubleValuesSource.constant(1)), reader );
+    String[] bestFragments = highlighter.getBestFragments(fieldQuery, reader, docId, "field", 54, 1);
+    // highlighted results are centered
+    assertEquals("This is a test where <b>foo</b> is highlighed and should be highlighted", bestFragments[0]);
+    bestFragments = highlighter.getBestFragments(fieldQuery, reader, docId, "field", 52, 1);
+    assertEquals("This is a test where <b>foo</b> is highlighed and should be", bestFragments[0]);
+    bestFragments = highlighter.getBestFragments(fieldQuery, reader, docId, "field", 30, 1);
+    assertEquals("a test where <b>foo</b> is highlighed", bestFragments[0]);
+    reader.close();
+    writer.close();
+    dir.close();
+  }
   
   public void testPhraseHighlightLongTextTest() throws IOException {
     Directory dir = newDirectory();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/queries/build.xml
----------------------------------------------------------------------
diff --git a/lucene/queries/build.xml b/lucene/queries/build.xml
index 5edb64d..20f9c4f 100644
--- a/lucene/queries/build.xml
+++ b/lucene/queries/build.xml
@@ -21,4 +21,12 @@
   </description>
 
   <import file="../module-build.xml"/>
+
+  <path id="test.classpath">
+    <pathelement path="${expressions.jar}"/>
+    <fileset dir="../expressions/lib"/>
+    <path refid="test.base.classpath"/>
+  </path>
+
+  <target name="compile-core" depends="jar-expressions,common.compile-core" />
 </project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
index dfebd98..452cbe6 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
@@ -48,7 +48,19 @@ import org.apache.lucene.search.Weight;
  * This code was originally made available here: 
  *   <a href="http://marc.theaimsgroup.com/?l=lucene-user&amp;m=108058407130459&amp;w=2">http://marc.theaimsgroup.com/?l=lucene-user&amp;m=108058407130459&amp;w=2</a>
  * and is documented here: http://wiki.apache.org/lucene-java/CommunityContributions
+ *
+ * Clients should instead use FunctionScoreQuery and the lucene-expressions library:
+ * <pre>
+ *   SimpleBindings bindings = new SimpleBindings();
+ *   bindings.add("score", DoubleValuesSource.SCORES);
+ *   bindings.add("context", DoubleValuesSource.fromQuery(new ConstantScoreQuery(myContextQuery, boost)));
+ *   Expression expr = JavascriptCompiler.compile("score * context");
+ *   FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));
+ * </pre>
+ *
+ * @deprecated Use {@link org.apache.lucene.queries.function.FunctionScoreQuery}
  */
+@Deprecated
 public class BoostingQuery extends Query {
     private final float boost;                            // the amount to boost by
     private final Query match;                            // query to match

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
index 62a1787..f82cf50 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
@@ -43,7 +43,19 @@ import org.apache.lucene.search.Weight;
  * Subclasses can modify the computation by overriding {@link #getCustomScoreProvider}.
  * 
  * @lucene.experimental
+ *
+ * Clients should instead use FunctionScoreQuery and the lucene-expressions library
+ * <pre>
+ *   SimpleBindings bindings = new SimpleBindings();
+ *   bindings.add("score", DoubleValuesSource.SCORES);
+ *   bindings.add("boost", DoubleValuesSource.fromIntField("myboostfield"));
+ *   Expression expr = JavascriptCompiler.compile("score * boost");
+ *   FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));
+ * </pre>
+ *
+ * @deprecated use {@link org.apache.lucene.queries.function.FunctionScoreQuery}
  */
+@Deprecated
 public class CustomScoreQuery extends Query implements Cloneable {
 
   private Query subQuery;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
index df3ab47..ddb261a 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
@@ -35,9 +35,20 @@ import org.apache.lucene.search.Weight;
 
 /**
  * Query that is boosted by a ValueSource
+ *
+ * Instead of using this query, clients can use a {@link FunctionScoreQuery} and the
+ * lucene-expressions library:
+ * <pre>
+ *   SimpleBindings bindings = new SimpleBindings();
+ *   bindings.add("score", DoubleValuesSource.SCORES);
+ *   bindings.add("boost", DoubleValuesSource.fromIntField("myboostfield"));
+ *   Expression expr = JavascriptCompiler.compile("score * boost");
+ *   FunctionScoreQuery q = new FunctionScoreQuery(inputQuery, expr.getDoubleValuesSource(bindings));
+ * </pre>
+ *
+ * @deprecated Use {@link FunctionScoreQuery}
  */
-// TODO: BoostedQuery and BoostingQuery in the same module? 
-// something has to give
+@Deprecated
 public final class BoostedQuery extends Query {
   private final Query q;
   private final ValueSource boostVal; // optional, can be null

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
index 649ba6e..d264267 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java
@@ -56,6 +56,13 @@ public final class FunctionScoreQuery extends Query {
     this.source = source;
   }
 
+  /**
+   * @return the wrapped Query
+   */
+  public Query getWrappedQuery() {
+    return in;
+  }
+
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
     Weight inner = in.createWeight(searcher, scoreMode.needsScores() && source.needsScores() ? scoreMode : ScoreMode.COMPLETE_NO_SCORES, 1f);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
index d70d5f3..301dce7 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
@@ -18,20 +18,19 @@
 package org.apache.lucene.queries.function;
 
 import java.io.IOException;
-import java.util.function.DoubleUnaryOperator;
-import java.util.function.ToDoubleBiFunction;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.SimpleBindings;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
-import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
@@ -81,8 +80,10 @@ public class TestFunctionScoreQuery extends FunctionTestSetup {
   // CustomScoreQuery and BoostedQuery equivalent
   public void testScoreModifyingSource() throws Exception {
 
-    DoubleValuesSource iii = DoubleValuesSource.fromIntField("iii");
-    DoubleValuesSource score = scoringFunction(iii, (v, s) -> v * s);
+    SimpleBindings bindings = new SimpleBindings();
+    bindings.add("score", DoubleValuesSource.SCORES);
+    bindings.add("iii", DoubleValuesSource.fromIntField("iii"));
+    Expression expr = JavascriptCompiler.compile("score * iii");
 
     BooleanQuery bq = new BooleanQuery.Builder()
         .add(new TermQuery(new Term(TEXT_FIELD, "first")), BooleanClause.Occur.SHOULD)
@@ -90,7 +91,7 @@ public class TestFunctionScoreQuery extends FunctionTestSetup {
         .build();
     TopDocs plain = searcher.search(bq, 1);
 
-    FunctionScoreQuery fq = new FunctionScoreQuery(bq, score);
+    FunctionScoreQuery fq = new FunctionScoreQuery(bq, expr.getDoubleValuesSource(bindings));
 
     QueryUtils.check(random(), fq, searcher, rarely());
 
@@ -104,12 +105,38 @@ public class TestFunctionScoreQuery extends FunctionTestSetup {
 
   }
 
+  // BoostingQuery equivalent
+  public void testCombiningMultipleQueryScores() throws Exception {
+
+    SimpleBindings bindings = new SimpleBindings();
+    bindings.add("score", DoubleValuesSource.SCORES);
+    bindings.add("testquery", DoubleValuesSource.fromQuery(new TermQuery(new Term(TEXT_FIELD, "rechecking"))));
+    Expression expr = JavascriptCompiler.compile("score + (testquery * 100)");
+
+    TermQuery q = new TermQuery(new Term(TEXT_FIELD, "text"));
+    TopDocs plain = searcher.search(q, 1);
+
+    FunctionScoreQuery fq = new FunctionScoreQuery(q, expr.getDoubleValuesSource(bindings));
+
+    QueryUtils.check(random(), fq, searcher, rarely());
+
+    int[] expectedDocs = new int[]{  6, 1, 0, 2, 8 };
+    TopDocs docs = searcher.search(fq, 5);
+    assertEquals(plain.totalHits, docs.totalHits);
+    for (int i = 0; i < expectedDocs.length; i++) {
+      assertEquals(expectedDocs[i], docs.scoreDocs[i].doc);
+
+    }
+  }
+
   // check boosts with non-distributive score source
   public void testBoostsAreAppliedLast() throws Exception {
 
-    DoubleValuesSource scores = function(DoubleValuesSource.SCORES, v -> Math.log(v + 4));
+    SimpleBindings bindings = new SimpleBindings();
+    bindings.add("score", DoubleValuesSource.SCORES);
+    Expression expr = JavascriptCompiler.compile("ln(score + 4)");
 
-    Query q1 = new FunctionScoreQuery(new TermQuery(new Term(TEXT_FIELD, "text")), scores);
+    Query q1 = new FunctionScoreQuery(new TermQuery(new Term(TEXT_FIELD, "text")), expr.getDoubleValuesSource(bindings));
     TopDocs plain = searcher.search(q1, 5);
 
     Query boosted = new BoostQuery(q1, 2);
@@ -122,106 +149,6 @@ public class TestFunctionScoreQuery extends FunctionTestSetup {
 
   }
 
-  public static DoubleValuesSource function(DoubleValuesSource in, DoubleUnaryOperator function) {
-    return new DoubleValuesSource() {
-      @Override
-      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
-        DoubleValues v = in.getValues(ctx, scores);
-        return new DoubleValues() {
-          @Override
-          public double doubleValue() throws IOException {
-            return function.applyAsDouble(v.doubleValue());
-          }
-
-          @Override
-          public boolean advanceExact(int doc) throws IOException {
-            return v.advanceExact(doc);
-          }
-        };
-      }
-
-      @Override
-      public boolean needsScores() {
-        return in.needsScores();
-      }
-
-      @Override
-      public boolean isCacheable(LeafReaderContext ctx) {
-        return in.isCacheable(ctx);
-      }
-
-      @Override
-      public DoubleValuesSource rewrite(IndexSearcher searcher) throws IOException {
-        return function(in.rewrite(searcher), function);
-      }
-
-      @Override
-      public int hashCode() {
-        return 0;
-      }
-
-      @Override
-      public boolean equals(Object obj) {
-        return false;
-      }
-
-      @Override
-      public String toString() {
-        return "fn";
-      }
-    };
-  }
-
-  private static DoubleValuesSource scoringFunction(DoubleValuesSource in, ToDoubleBiFunction<Double, Double> function) {
-    return new DoubleValuesSource() {
-      @Override
-      public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
-        DoubleValues v = in.getValues(ctx, scores);
-        return new DoubleValues() {
-          @Override
-          public double doubleValue() throws IOException {
-            return function.applyAsDouble(v.doubleValue(), scores.doubleValue());
-          }
-
-          @Override
-          public boolean advanceExact(int doc) throws IOException {
-            return v.advanceExact(doc);
-          }
-        };
-      }
-
-      @Override
-      public boolean needsScores() {
-        return true;
-      }
-
-      @Override
-      public boolean isCacheable(LeafReaderContext ctx) {
-        return in.isCacheable(ctx);
-      }
-
-      @Override
-      public DoubleValuesSource rewrite(IndexSearcher searcher) throws IOException {
-        return scoringFunction(in.rewrite(searcher), function);
-      }
-
-      @Override
-      public int hashCode() {
-        return 0;
-      }
-
-      @Override
-      public boolean equals(Object obj) {
-        return false;
-      }
-
-      @Override
-      public String toString() {
-        return "fn";
-      }
-    };
-  }
-
   public void testTruncateNegativeScores() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/solr/core/src/java/org/apache/solr/search/BoostQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/BoostQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/BoostQParserPlugin.java
index 5f7add8..7391f66 100644
--- a/solr/core/src/java/org/apache/solr/search/BoostQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/BoostQParserPlugin.java
@@ -16,11 +16,18 @@
  */
 package org.apache.solr.search;
 
-import org.apache.lucene.queries.function.BoostedQuery;
+import java.text.ParseException;
+
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.SimpleBindings;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.queries.function.FunctionQuery;
+import org.apache.lucene.queries.function.FunctionScoreQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.QueryValueSource;
+import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Query;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
@@ -60,7 +67,7 @@ public class BoostQParserPlugin extends QParserPlugin {
         } else {
           vs = new QueryValueSource(bq, 0.0f);
         }
-        return new BoostedQuery(q, vs);
+        return boostQuery(q, vs);
       }
 
 
@@ -84,4 +91,16 @@ public class BoostQParserPlugin extends QParserPlugin {
     };
   }
 
+  public static Query boostQuery(Query input, ValueSource vs) {
+    try {
+      SimpleBindings bindings = new SimpleBindings();
+      bindings.add("score", DoubleValuesSource.SCORES);
+      bindings.add("vs", vs.asDoubleValuesSource());
+      Expression expr = JavascriptCompiler.compile("score * vs");
+      return new FunctionScoreQuery(input, expr.getDoubleValuesSource(bindings));
+    } catch (ParseException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e); // should never happen!
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
index c48d127..5e74f4a 100644
--- a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java
@@ -33,7 +33,6 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.StopFilterFactory;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.BoostedQuery;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.ProductFloatFunction;
@@ -197,9 +196,9 @@ public class ExtendedDismaxQParser extends QParser {
     List<ValueSource> boosts = getMultiplicativeBoosts();
     if (boosts.size()>1) {
       ValueSource prod = new ProductFloatFunction(boosts.toArray(new ValueSource[boosts.size()]));
-      topQuery = new BoostedQuery(topQuery, prod);
+      topQuery = BoostQParserPlugin.boostQuery(topQuery, prod);
     } else if (boosts.size() == 1) {
-      topQuery = new BoostedQuery(topQuery, boosts.get(0));
+      topQuery = BoostQParserPlugin.boostQuery(topQuery, boosts.get(0));
     }
     
     return topQuery;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b01e6023/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index dc6411e..367eb64 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -27,7 +27,6 @@ import java.util.Map;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.queries.function.BoostedQuery;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.BoolDocValues;
@@ -326,8 +325,7 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
       public ValueSource parse(FunctionQParser fp) throws SyntaxError {
         Query q = fp.parseNestedQuery();
         ValueSource vs = fp.parseValueSource();
-        BoostedQuery bq = new BoostedQuery(q, vs);
-        return new QueryValueSource(bq, 0.0f);
+        return new QueryValueSource(BoostQParserPlugin.boostQuery(q, vs), 0.0f);
       }
     });
     addParser("joindf", new ValueSourceParser() {


[15/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Add OpenNLP Analysis capabilities as a module

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/chunks.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/chunks.txt b/lucene/analysis/opennlp/src/tools/test-model-data/chunks.txt
new file mode 100644
index 0000000..f02c5b7
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/chunks.txt
@@ -0,0 +1,3566 @@
+Iran NNP B-NP
+announced VBD B-VP
+tonight NN B-NP
+that IN B-PP
+its NNS B-NP
+major JJ B-NP
+offensive NN I-NP
+against IN B-PP
+Iraq NNP B-NP
+in IN B-PP
+the DT B-NP
+Gulf NNP I-NP
+war NN I-NP
+had VBD B-VP
+ended VBN I-VP
+after IN B-PP
+dealing VBG B-VP
+savage JJ B-NP
+blows NNS I-NP
+against IN B-PP
+the DT B-NP
+Baghdad NNP I-NP
+government NN I-NP
+. . O
+The DT B-NP
+Iranian JJ I-NP
+news NN I-NP
+agency NN I-NP
+IRNA NNP I-NP
+, , O
+in IN B-PP
+a DT B-NP
+report NN I-NP
+received VBN B-VP
+in IN B-PP
+London NNP B-NP
+, , O
+said VBD B-VP
+the DT B-NP
+operation NN I-NP
+code-named VBN B-VP
+Karbala-5 CD B-NP
+launched VBD B-VP
+into IN B-PP
+Iraq NNP B-NP
+on IN B-PP
+January NNP B-NP
+9 CD I-NP
+was VBD B-VP
+now RB B-ADVP
+over RP B-NP
+. . O
+It PRP B-NP
+quoted VBD B-VP
+a DT B-NP
+joint NN I-NP
+statewment NN I-NP
+by IN B-PP
+the DT B-NP
+Iranian JJ I-NP
+Army NNP I-NP
+and CC I-NP
+Revolutionary NNP I-NP
+Guards NNPS I-NP
+Corps NNP I-NP
+as IN B-PP
+saying VBG B-VP
+that IN B-SBAR
+their DT B-NP
+forces NNS I-NP
+had VBD B-VP
+" JJ B-NP
+dealt VBD B-VP
+one CD B-NP
+of IN B-PP
+the DT B-NP
+severest JJS I-NP
+blows NNS I-NP
+on IN B-PP
+the DT B-NP
+Iraqi JJ I-NP
+war NN I-NP
+machine NN I-NP
+in IN B-PP
+the DT B-NP
+history NN I-NP
+of IN B-PP
+the DT B-NP
+Iraq-imposed JJ I-NP
+war NN I-NP
+. . O
+" NN B-VP
+The DT B-NP
+statement NN I-NP
+by IN B-PP
+the DT B-NP
+Iranian JJ I-NP
+High NNP I-NP
+Command NNP I-NP
+appeared VBD B-VP
+to TO I-VP
+herald VB I-VP
+the DT B-NP
+close NN I-NP
+of IN B-PP
+an DT B-NP
+assault NN I-NP
+on IN B-PP
+the DT B-NP
+port JJ I-NP
+city NN I-NP
+of IN B-PP
+Basra NNP B-NP
+in IN B-PP
+southern JJ B-NP
+Iraq NNP I-NP
+. . O
+" NN B-VP
+The DT B-NP
+operation NN I-NP
+was VBD B-VP
+launched VBN I-VP
+at IN B-PP
+a DT B-NP
+time NN I-NP
+when WRB B-ADVP
+the DT B-NP
+Baghdad NNP I-NP
+government NN I-NP
+was VBD B-VP
+spreading VBG I-VP
+extensive JJ B-NP
+propaganda NN I-NP
+on IN B-PP
+the DT B-NP
+resistance NN I-NP
+power NN I-NP
+of IN B-PP
+its NNS B-NP
+army NN I-NP
+... NNS I-NP
+, , O
+" NNS B-NP
+said VBD B-VP
+the DT B-NP
+statement NN I-NP
+quoted VBN B-VP
+by IN B-PP
+IRNA NNP B-NP
+. . O
+It PRP B-NP
+claimed VBD B-VP
+massive JJ B-NP
+victories NNS I-NP
+in IN B-PP
+the DT B-NP
+seven-week NN I-NP
+offensive NN I-NP
+and CC O
+called VBN B-VP
+on IN B-PP
+supporters NNS B-NP
+of IN B-SBAR
+Baghdad NNP B-NP
+to TO B-VP
+" VB I-VP
+come VBN I-VP
+to TO B-PP
+their IN B-NP
+senses JJ I-NP
+" NNS I-NP
+and CC O
+discontinue VB B-VP
+support NN B-NP
+for IN B-PP
+what WP B-NP
+it PRP B-NP
+called VBD B-VP
+the DT B-NP
+tottering VBG I-NP
+regime NN I-NP
+in IN B-PP
+Iraq NNP B-NP
+. . I-NP
+Iran NNP I-NP
+said VBD B-VP
+its NNS B-NP
+forces NNS I-NP
+had VBD B-VP
+" CD B-NP
+liberated JJ I-NP
+" NN I-NP
+155 CD I-NP
+square JJ I-NP
+kilometers NNS I-NP
+of IN B-PP
+enemy-occupied JJ-occupied B-NP
+territory NN I-NP
+during IN B-PP
+the DT B-NP
+1987 CD I-NP
+offensive NN I-NP
+and CC O
+taken VBN B-VP
+over IN B-PP
+islands NNS B-NP
+, , O
+townships NNS B-NP
+, , O
+rivers NNS B-NP
+and CC O
+part NN B-NP
+of IN B-PP
+a DT B-NP
+road NN I-NP
+leading VBG B-VP
+into IN B-PP
+Basra NNP B-NP
+. . O
+The DT B-NP
+Iranian JJ I-NP
+forces NNS I-NP
+" NNS I-NP
+are VBP B-VP
+in IN B-PP
+full JJ B-NP
+control NN I-NP
+of IN B-PP
+these DT B-NP
+areas NNS I-NP
+, , O
+" NNS B-NP
+the DT B-NP
+statement NN I-NP
+said VBD B-VP
+. . O
+It PRP B-NP
+said VBD B-VP
+81 CD B-NP
+Iraqi JJ I-NP
+brigades NNS I-NP
+and CC I-NP
+battalions NNS I-NP
+were VBD B-VP
+totally RB I-VP
+destroyed VBN I-VP
+, , O
+along IN B-ADVP
+with IN B-PP
+700 CD B-NP
+tanks NNS I-NP
+and CC O
+1,500 CD B-NP
+other JJ I-NP
+vehicles NNS I-NP
+. . O
+The DT B-NP
+victory NN I-NP
+list NN I-NP
+also RB B-ADVP
+included VBD B-VP
+80 CD B-NP
+warplanes NNS I-NP
+downed VBD B-VP
+, , O
+250 CD B-NP
+anti- - I-NP
+aircraft NN I-NP
+guns NNS I-NP
+and CC O
+400 CD B-NP
+pieces NNS I-NP
+of IN B-PP
+military JJ B-NP
+hardware NN I-NP
+destroyed VBN B-VP
+and CC O
+the DT B-NP
+seizure NN I-NP
+of IN B-PP
+220 CD B-NP
+tanks NNS I-NP
+and CC O
+armoured JJ B-NP
+personnel NNS I-NP
+carriers NNS I-NP
+. . O
+U.S. NNP O
+bank NN I-NP
+discount NN I-NP
+window RB I-NP
+borrowings NNS I-NP
+less NNS I-NP
+extended VBN B-NP
+credits NN I-NP
+averaged VBD B-VP
+310 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+in IN B-PP
+the DT B-NP
+week NN I-NP
+to TO B-PP
+Wednesday NNP B-NP
+February NNP I-NP
+25 CD I-NP
+, , O
+the DT B-NP
+Federal JJ I-NP
+Reserve NNP I-NP
+said VBD B-VP
+. . O
+The DT B-NP
+Fed JJ I-NP
+said VBD B-VP
+that IN B-SBAR
+overall JJ B-NP
+borrowings NNS I-NP
+in IN B-PP
+the DT B-NP
+week NN I-NP
+fell MD B-VP
+131 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+to TO B-PP
+614 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+, , O
+with IN B-PP
+extended VBN B-NP
+credits NN I-NP
+up IN B-PP
+10 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+at IN B-PP
+304 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+. . O
+The DT B-NP
+week NN I-NP
+was VBD B-VP
+the DT B-NP
+second NN I-NP
+half NN I-NP
+of IN B-PP
+a DT B-NP
+two-week NN I-NP
+statement NN I-NP
+period. NNS I-NP
+Net VBD B-VP
+borrowings NNS B-NP
+in IN B-PP
+the DT B-NP
+prior NN I-NP
+week NN I-NP
+averaged RB B-NP
+451 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+. . O
+Commenting NNP O
+on IN B-PP
+the DT B-NP
+two-week NN I-NP
+statement NN I-NP
+period NNS I-NP
+ended VBD B-VP
+February NNP B-NP
+25 CD I-NP
+, , O
+the DT B-NP
+Fed NNP I-NP
+said VBD B-VP
+that NN B-SBAR
+banks NNS B-NP
+had VBD B-VP
+average JJ B-NP
+net NN I-NP
+free JJ I-NP
+reserves NN I-NP
+of IN B-PP
+644 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+a DT B-NP
+day NN I-NP
+, , O
+down IN B-PP
+from JJ B-NP
+1.34 NN I-NP
+billion NN I-NP
+two RB B-NP
+weeks NNS I-NP
+earlier IN B-ADVP
+. . O
+A RB B-ADJP
+Federal JJ I-ADJP
+Reserve . B-NP
+spokesman NN B-VP
+told VBN I-VP
+a DT B-NP
+press NN I-NP
+briefing VBG B-VP
+that IN B-SBAR
+there EX B-NP
+were VBD B-VP
+no RB B-NP
+large JJ I-NP
+single NN I-NP
+day NN I-NP
+net RB I-NP
+misses NNS I-NP
+in IN B-PP
+the DT B-NP
+Fed's default I-NP
+reserve NN I-NP
+projections NNS I-NP
+in IN B-PP
+the DT B-NP
+week NN I-NP
+to TO B-PP
+Wednesday NNP B-NP
+. . I-NP
+He NNP I-NP
+said VBD B-VP
+that NN B-NP
+natural JJ I-NP
+float NN I-NP
+had VBD B-VP
+been VBN I-VP
+" NN B-NP
+acting VBG B-VP
+a DT B-NP
+bit NN I-NP
+strangely RB B-VP
+" VBN I-VP
+for IN B-PP
+this DT B-NP
+time NN I-NP
+of IN B-PP
+year NN B-NP
+, , O
+noting VBG B-VP
+that IN B-SBAR
+there EX B-NP
+had VBD B-VP
+been VBN I-VP
+poor JJ B-NP
+weather NN I-NP
+during IN B-PP
+the DT B-NP
+latest JJ I-NP
+week NN I-NP
+. . O
+The DT B-NP
+spokesman NN I-NP
+said VBD B-VP
+that IN B-SBAR
+natural JJ B-NP
+float NN I-NP
+ranged VBN B-VP
+from IN B-PP
+under IN B-NP
+500 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+on IN B-PP
+Friday NNP B-NP
+, , O
+for IN B-PP
+which NNP B-NP
+he NN B-NP
+could VBN B-VP
+give JJ B-NP
+no RB I-NP
+reason NN I-NP
+, , O
+to TO B-PP
+nearly JJ B-NP
+one CD I-NP
+billion IN B-PP
+dlrs NN B-NP
+on IN B-PP
+both NN B-NP
+Thursday default B-NP
+and CC O
+Wednesday default B-NP
+. . O
+The DT B-NP
+Fed JJ I-NP
+spokeman NN I-NP
+could VBN B-VP
+give JJ B-NP
+no NN I-NP
+reason NN I-NP
+for IN B-PP
+Thursday's NNP B-NP
+high NN I-NP
+float NNS I-NP
+, , O
+but NNS B-NP
+he DT B-NP
+said VBD B-VP
+that IN B-PP
+about NN B-NP
+750 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+of IN B-PP
+Wednesday's NNP B-NP
+float NN I-NP
+figure NNS I-NP
+was VBD B-VP
+due VBD I-VP
+to TO I-VP
+holdover VB I-VP
+and CC O
+transportation NN B-VP
+float IN B-PRT
+at IN B-PP
+two NN B-NP
+widely WDT I-NP
+separated VBN B-VP
+Fed VBN B-NP
+districts NNS I-NP
+. . O
+For NNP O
+the DT B-NP
+week NN I-NP
+as IN B-PP
+a DT B-NP
+whole NN I-NP
+, , O
+he DT B-NP
+said VBD B-VP
+that IN B-SBAR
+float NN B-NP
+related VBN B-VP
+as IN B-PP
+of NNP B-NP
+adjustments NNS I-NP
+were VBD B-VP
+" RB B-ADJP
+small JJ I-ADJP
+, , O
+" IN B-PP
+adding VBG B-VP
+that IN B-SBAR
+they NN B-NP
+fell NN I-NP
+to TO B-PP
+a DT B-NP
+negative JJ I-NP
+750 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+on IN B-PP
+Tuesday NNP B-NP
+due NN I-NP
+to TO B-PP
+a DT B-NP
+number NN I-NP
+of IN B-PP
+corrections NN B-NP
+for IN B-PP
+unrelated VBN B-NP
+cash NN I-NP
+letter IN B-PP
+errors NNS B-NP
+in IN B-PP
+six JJ B-NP
+districts NNS I-NP
+around IN B-PP
+the DT B-NP
+country NN I-NP
+. . O
+The DT B-NP
+spokesman NN I-NP
+said VBD B-VP
+that NN B-NP
+on IN B-PP
+both JJ B-NP
+Tuesday NNP I-NP
+and CC I-NP
+Wednesday NNP B-NP
+, , O
+two IN B-PP
+different JJ B-NP
+clearing NN I-NP
+banks NNS I-NP
+had VBD B-VP
+system JJ B-NP
+problems NNS I-NP
+and CC O
+the DT B-NP
+securities NNS I-NP
+and CC I-NP
+Federal JJ I-NP
+funds NNS I-NP
+wires NNS I-NP
+had VBD B-VP
+to TO I-VP
+be VB I-VP
+held VBN I-VP
+open JJ B-NP
+until NNS I-NP
+about IN B-PP
+2000 CD B-NP
+or NNP I-NP
+2100 CD I-NP
+EST NNS I-NP
+on IN B-PP
+both JJ B-NP
+days NN I-NP
+. . O
+However NNP B-NP
+, , O
+he CD B-NP
+said VBD B-VP
+that IN B-SBAR
+both NNP B-NP
+problems NN I-NP
+were VBD B-VP
+cleared VBN I-VP
+up IN B-ADVP
+during VBG B-VP
+both IN B-PP
+afternoons NNS B-NP
+and CC O
+there DT B-NP
+was VBD B-VP
+no RB B-ADJP
+evidence JJ I-ADJP
+of IN B-PP
+any DT B-NP
+reserve JJ I-NP
+impact NN I-NP
+. . O
+During VBG B-VP
+the DT B-NP
+week NN I-NP
+ended VBN B-VP
+Wednesday NNP B-NP
+, , O
+45 CD B-NP
+pct NN I-NP
+of IN B-PP
+net JJ B-NP
+discount NN I-NP
+window NN I-NP
+borrowings NNS I-NP
+were VBD B-VP
+made JJ B-ADJP
+by IN B-PP
+the DT B-NP
+smallest NN I-NP
+banks NNS I-NP
+, , O
+with IN B-PP
+30 CD B-NP
+pct NN I-NP
+by IN B-PP
+the DT B-NP
+14 CD I-NP
+large RB I-NP
+money JJ I-NP
+center NN I-NP
+banks NNS I-NP
+and CC O
+25 CD B-NP
+pct NN I-NP
+by IN B-PP
+large JJ B-NP
+regional NN I-NP
+institutions NNS I-NP
+. . O
+On NNP B-NP
+Wednesday NNP I-NP
+, , O
+55 CD B-NP
+pct NN I-NP
+of IN B-PP
+the DT B-NP
+borrowing NN I-NP
+was VBD B-VP
+accounted VBN I-VP
+for IN B-PP
+by IN B-PP
+the DT B-NP
+money NN I-NP
+center NN I-NP
+banks NNS I-NP
+, , O
+with IN B-PP
+30 CD B-NP
+pct NN I-NP
+by IN B-PP
+the DT B-NP
+large JJ I-NP
+regionals NN I-NP
+and CC O
+15 CD B-NP
+pct NN I-NP
+by IN B-PP
+the DT B-NP
+smallest JJ I-NP
+banks NNS I-NP
+. . O
+The DT B-NP
+Fed JJ I-NP
+spokesman NN I-NP
+said VBD B-VP
+the DT B-NP
+banking NN I-NP
+system IN B-NP
+had VBD B-VP
+excess VBZ B-NP
+reserves NN I-NP
+on IN B-PP
+Thursday NNP B-NP
+, , O
+Monday NNP B-NP
+and CC I-NP
+Tuesday NNP I-NP
+and CC O
+a DT B-NP
+deficit NN I-NP
+on IN B-PP
+Friday NNP B-NP
+and CC O
+Wedndsday NNP B-NP
+. . I-NP
+That NNP I-NP
+produced VBD B-VP
+a DT B-NP
+small JJ I-NP
+daily NN I-NP
+average JJ I-NP
+deficit NN I-NP
+for IN B-PP
+the DT B-NP
+week NN I-NP
+as IN B-PP
+a DT B-NP
+whole NN I-NP
+. . B-VP
+For NNP B-PP
+the DT B-NP
+two-week NN I-NP
+period NNS I-NP
+, , O
+he NNS B-NP
+said VBD B-VP
+there EX B-NP
+were VBD B-VP
+relatively JJ B-NP
+high NN I-NP
+excess VBZ B-VP
+reserves NN B-NP
+on IN B-PP
+a DT B-NP
+daily JJ I-NP
+avearge NN I-NP
+, , O
+almost IN B-PP
+all DT B-NP
+of IN B-PP
+which CD B-NP
+were VBD B-VP
+at IN B-PP
+the DT B-NP
+smallest JJ I-NP
+banks NNS I-NP
+. . O
+Reuter IN B-PP
+&#3; CD B-NP
+American RB I-NP
+Express JJ I-NP
+Co NNP I-NP
+remained VBN I-NP
+silent NN I-NP
+on IN B-PP
+market NN B-NP
+rumors NN I-NP
+it PRP B-NP
+would VBD B-VP
+spinoff IN B-PP
+all DT B-NP
+or JJ I-NP
+part NN I-NP
+of IN B-PP
+its NNS B-NP
+Shearson NNP I-NP
+Lehman NNP I-NP
+Brothers NNS I-NP
+Inc NNP I-NP
+, , O
+but IN B-SBAR
+some DT B-NP
+analysts NNS I-NP
+said VBD B-VP
+the DT B-NP
+company NN I-NP
+may NN I-NP
+be VB B-VP
+considering NN B-NP
+such IN B-PP
+a DT B-NP
+move JJ I-NP
+because NN I-NP
+it PRP B-NP
+is VBZ B-VP
+unhappy NN B-NP
+with IN B-PP
+the DT B-NP
+market JJ I-NP
+value NN I-NP
+of IN B-PP
+its NNS B-NP
+stock NN I-NP
+. . B-ADVP
+American RB B-NP
+Express JJ I-NP
+stock NN I-NP
+got NN I-NP
+a DT B-NP
+lift NN I-NP
+from WRB B-ADVP
+the DT B-NP
+rumor NN I-NP
+, , O
+as IN B-SBAR
+the DT B-NP
+market NN I-NP
+calculated VBN B-VP
+a DT B-NP
+partially JJ I-NP
+public NN I-NP
+Shearson IN B-PP
+may NN B-NP
+command VBN B-VP
+a DT B-NP
+good JJ I-NP
+market NN I-NP
+value NN I-NP
+, , O
+thereby IN B-PP
+boosting VBG B-VP
+the DT B-NP
+total JJ I-NP
+value NN I-NP
+of IN B-PP
+American NNP B-NP
+Express default I-NP
+. . O
+The DT B-NP
+rumor NN I-NP
+also NN I-NP
+was VBD B-VP
+accompanied VBN I-VP
+by IN B-PP
+talk NN B-NP
+the DT B-NP
+financial JJ I-NP
+services NNS I-NP
+firm IN B-PP
+would JJ B-NP
+split NN I-NP
+its NNS I-NP
+stock IN B-PP
+and CC O
+boost JJ B-NP
+its NNS I-NP
+dividend VBD B-VP
+. . O
+American RB O
+Express VBZ B-VP
+closed VBN I-VP
+on IN B-PP
+the DT B-NP
+New JJ I-NP
+York NNP I-NP
+Stock NNP I-NP
+Exchange VBD B-VP
+at IN B-PP
+72-5/8 CD B-NP
+, , O
+up IN B-PP
+4-1/8 NN B-NP
+on IN B-PP
+heavy NN B-NP
+volume default I-NP
+. . B-ADVP
+American RB B-ADJP
+Express JJ I-ADJP
+would VBD B-VP
+not IN B-PP
+comment NN B-NP
+on IN B-PP
+the DT B-NP
+rumors NN I-NP
+or IN B-PP
+its NNS B-NP
+stock NN I-NP
+activity NN I-NP
+. . O
+Analysts NNS B-NP
+said VBD B-VP
+comments VBN I-VP
+by IN B-PP
+the DT B-NP
+company NN I-NP
+at IN B-PP
+an DT B-NP
+analysts' NN I-NP
+meeting VBG B-VP
+Tuesday default B-NP
+helped VBN I-NP
+fuel JJ B-NP
+the DT I-NP
+rumors NN I-NP
+as IN B-PP
+did NN B-NP
+an DT B-NP
+announcement JJ I-NP
+yesterday NN I-NP
+of IN B-PP
+management JJ B-NP
+changes NNS I-NP
+. . O
+At RB O
+the DT B-NP
+meeting VBG I-NP
+, , I-NP
+company NN I-NP
+officials IN B-NP
+said VBD B-VP
+American RB B-NP
+Express JJ I-NP
+stock NN I-NP
+is VBZ B-VP
+undervalued VBN I-VP
+and CC O
+does NNS B-VP
+not NN B-NP
+fully NN I-NP
+reflect NN B-VP
+the DT B-NP
+performance NN I-NP
+of IN B-PP
+Shearson NNP B-NP
+, , O
+according IN B-PP
+to TO B-PP
+analysts NNS B-NP
+. . O
+Yesterday NNP B-NP
+, , O
+Shearson NNP B-NP
+said VBD B-VP
+it PRP B-NP
+was VBD B-VP
+elevating VBG I-VP
+its NNS B-NP
+chief NNP I-NP
+operating VBG I-NP
+officer IN I-NP
+, , O
+Jeffery NNP B-NP
+Lane NNP I-NP
+, , O
+to TO B-PP
+the DT B-NP
+added JJ I-NP
+position NN I-NP
+of IN B-PP
+president NN B-NP
+, , O
+which IN B-NP
+had VBD B-VP
+been VBN I-VP
+vacant NN B-NP
+. . O
+It PRP B-NP
+also RB I-VP
+created VBN I-VP
+four IN B-PP
+new JJ B-NP
+positions NNS I-NP
+for IN B-PP
+chairmen NN B-NP
+of IN B-PP
+its NNS B-NP
+operating VBG I-NP
+divisions NNS I-NP
+. . O
+Analysts NNS B-NP
+speculated VBD B-VP
+a DT B-NP
+partial JJ I-NP
+spinoff NNP I-NP
+would VBD B-VP
+make NN B-NP
+most NN I-NP
+sense NNS I-NP
+, , O
+contrary JJ B-ADJP
+to TO B-PP
+one CD B-NP
+variation NN I-NP
+on IN B-PP
+market JJ B-NP
+rumors NN I-NP
+of IN B-PP
+a DT B-NP
+total JJ I-NP
+spinoff NNP I-NP
+. . O
+Some DT B-NP
+analysts NNS I-NP
+, , O
+however NNS B-NP
+, , O
+disagreed VBD B-VP
+that IN B-PP
+any JJ B-NP
+spinoff NN I-NP
+of IN B-PP
+Shearson NNP B-NP
+would VBD B-VP
+be VB I-VP
+good NN B-NP
+since IN B-SBAR
+it PRP B-NP
+is VBZ B-VP
+a DT B-NP
+strong VBG I-NP
+profit NN I-NP
+center NN I-NP
+for IN B-PP
+American NNP B-NP
+Express NNS I-NP
+, , O
+contributing VBG B-VP
+about IN B-NP
+20 CD I-NP
+pct NN I-NP
+of IN B-PP
+earnings NNS B-NP
+last JJ B-NP
+year NN I-NP
+. . O
+" NN B-NP
+I IN B-PP
+think NN B-NP
+it PRP B-NP
+is VBZ B-VP
+highly RB O
+unlikely JJ B-NP
+that NN I-NP
+American RB B-NP
+Express JJ I-NP
+is VBZ B-VP
+going VBG I-VP
+to TO B-PP
+sell JJ B-NP
+shearson NN I-NP
+, , O
+" IN B-NP
+said VBD B-VP
+Perrin CD B-NP
+Long VBG I-NP
+of IN B-PP
+Lipper NNP B-NP
+Analytical default I-NP
+. . O
+He JJ I-VP
+questioned VBD I-VP
+what IN B-NP
+would VBN B-VP
+be VB I-VP
+a DT B-NP
+better NN I-NP
+investment NN I-NP
+than NN I-NP
+" RB B-NP
+a DT I-NP
+very NN I-NP
+profitable NN I-NP
+securities NNS I-NP
+firm IN B-PP
+. . B-NP
+" NN I-NP
+Several JJ I-NP
+analysts NNS I-NP
+said VBD B-VP
+American RB B-ADJP
+Express JJ I-ADJP
+is VBZ B-VP
+not RB O
+in IN B-PP
+need JJ B-NP
+of IN B-PP
+cash NNP B-NP
+, , O
+which IN B-PP
+might NN B-NP
+be VB B-VP
+the DT B-NP
+only JJ I-NP
+reason NN I-NP
+to TO B-VP
+sell JJ I-VP
+a DT B-NP
+part NN I-NP
+of IN B-PP
+a DT B-NP
+strong NN I-NP
+asset IN B-PP
+. . B-NP
+But JJ I-NP
+others NNS I-NP
+believe VBP B-VP
+the DT B-NP
+company NN I-NP
+could VBN B-VP
+very JJ B-ADVP
+well RB B-ADVP
+of IN B-ADVP
+considered VBD B-VP
+the DT B-NP
+option NN I-NP
+of IN B-PP
+spinning VBG B-VP
+out JJ B-NP
+part NN I-NP
+of IN B-PP
+Shearson NNP B-NP
+, , O
+and CC O
+one JJ B-NP
+rumor NN I-NP
+suggests NNS I-NP
+selling VBG B-VP
+about IN B-NP
+20 CD I-NP
+pct NN I-NP
+of IN B-PP
+it PRP B-NP
+in IN B-PP
+the DT B-NP
+market NN I-NP
+. . O
+Larry JJ O
+Eckenfelder . O
+of IN B-PP
+Prudential-Bache DT B-NP
+Securities NNS I-NP
+said VBD B-VP
+he DT B-NP
+believes NN I-NP
+American RB B-VP
+Express VBN I-VP
+could VBN I-VP
+have VBP B-VP
+considered VBN I-VP
+a DT B-NP
+partial JJ I-NP
+spinoff NN I-NP
+in IN B-PP
+the DT B-NP
+past NN I-NP
+. . O
+" IN B-PP
+Shearson NNP B-NP
+being NN I-NP
+as IN B-PP
+profitable NN B-NP
+as IN B-SBAR
+it PRP B-NP
+is VBZ B-VP
+would VBD I-VP
+have VBP I-VP
+fetched VBN I-VP
+a DT B-NP
+big NN I-NP
+premium NN I-NP
+in IN B-PP
+the DT B-NP
+market NN I-NP
+place. NN I-NP
+Shearson's NNP I-NP
+book NN I-NP
+value NN I-NP
+is VBZ B-VP
+in IN B-PP
+the DT B-NP
+1.4 CD I-NP
+mln NN I-NP
+dlr IN B-PP
+range NN B-NP
+. . O
+Shearson NNP O
+in IN B-PP
+the DT B-NP
+market NN I-NP
+place NN I-NP
+would MD B-VP
+probably RB I-VP
+be VB I-VP
+worth RB B-ADVP
+three DT B-NP
+to TO I-NP
+3.5 CD I-NP
+bilion NN I-NP
+dlrs NN I-NP
+in IN B-PP
+terms NN B-NP
+of IN B-PP
+market JJ B-NP
+capitalization NN I-NP
+, , O
+" IN B-NP
+said VBD B-VP
+Eckenfelder CD B-NP
+. . O
+Some DT B-NP
+analysts NNS I-NP
+said VBD B-VP
+American RB B-NP
+Express JJ I-NP
+could VBN B-VP
+use IN B-PP
+capital JJ B-NP
+since NN I-NP
+it PRP B-NP
+plans VBD B-VP
+to TO I-VP
+expand NNS B-NP
+globally JJ B-ADJP
+. . O
+" NNS B-VP
+They NNP B-NP
+have VBP B-VP
+enormous NNS B-NP
+internal JJ B-NP
+growth NNS I-NP
+plans NNS I-NP
+that IN B-PP
+takes NNS B-NP
+capital JJ B-ADJP
+. . O
+You NNP B-NP
+want NN I-NP
+your NN I-NP
+stock RB B-ADVP
+to TO B-PP
+reflect JJ B-NP
+realistic NN I-NP
+valuations NNS I-NP
+to TO B-PP
+enhance JJ B-NP
+your NN I-NP
+ability NN I-NP
+to TO B-PP
+make JJ B-NP
+all DT I-NP
+kinds NN I-NP
+of IN B-PP
+endeavors NNS B-NP
+down IN B-PP
+the DT B-NP
+road NN I-NP
+, , O
+" IN B-NP
+said VBD B-VP
+E.F. CD B-NP
+Hutton NNP I-NP
+Group NNP I-NP
+analyst IN B-PP
+Michael default B-NP
+Lewis default I-NP
+. . B-NP
+" NN I-NP
+They've DT B-NP
+outlined VBD B-VP
+the DT B-NP
+fact NN I-NP
+that IN B-SBAR
+they're DT B-NP
+investing VBG I-NP
+heavily NN I-NP
+in IN B-PP
+the DT B-NP
+future NNS I-NP
+, , O
+which IN B-PP
+goes NNS B-NP
+heavily NN I-NP
+into IN B-PP
+the DT B-NP
+international JJ I-NP
+arena, NN I-NP
+" NN I-NP
+said VBD B-VP
+Lewis CD B-NP
+. . O
+" default B-VP
+. . I-VP
+..That . O
+does NNS B-VP
+not NN B-NP
+preclude NN I-NP
+acquisitions NNS I-NP
+and CC O
+divestitures NNS B-NP
+along IN B-PP
+the DT B-NP
+way NN I-NP
+, , O
+" IN B-PP
+he DT B-NP
+said VBD I-NP
+. . O
+Lewis VBZ O
+said VBD B-VP
+if CD B-NP
+American RB I-NP
+Express JJ I-NP
+reduced VBN I-NP
+its NNS I-NP
+exposure NN I-NP
+to TO B-PP
+the DT B-NP
+brokerage NN I-NP
+business NNS I-NP
+by IN B-PP
+selling VBG B-VP
+part NN B-NP
+of IN B-PP
+shearson NN B-NP
+, , O
+its NNS B-NP
+stock NN I-NP
+might NN I-NP
+better IN B-PP
+reflect NN B-NP
+other IN B-PP
+assets NNS B-NP
+, , O
+such NNS B-NP
+as IN B-PP
+the DT B-NP
+travel NN I-NP
+related VBN I-NP
+services NNS I-NP
+business NNS I-NP
+. . O
+" NN B-VP
+It PRP B-NP
+could VBD B-VP
+find CD B-NP
+its NNS I-NP
+true VBD B-VP
+water IN B-PP
+mark NN B-NP
+with IN B-PP
+a DT B-NP
+lesser JJ I-NP
+exposure NN I-NP
+to TO B-VP
+brokerage VB I-VP
+. . O
+The DT B-NP
+value NN I-NP
+of IN B-PP
+the DT B-NP
+other NN I-NP
+components NNP-named I-NP
+could VBN B-VP
+command VBN I-VP
+a DT B-NP
+higher NN I-NP
+multiple WRB B-ADVP
+because NN B-NP
+they NN I-NP
+constitute VBD B-VP
+a DT B-NP
+higher NN I-NP
+percentage NN I-NP
+of IN B-PP
+the DT B-NP
+total NN I-NP
+operating IN B-PP
+earnings NNS B-NP
+of IN B-PP
+the DT B-NP
+company NN I-NP
+, , O
+" IN B-PP
+he DT B-NP
+said VBD I-NP
+. . O
+Lewis VBZ O
+said VBD B-VP
+Shearson CD B-NP
+contributed VBN B-VP
+316 CD B-NP
+mln NN I-NP
+in IN B-PP
+after-tax JJ B-NP
+operating VBG I-NP
+earnings NNS I-NP
+, , O
+up NNS B-NP
+from IN B-PP
+about NN B-NP
+200 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+in IN B-PP
+1985 default B-NP
+. . O
+Reuter IN B-PP
+&#3; CD B-NP
+Coleco NNP I-NP
+Industries NNP I-NP
+Inc NNP I-NP
+said VBD B-VP
+it PRP B-NP
+expects NNS B-VP
+to TO B-NP
+return JJ I-VP
+to TO B-PP
+profitability NN B-NP
+in IN B-PP
+1987 default B-NP
+. . O
+Earlier NNP B-NP
+, , O
+Coleco NNP B-NP
+reported VBN B-VP
+a DT B-NP
+net JJ I-NP
+loss CD I-NP
+of IN B-PP
+111.2 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+for IN B-PP
+the DT B-NP
+year NN I-NP
+ended VBN B-VP
+December IN B-PP
+31 CD B-NP
+compared VBN B-VP
+to TO B-PP
+a DT B-NP
+profit NN I-NP
+of IN B-PP
+64.2 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+in IN B-PP
+the DT B-NP
+year NN I-NP
+earlier IN B-PP
+. . B-NP
+In IN B-PP
+a DT B-NP
+prepared JJ I-NP
+statement NN I-NP
+, , O
+the DT B-NP
+company NN I-NP
+said VBD B-VP
+the DT B-NP
+dramatic NN I-NP
+swing IN B-PP
+in IN B-PP
+operating VBG B-NP
+results NNS I-NP
+was VBD B-VP
+due JJ B-NP
+primarily NN I-NP
+to TO B-PP
+the DT B-NP
+steep NN I-NP
+decline NN I-NP
+in IN B-PP
+sales NNS B-NP
+of IN B-PP
+Cabbage JJ B-NP
+Patch NNP I-NP
+Kids NNP I-NP
+products NNS I-NP
+from IN B-PP
+600 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+to TO B-PP
+230 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+. . O
+Coleco NNP B-NP
+said VBD B-VP
+it PRP B-NP
+changed VBD B-VP
+from VBN I-VP
+a DT B-NP
+single JJ I-NP
+product NN I-NP
+company NN I-NP
+to TO B-PP
+a DT B-NP
+more JJ I-NP
+diversified CD I-NP
+organization NN I-NP
+through IN B-PP
+four JJ B-NP
+major NN I-NP
+acquisitions NNS I-NP
+last JJ B-NP
+year NN I-NP
+. . O
+Products NNS B-NP
+from IN B-PP
+the DT B-NP
+new NN I-NP
+acquisitions NNS I-NP
+and CC O
+other VB B-VP
+new RB B-NP
+product NN I-NP
+introductions NNS I-NP
+are VBP B-VP
+expected VBN I-VP
+to TO I-VP
+enable NNS B-NP
+it PRP B-NP
+to TO B-VP
+return JJ B-NP
+to TO B-PP
+profitability NN B-NP
+, , O
+it PRP B-NP
+said VBD B-VP
+. . O
+At RB O
+the DT B-NP
+annual JJ I-NP
+Toy NNP I-NP
+Fair NNP I-NP
+earlier IN B-PP
+this DT B-NP
+month JJ I-NP
+, , I-NP
+vice JJ I-NP
+president NN I-NP
+Morton NNP I-NP
+Handel NNP I-NP
+said VBD B-VP
+analysts' CD B-NP
+1987 NN I-NP
+projected VBN I-NP
+earnings NNS I-NP
+of IN B-PP
+90 CD B-NP
+cts NNS I-NP
+a DT B-NP
+share NN I-NP
+on IN B-PP
+sales NNS B-NP
+of IN B-PP
+600 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+are VBP B-VP
+reasonable NN B-NP
+. . O
+Venezuela NNP-5 B-NP
+is VBZ B-VP
+seeking VBG I-VP
+a DT B-NP
+'constructive JJ I-NP
+and CC I-NP
+flexible' NNS I-NP
+attitude IN B-PP
+from JJ B-NP
+its NNS I-NP
+creditor NN I-NP
+banks NNS I-NP
+in IN B-PP
+current JJ B-NP
+talks NNS I-NP
+to TO B-PP
+reschedule JJ B-NP
+21 CD I-NP
+billion NN I-NP
+dlrs NN I-NP
+in IN B-PP
+foreign NN B-NP
+debt VBN B-VP
+, , O
+finance JJ B-NP
+minister NN I-NP
+manuel JJ I-NP
+azpurua NN I-NP
+told VBN B-VP
+a DT B-NP
+press NN I-NP
+conference. NN I-NP
+He NNP I-NP
+declined VBD B-VP
+to TO B-PP
+comment NN B-NP
+on IN B-PP
+meetings NNS B-NP
+this DT B-NP
+week NN I-NP
+in IN B-PP
+new JJ B-NP
+york NN I-NP
+between VBN B-VP
+public IN B-PP
+finances NNS B-NP
+director IN B-PP
+jorge JJ B-NP
+marcano NN I-NP
+and CC O
+venezuela's VBN B-NP
+13-bank NN I-NP
+advisory NN I-NP
+committee NNP-named I-NP
+except NN I-NP
+to TO B-PP
+say NN B-NP
+, , O
+" IN B-PP
+they NN B-NP
+are VBP B-VP
+progressing NNS B-NP
+. . O
+" NNP B-NP
+Azpurua NNP I-NP
+said VBD B-VP
+venezuela NN B-NP
+has NNS I-NP
+shown IN B-PP
+solidarity JJ B-NP
+with IN B-PP
+brazil's NNS B-NP
+decision VBD B-VP
+to TO B-PP
+suspend CD B-NP
+payments NNS I-NP
+, , O
+but NNS B-NP
+each IN B-PP
+country NN B-NP
+must VBZ B-VP
+negotiate RB I-VP
+according VBG I-VP
+to TO B-PP
+its NNS B-NP
+own JJ I-NP
+interest NN I-NP
+. . O
+Asked VBD B-VP
+to TO B-PP
+comment NN B-NP
+on IN B-PP
+chile's NN B-NP
+agreement NN I-NP
+with IN B-PP
+its NNS B-NP
+creditors NN I-NP
+today NN I-NP
+, , O
+which IN B-PP
+includes NNS B-NP
+an DT B-NP
+interest JJ I-NP
+rate NN I-NP
+margin NN I-NP
+of IN B-PP
+one CD B-NP
+pct NN I-NP
+over IN B-PP
+libor JJ B-NP
+, , O
+azpurua NNP B-NP
+said VBD B-VP
+only NN B-NP
+, , O
+" IN B-SBAR
+that NN B-NP
+is VBZ B-VP
+good JJ B-NP
+news NNS I-NP
+. . O
+" NNS B-NP
+According VBG B-VP
+to TO B-PP
+banking VBG B-NP
+sources NNS I-NP
+, , O
+the DT B-NP
+banks' NN I-NP
+latest NN I-NP
+offer IN B-PP
+to TO B-PP
+venezuela CD B-NP
+is VBZ B-VP
+also RB B-ADVP
+a DT B-NP
+one JJ I-NP
+pct NN I-NP
+margin JJ I-NP
+as IN B-PP
+against NN B-NP
+the DT B-NP
+last JJ I-NP
+february's NN I-NP
+1-1/8 CD B-NP
+pct NN I-NP
+rescheduling VBG I-NP
+accord NNS I-NP
+and CC O
+the DT B-NP
+7/8 NN I-NP
+pct NN I-NP
+Venezuela NNP I-NP
+wants NNS I-NP
+. . O
+Azpurua NNP B-NP
+said VBD B-VP
+four NN B-NP
+basic NN I-NP
+elements NNS I-NP
+are VBP B-VP
+being VBN I-VP
+negotiated VBN I-VP
+with IN B-PP
+the DT B-NP
+banks NNS I-NP
+now: NN I-NP
+spread VBD B-VP
+reduction VBN I-VP
+, , O
+deferral JJ B-ADJP
+of IN B-PP
+principal JJ B-NP
+payments NNS I-NP
+due NNS I-NP
+in IN B-PP
+1987 CD B-NP
+and CC I-NP
+1988 CD I-NP
+, , O
+lenghtening VBG B-VP
+the DT B-NP
+12-1/2 CD I-NP
+year NN I-NP
+repayment NN I-NP
+schedule NN I-NP
+, , O
+and CC O
+debt VBN B-VP
+capitalization IN B-PP
+schemes NNS B-NP
+. . O
+Azpurua NNP B-NP
+said VBD B-VP
+the DT B-NP
+governent NN I-NP
+plans NN I-NP
+to TO B-PP
+pay NN B-NP
+2.1 CD I-NP
+billion NN I-NP
+dlrs NN I-NP
+in IN B-PP
+public NNP B-NP
+and CC O
+private JJ B-NP
+debt NN I-NP
+principal NN I-NP
+this DT B-NP
+year NN I-NP
+. . O
+It PRP B-NP
+was VBD B-VP
+due VBD I-VP
+to TO I-VP
+amortize VB I-VP
+1.05 CD B-NP
+billion NN I-NP
+dlrs NN I-NP
+under IN B-PP
+the DT B-NP
+rescheduling NN I-NP
+, , O
+and CC O
+pay NN B-NP
+420 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+in IN B-PP
+non-restructured JJ B-NP
+principal NN I-NP
+, , O
+both IN B-PP
+public JJ B-NP
+sector NN I-NP
+. . O
+He NNP B-NP
+said VBD B-VP
+venezuela's CD B-NP
+original JJ I-NP
+proposal NN I-NP
+was VBD B-VP
+to TO B-PP
+pay NN B-NP
+no RB I-NP
+principal JJ I-NP
+on IN B-PP
+restructured JJ B-NP
+debt NN I-NP
+this DT B-NP
+year NN I-NP
+, , O
+but IN B-NP
+is VBZ B-VP
+now RB I-VP
+insisting VBG I-VP
+that IN B-SBAR
+if NNP B-NP
+it PRP B-NP
+makes VBZ B-VP
+payments NNS B-NP
+they IN B-PP
+be VB B-NP
+compensated VBN B-VP
+by IN B-PP
+new JJ B-NP
+bank NN I-NP
+loans NNS I-NP
+. . O
+The DT B-NP
+banking VBG I-NP
+sources NNS I-NP
+said VBD B-VP
+the DT B-NP
+committee NN I-NP
+has NNS B-VP
+been VBN I-VP
+prepared VBN I-VP
+to TO I-VP
+lower VB I-VP
+amortizations VBN I-VP
+to TO B-PP
+around IN B-NP
+400 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+this IN B-PP
+year NN B-NP
+, , O
+but IN B-PP
+that NN B-NP
+no RB B-NP
+direct JJ I-NP
+commitment NN I-NP
+was VBD B-VP
+likely JJ B-ADJP
+on IN B-PP
+new JJ B-NP
+loans NNS I-NP
+. . O
+" CD B-NP
+debtors NNS I-NP
+and CC I-NP
+bank NNS I-NP
+creditors NN I-NP
+have VBP B-VP
+a DT B-NP
+joint JJ I-NP
+responsibility NN I-NP
+and CC O
+there DT B-NP
+will MD B-VP
+be VB I-VP
+no RB I-VP
+lasting VBG I-VP
+solution NN B-NP
+unless NNS I-NP
+a DT B-NP
+positive JJ I-NP
+flow NN I-NP
+of IN B-PP
+financing VBG B-NP
+is VBZ B-VP
+guaranteed NNS B-NP
+, , O
+" NNS B-NP
+azpurua DT B-NP
+said VBD B-VP
+. . O
+However IN B-ADVP
+, , O
+he NNS B-NP
+appeared VBD B-VP
+to TO I-VP
+discard VB I-VP
+earlier JJ B-NP
+venezuelan NN I-NP
+proposals NN I-NP
+for IN B-PP
+a DT B-NP
+direct NN I-NP
+link NN I-NP
+between VBN B-VP
+oil JJ B-NP
+income NN I-NP
+and CC O
+debt VBN B-NP
+payments NNS I-NP
+, , O
+"because NNS B-NP
+circumstances NNS I-NP
+change VBD B-VP
+too RB B-ADJP
+quickly JJ I-ADJP
+. . O
+" NNS B-VP
+At RB B-ADVP
+the DT B-NP
+same JJ I-NP
+time NN I-NP
+, , O
+he NN B-NP
+said VBD B-VP
+the DT B-NP
+government NN I-NP
+is VBZ B-VP
+presently RB I-VP
+studying VBG I-VP
+possible JJ B-NP
+mechanisms NNS I-NP
+for IN B-PP
+capitlizing VBG B-VP
+public NN B-NP
+and CC O
+private RB B-NP
+sector JJ I-NP
+foreign NNS I-NP
+debt VBD B-VP
+, , O
+based NNS B-NP
+on IN B-PP
+experience NN B-NP
+in IN B-PP
+other JJ B-NP
+countries NNS I-NP
+. . O
+The DT B-NP
+rules NN I-NP
+would MD B-VP
+be VB I-VP
+published VBN I-VP
+by IN B-PP
+the DT B-NP
+finance JJ I-NP
+ministry NN I-NP
+and CC O
+the DT B-NP
+central JJ I-NP
+bank NN I-NP
+. . O
+Thomson NNP B-NP
+McKinnon NNP I-NP
+Mortgage NNP I-NP
+Assets NNS I-NP
+Corp NNP I-NP
+, , O
+a DT B-NP
+unit NN I-NP
+of IN B-PP
+Thomson NNP B-NP
+McKinnon NNP I-NP
+Inc NNP I-NP
+, , O
+is VBZ B-VP
+offering IN B-NP
+100 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+of IN B-PP
+collateralized VBN B-NP
+mortgage NN I-NP
+obligations NNS I-NP
+in IN B-PP
+three DT B-NP
+tranches NNS I-NP
+that IN B-PP
+include JJ B-NP
+floating NN I-NP
+rate NN I-NP
+and CC O
+inverse RB B-VP
+floating VBG I-VP
+rate JJ B-NP
+CMOS NNP I-NP
+. . O
+The DT B-NP
+floating VBG I-NP
+rate NN I-NP
+class NN I-NP
+amounts NNS I-NP
+to TO B-PP
+60 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+. . O
+It PRP B-NP
+has VBZ B-VP
+an DT B-NP
+average JJ I-NP
+life NN I-NP
+of IN B-PP
+7.11 CD B-NP
+years NNS I-NP
+and CC O
+matures NNS B-NP
+2018. CD B-PP
+The DT B-NP
+CMOs JJ I-NP
+have NN I-NP
+an DT B-NP
+initial JJ I-NP
+coupon NN I-NP
+of IN B-PP
+7.0375 CD B-NP
+pct NN I-NP
+, , O
+which IN B-NP
+will MD B-VP
+be VB I-VP
+reset NN B-NP
+60 CD I-NP
+basis NNS I-NP
+points NNS I-NP
+above VBP B-VP
+LIBOR VBN I-VP
+, , O
+said VBD B-VP
+sole CD B-NP
+manager NN I-NP
+Thomson NNP I-NP
+McKinnon NNP I-NP
+. . O
+The DT B-NP
+inverse JJ I-NP
+floater NN I-NP
+totals IN B-PP
+4.8 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+. . O
+It PRP B-NP
+has VBZ B-VP
+an DT B-NP
+average JJ I-NP
+life NN I-NP
+of IN B-PP
+13.49 CD B-NP
+years NNS I-NP
+and CC O
+matures NNS B-NP
+2018. CD B-NP
+These NNP I-NP
+CMOs NNS I-NP
+were VBD B-VP
+given JJ I-VP
+an DT B-NP
+initial JJ I-NP
+coupon NN I-NP
+of IN B-PP
+11-1/2 CD B-NP
+pct NN I-NP
+and CC O
+priced VBN B-VP
+at IN B-PP
+104.40. CD B-NP
+Subsequent JJ I-NP
+rates NNS I-NP
+on IN B-PP
+the DT B-NP
+inverse NN I-NP
+floater IN B-PP
+will DT B-NP
+equal JJ I-NP
+11-1/2 CD I-NP
+pct NN I-NP
+minus VBZ B-VP
+the DT B-NP
+product NN I-NP
+of IN B-PP
+three DT B-NP
+times NNS I-NP
+(LIBOR NNP I-NP
+minus NNS I-NP
+6-1/2 CD B-NP
+pct) NNS I-NP
+. . O
+A RB O
+Thomson NNP B-NP
+officer IN I-PRT
+explained VBN B-VP
+that IN B-PP
+the DT B-NP
+coupon NN I-NP
+of IN B-PP
+the DT B-NP
+inverse NN I-NP
+floating IN B-PP
+rate JJ B-NP
+tranche NN I-NP
+would VBN B-VP
+increase IN B-PP
+if NNP B-NP
+LIBOR default I-NP
+declined VBD B-VP
+. . O
+" NN B-VP
+The DT B-NP
+yield JJ I-NP
+floats NN I-NP
+opposite NN I-NP
+of IN B-PP
+LIBOR NNP B-NP
+, , O
+" CD B-NP
+he NN I-NP
+said VBD B-VP
+. . O
+The DT B-NP
+fixed-rate JJ I-NP
+tranche NN I-NP
+totals NNS I-NP
+35.2 IN B-PP
+mln NN B-NP
+dlrs NN I-NP
+. . O
+It PRP B-NP
+has VBZ B-VP
+an DT B-NP
+average JJ I-NP
+life NN I-NP
+of IN B-PP
+3.5 CD B-NP
+years NNS I-NP
+and CC O
+matures NNS B-NP
+2016. CD B-PP
+The DT B-NP
+CMOs JJ I-NP
+were NN I-NP
+assigned VBN B-VP
+a DT B-NP
+7.65 NN I-NP
+pct NN I-NP
+coupon NN I-NP
+and CC O
+par RB B-VP
+pricing VBG I-VP
+. . B-PP
+The DT B-NP
+issue NN I-NP
+is VBZ B-VP
+rated VBN I-VP
+AAA RB B-ADVP
+by IN B-PP
+Standard NNP B-NP
+and CC I-NP
+Poor's NNP I-NP
+and CC O
+secured JJ B-ADVP
+by IN B-SBAR
+Federal JJ B-NP
+Home . I-NP
+Loan NNP I-NP
+Mortgage NNP I-NP
+Corp NNP I-NP
+, , O
+Freddie NNP B-NP
+Mac NNP I-NP
+, , O
+certificates NNS B-NP
+. . O
+OPEC NNP B-NP
+may NN I-NP
+be VB B-VP
+forced VBD I-VP
+to TO B-PP
+meet NN B-NP
+before NN I-NP
+a DT B-NP
+scheduled JJ I-NP
+June CD I-NP
+session NN I-NP
+to TO B-PP
+readdress JJ B-NP
+its NNS I-NP
+production NN I-NP
+cutting VBG I-NP
+agreement NN I-NP
+if IN B-SBAR
+the DT B-NP
+organization NN I-NP
+wants NNS B-VP
+to TO I-VP
+halt VB I-VP
+the DT B-NP
+current NN I-NP
+slide NN I-NP
+in IN B-PP
+oil JJ B-NP
+prices NNS I-NP
+, , O
+oil JJ B-NP
+industry NN I-NP
+analysts NNS I-NP
+said VBD B-VP
+. . O
+" NN B-VP
+The DT B-NP
+movement NN I-NP
+to TO B-VP
+higher VB I-VP
+oil JJ B-NP
+prices NNS I-NP
+was VBD B-VP
+never IN B-ADVP
+to TO B-VP
+be VB I-VP
+as IN B-PP
+easy NN B-NP
+as IN B-PP
+OPEC NNP B-NP
+thought IN B-PP
+. . B-NP
+They NNP I-NP
+may NN I-NP
+need VBD B-VP
+an DT B-NP
+emergency NN I-NP
+meeting VBG B-VP
+to TO B-PP
+sort NN B-NP
+out IN B-PP
+the DT B-NP
+problems NN I-NP
+, , O
+" IN B-NP
+said VBD B-VP
+Daniel CD B-NP
+Yergin NNP I-NP
+, , O
+director IN B-PP
+of IN B-PP
+Cambridge JJ B-NP
+Energy NNP I-NP
+Research NNP I-NP
+Associates NNS I-NP
+, , O
+CERA NNP B-NP
+. . O
+Analysts NNS B-NP
+and CC O
+oil JJ B-NP
+industry NN I-NP
+sources NNS I-NP
+said VBD B-VP
+the DT B-NP
+problem NN I-NP
+OPEC IN B-PP
+faces NNS B-NP
+is VBZ B-VP
+excess NNS B-NP
+oil JJ B-ADJP
+supply RB B-ADVP
+in IN B-PP
+world JJ B-NP
+oil NNS I-NP
+markets NNS I-NP
+. . O
+" NN B-NP
+OPEC's NNS I-NP
+problem NN I-NP
+is VBZ B-VP
+not RB O
+a DT B-NP
+price NN I-NP
+problem NN I-NP
+but NN B-VP
+a DT B-NP
+production NN I-NP
+issue NNS I-NP
+and CC O
+must JJ B-VP
+be VB I-VP
+addressed VBN I-VP
+in IN B-PP
+that DT B-NP
+way NN I-NP
+, , O
+" IN B-NP
+said VBD B-VP
+Paul default B-NP
+Mlotok NNP I-NP
+, , O
+oil JJ B-NP
+analyst NN I-NP
+with IN B-PP
+Salomon NNP B-NP
+Brothers NNS I-NP
+Inc NNP I-NP
+. . O
+He JJ O
+said VBD B-VP
+the DT B-NP
+market's NN I-NP
+earlier IN B-PP
+optimism JJ B-NP
+about NN I-NP
+OPEC NNS I-NP
+and CC O
+its NNS B-NP
+ability NN I-NP
+to TO B-PP
+keep CD B-NP
+production NN I-NP
+under IN B-PP
+control JJ B-NP
+have NN I-NP
+given NNS I-NP
+way NN B-ADJP
+to TO B-PP
+a DT B-NP
+pessimistic NN I-NP
+outlook NN I-NP
+that IN B-PP
+the DT B-NP
+organization NN I-NP
+must NN I-NP
+address NNS I-NP
+soon IN B-PP
+if NNP B-NP
+it PRP B-NP
+wishes VBD B-VP
+to TO B-PP
+regain JJ B-NP
+the DT I-NP
+initiative JJ I-NP
+in IN B-PP
+oil JJ B-NP
+prices NNS I-NP
+. . O
+But JJ B-NP
+some NN I-NP
+other IN B-PP
+analysts NNS B-NP
+were VBD B-VP
+uncertain RB B-ADJP
+that IN B-PP
+even VBN B-NP
+an DT B-NP
+emergency NN I-NP
+meeting VBG B-VP
+would JJ B-NP
+address NNS I-NP
+the DT B-NP
+problem NN I-NP
+of IN B-PP
+OPEC NNP B-NP
+production NN I-NP
+above IN B-PP
+the DT B-NP
+15.8 CD I-NP
+mln NN I-NP
+bpd NNS I-NP
+quota IN B-PP
+set NN B-NP
+last JJ B-NP
+December NNP I-NP
+. . O
+" IN B-SBAR
+OPEC NNP B-NP
+has NNS I-NP
+to TO B-PP
+learn JJ B-NP
+that NN I-NP
+in IN B-PP
+a DT B-NP
+buyers NNS I-NP
+market NN I-NP
+you IN B-PP
+cannot NN B-NP
+have VBP B-VP
+deemed VBN I-VP
+quotas NNS B-NP
+, , O
+fixed NNS B-NP
+prices NNS I-NP
+and CC O
+set NN B-NP
+differentials NNS I-NP
+, , O
+" NNS B-NP
+said VBD B-VP
+the DT B-NP
+regional JJ I-NP
+manager NN I-NP
+for IN B-PP
+one NN B-NP
+of IN B-PP
+the DT B-NP
+major NN I-NP
+oil NN I-NP
+companies NNS I-NP
+who IN B-PP
+spoke NN B-NP
+on IN B-PP
+condition NN B-NP
+that IN B-SBAR
+he DT B-NP
+not NN I-NP
+be VB B-VP
+named RB I-VP
+. . I-VP
+" NN I-VP
+The DT B-NP
+market NN I-NP
+is VBZ B-VP
+now RB I-VP
+trying VBG I-VP
+to TO I-VP
+teach IN B-PP
+them DT B-NP
+that NN I-NP
+lesson IN B-PP
+again NN B-NP
+, , O
+" IN B-PP
+he DT B-NP
+added VBD I-NP
+. . O
+David VBD B-VP
+T NNP B-NP
+. . I-NP
+Mizrahi NNP I-NP
+, , O
+editor CD B-NP
+of IN B-PP
+Mideast JJ B-NP
+reports NNS I-NP
+, , O
+expects NNS B-NP
+OPEC VBD B-VP
+to TO B-PP
+meet JJ B-NP
+before NN I-NP
+June JJ I-NP
+, , O
+although IN B-SBAR
+not JJ B-NP
+immediately NN I-NP
+. . O
+However NNP B-NP
+, , O
+he CD B-NP
+is VBZ B-VP
+not RB O
+optimistic JJ B-NP
+that NN I-NP
+OPEC default I-NP
+can NN I-NP
+address NNS I-NP
+its NNS B-NP
+principal JJ I-NP
+problems NNS I-NP
+. . O
+" IN O
+They NNP B-NP
+will MD B-VP
+not JJ B-NP
+meet NN I-NP
+now NN I-NP
+as IN B-PP
+they NN B-NP
+try NN I-NP
+to TO B-VP
+take VB I-VP
+advantage NN B-NP
+of IN B-PP
+the DT B-NP
+winter NN I-NP
+demand VBN B-VP
+to TO B-PP
+sell JJ B-NP
+their NN I-NP
+oil NNS I-NP
+, , O
+but NNS B-NP
+in IN B-PP
+late JJ B-NP
+March NNP I-NP
+and CC O
+April default B-NP
+when JJ I-NP
+demand VBN I-NP
+slackens NNS I-NP
+, , O
+" NNS B-NP
+Mizrahi NNP I-NP
+said VBD B-VP
+. . O
+But JJ B-NP
+Mizrahi NNP I-NP
+said VBD B-VP
+that NN B-SBAR
+OPEC default B-NP
+is VBZ B-VP
+unlikely RB B-ADJP
+to TO B-VP
+do VB I-VP
+anything VBG I-VP
+more JJ B-NP
+than NN I-NP
+reiterate NN I-NP
+its NNS I-NP
+agreement NN I-NP
+to TO B-PP
+keep CD B-NP
+output NN I-NP
+at IN B-PP
+15.8 CD B-NP
+mln NN I-NP
+bpd NNS I-NP
+. . O
+" NNP B-NP
+Analysts NNS I-NP
+said VBD B-VP
+that IN B-SBAR
+the DT B-NP
+next NN I-NP
+two NN I-NP
+months VBZ B-VP
+will RB I-VP
+be VB I-VP
+critical NN B-NP
+for IN B-PP
+OPEC's NNP B-NP
+ability NN I-NP
+to TO B-VP
+hold VB I-VP
+together NN B-NP
+prices NNS I-NP
+and CC O
+output JJ B-NP
+. . O
+" IN B-PP
+OPEC NNP B-NP
+must JJ I-NP
+hold VBD B-VP
+to TO B-PP
+its NNS B-NP
+pact NN I-NP
+for IN B-PP
+the DT B-NP
+next NN I-NP
+six IN B-PP
+to TO B-PP
+eight JJ B-NP
+weeks NNS I-NP
+since IN B-SBAR
+buyers NNS B-NP
+will MD B-VP
+come VBN I-VP
+back NN B-NP
+into IN B-PP
+the DT B-NP
+market NN I-NP
+then VBN B-VP
+, , O
+" NNP B-NP
+said VBD B-VP
+Dillard CD B-NP
+Spriggs NNS I-NP
+of IN B-PP
+Petroleum default B-NP
+Analysis RB B-VP
+Ltd VBN I-VP
+in IN B-PP
+New NNP B-NP
+York NNP I-NP
+. . O
+But JJ B-NP
+Bijan NNP I-NP
+Moussavar-Rahmani NNP I-NP
+of IN B-PP
+Harvard default B-NP
+University's NNS I-NP
+Energy NNP I-NP
+and CC I-NP
+Environment JJ I-NP
+Policy NNP I-NP
+Center NNP I-NP
+said VBD B-VP
+that NN B-SBAR
+the DT B-NP
+demand VBN B-VP
+for IN B-PP
+OPEC NNP B-NP
+oil NN I-NP
+has NNS I-NP
+been VBN B-VP
+rising VBG I-VP
+through IN B-PP
+the DT B-NP
+first JJ I-NP
+quarter NN I-NP
+and CC O
+this DT B-NP
+may NN I-NP
+have VBP B-VP
+prompted VBN I-VP
+excesses NNS B-NP
+in IN B-PP
+its NNS B-NP
+production NN I-NP
+. . O
+" CD B-NP
+Demand CD I-NP
+for IN B-PP
+their NN B-NP
+(OPEC) default I-NP
+oil JJ I-NP
+is VBZ B-VP
+clearly RB B-ADJP
+above JJ I-ADJP
+15.8 CD B-NP
+mln NN I-NP
+bpd NNS I-NP
+and CC O
+is VBZ B-VP
+probably RB B-NP
+closer NN I-NP
+to TO B-PP
+17 CD B-NP
+mln NN I-NP
+bpd NNS I-NP
+or IN B-PP
+higher NN B-NP
+now RB I-NP
+so JJ I-NP
+what NN I-NP
+we NNS I-NP
+are VBP B-VP
+seeing IN O
+characterized VBN B-VP
+as IN B-PP
+cheating VBG B-NP
+is VBZ B-VP
+OPEC NNP B-NP
+meeting VBG I-NP
+this DT B-NP
+demand VBD B-VP
+through IN B-PP
+current NN B-NP
+production NN I-NP
+, , O
+" IN B-PP
+he DT B-NP
+told JJ I-NP
+Reuters NNS I-NP
+in IN B-PP
+a DT B-NP
+telephone NN I-NP
+interview WRB B-ADVP
+. . O
+BankAmerica NNP B-NP
+Corp NNP I-NP
+is VBZ B-VP
+not RB O
+under IN B-PP
+pressure NN B-NP
+to TO B-PP
+act IN B-NP
+quickly JJ I-NP
+on IN B-PP
+its NNS B-NP
+proposed VBD B-VP
+equity JJ I-VP
+offering VBG I-VP
+and CC I-VP
+would VBN I-VP
+do IN B-PP
+well JJ B-NP
+to TO B-VP
+delay NN I-VP
+it PRP B-NP
+because NN B-NP
+of IN B-PP
+the DT B-NP
+stock's NN I-NP
+recent NN I-NP
+poor NNS I-NP
+performance NNS I-NP
+, , O
+banking NNS B-NP
+analysts NNS I-NP
+said VBD B-VP
+. . B-NP
+Some JJ I-NP
+analysts NNS I-NP
+said VBD B-VP
+they IN B-PP
+have NN B-NP
+recommended VBN B-VP
+BankAmerica NNP B-NP
+delay NN I-NP
+its NNS I-NP
+up VBD B-VP
+to TO B-PP
+one-billion-dlr CD B-NP
+equity NN I-NP
+offering VBG I-NP
+, , O
+which IN B-SBAR
+has NNS B-NP
+yet VBD B-VP
+to TO I-VP
+be VB I-VP
+approved VBN I-VP
+by IN B-PP
+the DT B-NP
+Securities NNS I-NP
+and CC I-NP
+Exchange JJ I-NP
+Commission NNP I-NP
+. . I-NP
+BankAmerica NNP I-NP
+stock NN I-NP
+fell NNS I-NP
+this IN B-PP
+week NN B-NP
+, , O
+along IN B-PP
+with IN B-PP
+other NN B-NP
+banking VBG I-NP
+issues NNS I-NP
+, , O
+on IN B-PP
+the DT B-NP
+news NN I-NP
+that IN B-PP
+Brazil JJ B-NP
+has NNS I-NP
+suspended VBD B-VP
+interest IN B-PP
+payments NNS B-NP
+on IN B-PP
+a DT B-NP
+large JJ I-NP
+portion NN I-NP
+of IN B-PP
+its NNS B-NP
+foreign IN B-PP
+debt NN B-NP
+. . O
+The DT B-NP
+stock NN I-NP
+traded VBN B-VP
+around IN B-PP
+12 CD B-NP
+, , O
+down IN B-PP
+1/8 NN B-NP
+, , O
+this IN B-PP
+afternoon NN B-NP
+, , O
+after IN B-PP
+falling VBG B-VP
+to TO B-PP
+11-1/2 CD B-NP
+earlier NN I-NP
+this IN B-PP
+week NN B-NP
+on IN B-PP
+the DT B-NP
+news NN I-NP
+. . O
+Banking NNP B-NP
+analysts NNS I-NP
+said VBD B-VP
+that IN B-SBAR
+with IN B-PP
+the DT B-NP
+immediate JJ I-NP
+threat NN I-NP
+of IN B-PP
+the DT B-NP
+First JJ I-NP
+Interstate NNP I-NP
+Bancorp NNP I-NP
+<I> NNP I-NP
+takeover IN B-PP
+bid NN B-NP
+gone NN I-NP
+, , O
+BankAmerica NNP B-NP
+is VBZ B-VP
+under IN B-PP
+no NN B-NP
+pressure NN I-NP
+to TO B-PP
+sell JJ B-NP
+the DT I-NP
+securities NN I-NP
+into IN B-PP
+a DT B-NP
+market NN I-NP
+that IN B-NP
+will MD B-VP
+be VB I-VP
+nervous RB B-ADJP
+on IN B-PP
+bank NN B-NP
+stocks NNS I-NP
+in IN B-PP
+the DT B-NP
+near JJ I-NP
+term NN I-NP
+. . O
+BankAmerica NNP O
+filed VBD B-VP
+the DT B-NP
+offer NN I-NP
+on IN B-PP
+January NNP B-NP
+26. CD I-NP
+It PRP B-NP
+was VBD B-VP
+seen JJ B-ADJP
+as IN B-PP
+one NN B-NP
+of IN B-PP
+the DT B-NP
+major NN I-NP
+factors NNS I-NP
+leading VBG B-VP
+the DT B-NP
+First JJ I-NP
+Interstate NNP I-NP
+withdrawing VBG B-VP
+its NNS B-NP
+takeover IN B-PP
+bid VBN B-NP
+on IN B-PP
+February NNP B-NP
+9. CD I-NP
+A RB I-NP
+BankAmerica NNP I-NP
+spokesman NN I-NP
+said VBD B-VP
+SEC CD B-NP
+approval JJ I-NP
+is VBZ B-VP
+taking IN B-PP
+longer JJ B-NP
+than NN I-NP
+expected VBN I-NP
+and CC I-NP
+market JJ I-NP
+conditions NN I-NP
+must JJ I-NP
+now RB B-ADVP
+be VB B-VP
+re-evaluated VBN I-VP
+. . O
+" IN B-PP
+The DT B-NP
+circumstances NNS I-NP
+at IN B-PP
+the DT B-NP
+time NN I-NP
+will MD B-VP
+determine NN I-VP
+what IN B-NP
+we JJ B-NP
+do NN I-NP
+, , O
+" IN B-NP
+said VBD B-VP
+Arthur RB B-ADJP
+Miller JJ I-ADJP
+, , O
+BankAmerica's NNP B-NP
+Vice JJ I-NP
+President NN I-NP
+for IN B-PP
+Financial JJ B-NP
+Communications NNP I-NP
+, , O
+when JJ B-NP
+asked VBD B-VP
+if NNP B-NP
+BankAmerica NNP I-NP
+would VBD B-VP
+proceed NN B-NP
+with IN B-PP
+the DT B-NP
+offer NN I-NP
+immediately NN I-NP
+after IN B-PP
+it PRP B-NP
+receives NNS B-VP
+SEC NNP B-NP
+approval JJ I-NP
+. . O
+" IN B-PP
+I'd NNP B-NP
+put NN B-VP
+it PRP B-NP
+off NNP B-NP
+as IN B-PP
+long NN B-NP
+as IN B-PP
+they NN B-NP
+conceivably NN I-NP
+could VBN B-VP
+, , O
+" NNP B-NP
+said VBD B-VP
+Lawrence CD B-NP
+Cohn NNP I-NP
+, , I-NP
+analyst JJ I-NP
+with IN B-PP
+Merrill default B-NP
+Lynch NNP I-NP
+, , I-NP
+Pierce NNP I-NP
+, , I-NP
+Fenner NNP I-NP
+and CC I-NP
+Smith NNP I-NP
+. . O
+Cohn NNP B-NP
+said VBD B-VP
+the DT B-NP
+longer NN I-NP
+BankAmerica NNP I-NP
+waits NNS I-NP
+, , O
+the DT B-NP
+longer JJR I-NP
+they NN I-NP
+have VBP B-VP
+to TO I-VP
+show WRB I-VP
+the DT B-NP
+market NN I-NP
+an DT B-NP
+improved VBD B-VP
+financial JJ B-NP
+outlook NN I-NP
+. . O
+Although RB O
+BankAmerica NNP B-NP
+has NNS I-NP
+yet VBD B-VP
+to TO I-VP
+specify VB I-VP
+the DT B-NP
+types NNS I-NP
+of IN B-PP
+equities NNS B-NP
+it PRP B-NP
+would VBD B-VP
+offer IN B-ADVP
+, , O
+most JJ B-NP
+analysts NN I-NP
+believed VBN B-VP
+a DT B-NP
+convertible NN I-NP
+preferred VBN I-NP
+stock NN I-NP
+would VBD B-VP
+encompass VBN I-VP
+at IN B-PP
+least JJ B-NP
+part NN I-NP
+of IN B-PP
+it PRP B-NP
+. . O
+Such NNP O
+an DT B-NP
+offering VBG I-NP
+at IN B-PP
+a DT B-NP
+depressed JJ I-NP
+stock NN I-NP
+price NN I-NP
+would VBN B-VP
+mean VBN I-VP
+a DT B-NP
+lower JJ I-NP
+conversion NN I-NP
+price NN I-NP
+and CC O
+more RB B-VP
+dilution VBN I-VP
+to TO B-PP
+BankAmerica NNP B-NP
+stock NN I-NP
+holders NNS I-NP
+, , O
+noted JJ B-NP
+Daniel . I-NP
+Williams NNS I-NP
+, , O
+analyst JJ B-ADJP
+with IN B-PP
+Sutro NNP B-NP
+Group default I-NP
+. . O
+Several JJ B-NP
+analysts NNS I-NP
+said VBD B-VP
+that IN B-SBAR
+while NN B-NP
+they NN I-NP
+believe VB B-VP
+the DT B-NP
+Brazilian JJ I-NP
+debt NN I-NP
+problem NN I-NP
+will RB B-VP
+continue VBD I-VP
+to TO I-VP
+hang NNS B-NP
+over IN B-PP
+the DT B-NP
+banking VBG I-NP
+industry NN I-NP
+through IN B-PP
+the DT B-NP
+quarter NN I-NP
+, , O
+the DT B-NP
+initial JJ I-NP
+shock NN I-NP
+reaction NN I-NP
+is VBZ B-VP
+likely RB B-ADVP
+to TO B-PP
+ease NNS B-NP
+over IN B-PP
+the DT B-NP
+coming VBG I-NP
+weeks NNS I-NP
+. . O
+Nevertheless NNP B-NP
+, , O
+BankAmerica, NNP B-NP
+which IN B-PP
+holds NN B-NP
+about IN B-PP
+2.70 CD B-NP
+billion NN I-NP
+dlrs NN I-NP
+in IN B-PP
+Brazilian JJ B-NP
+loans NNS I-NP
+, , O
+stands NNS B-NP
+to TO B-PP
+lose JJ B-NP
+15-20 CD I-NP
+mln NN I-NP
+dlrs NN I-NP
+if IN B-PP
+the DT B-NP
+interest NN I-NP
+rate NN I-NP
+is VBZ B-VP
+reduced VBN I-VP
+on IN B-PP
+the DT B-NP
+debt NN I-NP
+, , O
+and CC O
+as IN B-PP
+much NN B-NP
+as IN B-PP
+200 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+if NNP I-NP
+Brazil NNP I-NP
+pays NNS B-VP
+no RB B-NP
+interest NN I-NP
+for IN B-PP
+a DT B-NP
+year NN I-NP
+, , O
+said VBD B-VP
+Joseph NNP B-NP
+Arsenio RB I-NP
+, , O
+analyst JJ B-ADJP
+with IN B-PP
+Birr NNP B-NP
+, , I-NP
+Wilson NNP I-NP
+and CC I-NP
+Co JJ I-NP
+. . O
+He DT B-NP
+noted VBN B-VP
+, , O
+however IN B-ADVP
+, , O
+that IN B-SBAR
+any NN B-NP
+potential JJ I-NP
+losses NNS I-NP
+would VBD B-VP
+not RB B-NP
+show JJ I-NP
+up NNS I-NP
+in IN B-PP
+the DT B-NP
+current NN I-NP
+quarter IN B-PP
+. . O
+The DT B-NP
+Federal JJ I-NP
+Deposit NN I-NP
+Insurance IN B-PP
+Corp NNP B-NP
+(FDIC) NNP I-NP
+said VBD B-VP
+three NN B-NP
+troubled VBD I-NP
+banks NNS I-NP
+in IN B-PP
+Texas NNP B-NP
+and CC I-NP
+Louisiana NNP I-NP
+were VBD B-VP
+merged VBN I-VP
+with IN B-PP
+healthy NN B-NP
+financial JJ I-NP
+institutions NNS I-NP
+. . O
+The DT B-NP
+FDIC NNP I-NP
+said VBD B-VP
+it PRP B-NP
+subsidized VBD B-VP
+the DT B-NP
+merger NN I-NP
+of IN B-PP
+Central JJ B-NP
+Bank NNP I-NP
+and CC I-NP
+Trust JJ I-NP
+Co NNP I-NP
+, , I-NP
+Glenmora NNP I-NP
+, , I-NP
+La. NNP I-NP
+, , I-NP
+with IN B-PP
+the DT B-NP
+healthy NN I-NP
+Peoples NNS I-NP
+Bank NNP I-NP
+and CC I-NP
+Trust JJ I-NP
+Co NNP I-NP
+, , O
+Natchitoches NNS B-NP
+, , O
+La. NNP B-NP
+, , O
+after IN B-PP
+state NN B-NP
+regulators VBN B-VP
+notified VBN I-VP
+it PRP B-NP
+that IN B-PP
+Central JJ B-NP
+was NNS I-NP
+in IN B-PP
+danger NN B-NP
+of IN B-PP
+failing VBG B-NP
+. . O
+Central JJ O
+had VBD B-VP
+assets NNS B-NP
+of IN B-PP
+28.3 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+. . O
+The DT B-NP
+FDIC JJ I-NP
+said VBD B-VP
+the DT B-NP
+deposits NN I-NP
+of IN B-PP
+the DT B-NP
+failed NN I-NP
+Farmers NNS I-NP
+State VBD B-VP
+Bank NNP B-NP
+, , O
+Hart NNP B-NP
+, , O
+Tex NNP B-NP
+. . O
+, , O
+were VBD B-VP
+assumed VBN I-VP
+by IN B-PP
+Hale NNP B-NP
+County NNP I-NP
+State NNP I-NP
+Bank NNP I-NP
+, , O
+Plainview NNP B-NP
+, , O
+Tex NNP B-NP
+. . O
+Farmers NNS B-NP
+, , O
+with IN B-PP
+9.6 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+in IN B-PP
+assets NN B-NP
+, , O
+was VBD B-VP
+closed VBN I-VP
+by IN B-PP
+Texas NNP B-NP
+bank NN I-NP
+regulators NNS I-NP
+. . O
+The DT B-NP
+deposits NN I-NP
+of IN B-PP
+the DT B-NP
+failed NN I-NP
+First IN B-PP
+National JJ B-NP
+Bank NNP I-NP
+of IN B-PP
+Crosby NNP B-NP
+, , O
+Crosby NNP B-NP
+, , O
+Tex NNP B-NP
+. . O
+, , O
+with IN B-PP
+total JJ B-NP
+assets NNS I-NP
+of IN B-PP
+8.2 CD B-NP
+mln NN I-NP
+dlrs NN I-NP
+, , O
+were VBD B-VP
+assumed VBN I-VP
+by IN B-PP
+Central JJ B-NP
+Bancshares NNS I-NP
+of IN B-PP
+the DT B-NP
+South NNP I-NP
+Inc NNP I-NP
+, , O
+Birmingham NNP B-NP
+, , O
+Ala. NNP B-NP
+, , O
+after IN B-PP
+First JJ B-NP
+National JJ I-NP
+was VBD I-NP
+closed VBN B-VP
+by IN B-PP
+federal JJ B-NP
+bank NN I-NP
+regulators NNS I-NP
+, , O
+the DT B-NP
+FDIC NNP I-NP
+said VBD B-VP
+. . O
+Brazil's JJ O
+14-bank NN B-NP
+advisory NN I-NP
+committee NNP-named I-NP
+expressed VBN B-VP
+" IN B-PP
+grave JJ B-NP
+concern NN I-NP
+" NN I-NP
+to TO B-PP
+chief NNP B-NP
+debt VBD B-VP
+negotiator NN B-NP
+Antonio RB B-ADVP
+Padua NNP-5 B-NP
+de VBD B-VP
+Seixas NNS B-NP
+over IN B-PP
+the DT B-NP
+country's NN I-NP
+suspension NN I-NP
+of IN B-PP
+interest JJ B-NP
+payments NNS I-NP
+, , O
+according IN B-PP
+to TO B-PP
+a DT B-NP
+telex NN I-NP
+from IN B-PP
+committee NN B-NP
+chairman NN I-NP
+Citibank NNP I-NP
+to TO B-PP
+creditor NN B-NP
+banks NNS I-NP
+worldwide VBD B-VP
+. . B-NP
+Bankers NNS I-NP
+said VBD B-VP
+the DT B-NP
+diplomatic NN I-NP
+phrase NN I-NP
+belied VBN B-VP
+the DT B-NP
+deep NN I-NP
+anger IN B-PP
+and CC O
+frustration VBN B-VP
+on IN B-PP
+the DT B-NP
+committee NN I-NP
+over IN B-PP
+Brazil's NNP B-NP
+unilateral JJ I-NP
+move NN I-NP
+last JJ B-NP
+Friday NNP I-NP
+and CC O
+its NNS B-NP
+subsequent JJ I-NP
+freeze NNS I-NP
+on IN B-PP
+some DT B-NP
+15 CD I-NP
+billion NN I-NP
+dlrs NN I-NP
+of IN B-PP
+short-term NN B-NP
+trade NN I-NP
+and CC O
+interbank RB B-NP
+lines NNS I-NP
+. . O
+Seixas NNS B-NP
+, , O
+director NNS B-NP
+of IN B-PP
+the DT B-NP
+Brazilian JJ I-NP
+central NN I-NP
+bank's NNS I-NP
+foreign IN B-PP
+debt NN B-NP
+department NN I-NP
+, , O
+met IN B-PP
+the DT B-NP
+full JJ I-NP
+panel NN I-NP
+on IN B-PP
+Tuesday NNP B-NP
+and CC I-NP
+Wednesday NNP B-NP
+. . O
+Seixas NNS B-NP
+, , O
+who NNS B-NP
+met NN I-NP
+again IN B-PP
+this DT B-NP
+morning VBG I-NP
+with IN B-PP
+senior NNP B-NP
+Citibank NNP I-NP
+executive JJ I-NP
+William . I-NP
+Rhodes NNS I-NP
+and CC I-NP
+representatives NNS I-NP
+from IN B-PP
+committee NN B-NP
+vice-chairmen VBN I-NP
+Morgan NNP I-NP
+Guaranty NNP I-NP
+Trust JJ I-NP
+Co NNP I-NP
+and CC I-NP
+Lloyds NNP I-NP
+Bank NNP I-NP
+Plc NNP I-NP
+, , O
+told JJ B-NP
+the DT I-NP
+banks NNS I-NP
+that IN B-PP
+the DT B-NP
+government NN I-NP
+was VBD B-VP
+preparing VBG I-VP
+a DT B-NP
+telex NN I-NP
+to TO B-PP
+explain NNS B-NP
+and CC O
+clarify VB B-VP
+the DT B-NP
+freeze NN I-NP
+on IN B-PP
+short-term JJ B-NP
+credits NN I-NP
+. . O
+The DT B-NP
+telex NN I-NP
+could VBN B-VP
+be VB I-VP
+sent NN B-NP
+to TO B-PP
+creditors NN B-NP
+as IN B-PP
+early NN B-NP
+as IN B-PP
+today NN B-NP
+, , O
+bankers NNS B-NP
+said VBD B-VP
+. . O
+Despite JJ O
+the DT B-NP
+rising VBG I-NP
+tempers NNS I-NP
+, , O
+bankers NNS B-NP
+said VBD B-VP
+there EX B-NP
+are VBP B-VP
+no RB I-VP
+plans VBN I-VP
+for IN B-PP
+Brazilian NNP B-NP
+finance NN I-NP
+minister NN I-NP
+Dilson NNP I-NP
+Funaro NNP I-NP
+to TO B-PP
+meet JJ B-NP
+commercial NN I-NP
+bankers NNS I-NP
+during IN B-PP
+his NNS B-NP
+trip VBD B-VP
+to TO B-PP
+Washington NNP B-NP
+on IN B-PP
+Friday NNP B-NP
+and CC O
+Saturday NNP B-NP
+. . I-NP
+Funaro NNP I-NP
+will MD B-VP
+be VB I-VP
+explaining VBG I-VP
+Brazil's NNS B-NP
+actions VBD B-VP
+to TO B-PP
+U.S. NNP B-NP
+Treasury NNP I-NP
+Secretary NNP I-NP
+James NNP I-NP
+Baker NNP I-NP
+, , O
+Federal JJ B-NP
+Reserve . I-NP
+Board NNP I-NP
+chairman NN I-NP
+Paul default I-NP
+Volcker NNP I-NP
+and CC I-NP
+International JJ I-NP
+Monetary NNP I-NP
+Fund NNP I-NP
+managing VBG B-VP
+director IN B-PP
+Michel default B-NP
+Camdessus NNP I-NP
+before NN I-NP
+travelling IN B-NP
+to TO B-VP
+Europe VB I-VP
+at IN B-PP
+the DT B-NP
+weekend JJ I-NP
+. . O
+Sentence NN B-NP
+number NN I-NP
+1 CD I-NP
+has VBZ B-VP
+6 CD I-NP
+words NNS I-NP
+. . O
+Sentence NN B-NP
+number NN I-NP
+2 CD I-NP
+, , O
+5 CD B-NP
+words NNS I-NP
+. . O

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/lemmas.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/lemmas.txt b/lucene/analysis/opennlp/src/tools/test-model-data/lemmas.txt
new file mode 100644
index 0000000..be02176
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/lemmas.txt
@@ -0,0 +1,875 @@
+Showers	NNS	shower
+continued	VBD	continue
+throughout	IN	throughout
+the	DT	the
+week	NN	week
+in	IN	in
+the	DT	the
+Bahia	NNP	bahia
+cocoa	NN	cocoa
+zone	NN	zone
+,	,	,
+alleviating	VBG	alleviate
+the	DT	the
+drought	NN	drought
+since	IN	since
+early	JJ	early
+January	NNP	january
+and	CC	and
+improving	VBG	improve
+prospects	NNS	prospect
+for	IN	for
+the	DT	the
+coming	VBG	come
+temporao	NN	temporao
+,	,	,
+although	IN	although
+normal	JJ	normal
+humidity	NN	humidity
+levels	NNS	level
+have	VBP	have
+not	RB	not
+been	VBN	be
+restored	VBN	restore
+,	,	,
+Comissaria	NNP	comissaria
+Smith	NNP	smith
+said	VBD	say
+in	IN	in
+its	PRP$	its
+weekly	JJ	weekly
+review	NN	review
+.	.	.
+
+The	DT	the
+dry	JJ	dry
+period	NN	period
+means	VBZ	mean
+the	DT	the
+temporao	NN	temporao
+will	MD	will
+be	VB	be
+late	RB	late
+this	DT	this
+year	NN	year
+.	.	.
+
+Arrivals	NNS	arrival
+for	IN	for
+the	DT	the
+week	NN	week
+ended	VBN	end
+February	NNP	february
+22	CD	22
+were	VBD	be
+155	CD	155
+bags	NNS	bag
+of	IN	of
+60	CD	60
+kilos	NN	kilo
+making	VBG	make
+a	DT	a
+cumulative	JJ	cumulative
+total	NN	total
+for	IN	for
+the	DT	the
+season	NN	season
+of	IN	of
+5	CD	5
+mln	NN	mln
+against	IN	against
+5	CD	5
+at	IN	at
+the	DT	the
+same	JJ	same
+stage	NN	stage
+last	JJ	last
+year	NN	year
+.	.	.
+
+Again	RB	again
+it	PRP	it
+seems	VBZ	seem
+that	IN	that
+cocoa	NN	cocoa
+delivered	VBN	deliver
+earlier	RBR	early
+on	IN	on
+consignment	NN	consignment
+was	VBD	be
+included	VBN	include
+in	IN	in
+the	DT	the
+arrivals	NNS	arrival
+figures	NNS	figure
+.	.	.
+
+Comissaria	NNP	comissaria
+Smith	NNP	smith
+said	VBD	say
+there	EX	there
+is	VBZ	be
+still	RB	still
+some	DT	some
+doubt	NN	doubt
+as	IN	as
+to	TO	to
+how	WRB	how
+much	JJ	much
+old	JJ	old
+crop	NN	crop
+cocoa	NN	cocoa
+is	VBZ	be
+still	RB	still
+available	JJ	available
+as	IN	as
+harvesting	NN	harvesting
+has	VBZ	has
+practically	RB	practically
+come	VBN	come
+to	TO	to
+an	DT	an
+end	NN	end
+.	.	.
+
+With	IN	with
+total	JJ	total
+Bahia	NNP	bahia
+crop	NN	crop
+estimates	NNS	estimate
+around	IN	around
+6	CD	6
+mln	NN	mln
+bags	NNS	bag
+and	CC	and
+sales	NNS	sale
+standing	VBG	stand
+at	IN	at
+almost	RB	almost
+6	CD	6
+mln	NN	mln
+there	EX	there
+are	VBP	are
+a	DT	a
+few	JJ	few
+hundred	CD	hundred
+thousand	CD	thousand
+bags	NNS	bag
+still	RB	still
+in	IN	in
+the	DT	the
+hands	NNS	hand
+of	IN	of
+farmers	NNS	farmer
+,	,	,
+middlemen	NNS	middleman
+,	,	,
+exporters	NNS	exporter
+and	CC	and
+processors	NNS	processor
+.	.	.
+
+There	EX	there
+are	VBP	are
+doubts	NNS	doubt
+as	IN	as
+to	TO	to
+how	WRB	how
+much	RB	much
+of	IN	of
+this	DT	this
+cocoa	NN	cocoa
+would	MD	would
+be	VB	be
+fit	NN	fit
+for	IN	for
+export	NN	export
+as	IN	as
+shippers	NNS	shipper
+are	VBP	are
+now	RB	now
+experiencing	VBG	experience
+dificulties	NNS	dificulty
+in	IN	in
+obtaining	VBG	obtain
++	+	+
+Bahia	NNP	bahia
+superior	JJ	superior
++	+	+
+certificates	NNS	certificate
+.	.	.
+
+In	IN	in
+view	NN	view
+of	IN	of
+the	DT	the
+lower	JJR	low
+quality	NN	quality
+over	IN	over
+recent	JJ	recent
+weeks	NNS	week
+farmers	NNS	farmer
+have	VBP	have
+sold	VBN	sold
+a	DT	a
+good	JJ	good
+part	NN	part
+of	IN	of
+their	PRP$	their
+cocoa	NN	cocoa
+held	VBN	held
+on	IN	on
+consignment	NN	consignment
+.	.	.
+
+Comissaria	NNP	comissaria
+Smith	NNP	smith
+said	VBD	say
+spot	NN	spot
+bean	NN	bean
+prices	NNS	price
+rose	VBD	rise
+to	TO	to
+340	CD	340
+to	TO	to
+350	CD	350
+cruzados	NN	cruzado
+per	IN	per
+arroba	NN	arroba
+of	IN	of
+15	CD	15
+kilos	NN	kilo
+.	.	.
+
+Bean	NNP	bean
+shippers	NNS	shipper
+were	VBD	be
+reluctant	JJ	reluctant
+to	TO	to
+offer	VB	offer
+nearby	JJ	nearby
+shipment	NN	shipment
+and	CC	and
+only	RB	only
+limited	JJ	limited
+sales	NNS	sale
+were	VBD	be
+booked	VBN	book
+for	IN	for
+March	NNP	march
+shipment	NN	shipment
+at	IN	at
+1	CD	1
+to	TO	to
+1	CD	1
+dlrs	NNS	dlr
+per	IN	per
+tonne	NN	tonne
+to	TO	to
+ports	NNS	port
+to	TO	to
+be	VB	be
+named	VBN	name
+.	.	.
+
+New	JJ	new
+crop	NN	crop
+sales	NNS	sale
+were	VBD	be
+also	RB	also
+light	JJ	light
+and	CC	and
+all	DT	all
+to	TO	to
+open	JJ	open
+ports	NNS	port
+with	IN	with
+June	NNP	june
+/	/	/
+July	NNP	july
+going	VBG	go
+at	IN	at
+1	CD	1
+and	CC	and
+1	CD	1
+dlrs	NNS	dlr
+and	CC	and
+at	IN	at
+35	CD	35
+and	CC	and
+45	CD	45
+dlrs	NNS	dlr
+under	IN	under
+New	NNP	New
+York	NNP	York
+july	NN	july
+,	,	,
+Aug	NNP	Aug
+/	/	/
+Sept	NNP	Sept
+at	IN	at
+1	CD	1
+,	,	,
+1	CD	1
+and	CC	and
+1	CD	1
+dlrs	NNS	dlr
+per	IN	per
+tonne	NN	tonne
+FOB	NNP	FOB
+.	.	.
+
+Routine	JJ	routine
+sales	NNS	sale
+of	IN	of
+butter	NN	butter
+were	VBD	be
+made	VBN	make
+.	.	.
+
+March	NNP	march
+/	/	/
+April	NNP	april
+sold	VBD	sell
+at	IN	at
+4	CD	4
+,	,	,
+4	CD	4
+and	CC	and
+4	CD	4
+dlrs	NNS	dlr
+.	.	.
+
+April	NNP	april
+/	/	/
+May	NNP	may
+butter	NN	butter
+went	VBD	went
+at	IN	at
+2	CD	2
+times	NNS	time
+New	NNP	new
+York	NNP	york
+May	NNP	may
+,	,	,
+June	NNP	june
+/	/	/
+July	NNP	july
+at	IN	at
+4	CD	4
+and	CC	and
+4	CD	4
+dlrs	NNS	dlr
+,	,	,
+Aug	NNP	aug
+/	/	/
+Sept	NNP	sept
+at	IN	at
+4	CD	4
+to	TO	to
+4	CD	4
+dlrs	NNS	dlr
+and	CC	and
+at	IN	at
+2	CD	2
+and	CC	and
+2	CD	2
+times	NNS	time
+New	NNP	new
+York	NNP	york
+Sept	NNP	sept
+and	CC	and
+Oct	NNP	oct
+/	/	/
+Dec	NNP	dec
+at	IN	at
+4	CD	4
+dlrs	NNS	dlr
+and	CC	and
+2	CD	2
+times	NNS	time
+New	NNP	new
+York	NNP	york
+Dec	NNP	dec
+,	,	,
+Comissaria	NNP	comissaria
+Smith	NNP	smith
+said	VBD	say
+.	.	.
+
+Destinations	NNS	destination
+were	VBD	be
+the	DT	the
+U.S.	NNP	u.s.
+,	,	,
+Covertible	JJ	covertible
+currency	NN	currency
+areas	NNS	area
+,	,	,
+Uruguay	NNP	uruguay
+and	CC	and
+open	JJ	open
+ports	NNS	port
+.	.	.
+
+Cake	NNP	cake
+sales	NNS	sale
+were	VBD	be
+registered	VBN	register
+at	IN	at
+785	CD	785
+to	TO	to
+995	CD	995
+dlrs	NNS	dlr
+for	IN	for
+March	NNP	march
+/	/	/
+April	NNP	april
+,	,	,
+785	CD	785
+dlrs	NNS	dlr
+for	IN	for
+May	NNP	may
+,	,	,
+753	CD	753
+dlrs	NNS	dlr
+for	IN	for
+Aug	NNP	aug
+and	CC	and
+0	CD	0
+times	NNS	time
+New	NNP	new
+York	NNP	york
+Dec	NNP	dec
+for	IN	for
+Oct	NNP	oct
+/	/	/
+Dec	NNP	dec
+.	.	.
+
+Buyers	NNS	buyer
+were	VBD	be
+the	DT	the
+U.S.	NNP	u.s.
+,	,	,
+Argentina	NNP	argentina
+,	,	,
+Uruguay	NNP	uruguay
+and	CC	and
+convertible	JJ	convertible
+currency	NN	currency
+areas	NNS	area
+.	.	.
+
+Liquor	NNP	liquor
+sales	NNS	sale
+were	VBD	be
+limited	VBN	limit
+with	IN	with
+March	NNP	march
+/	/	/
+April	NNP	april
+selling	VBG	sell
+at	IN	at
+2	CD	2
+and	CC	and
+2	CD	2
+dlrs	NNS	dlr
+,	,	,
+June	NNP	june
+/	/	/
+July	NNP	july
+at	IN	at
+2	CD	2
+dlrs	NNS	dlr
+and	CC	and
+at	IN	at
+1	CD	1
+times	NNS	time
+New	NNP	new
+York	NNP	york
+July	NNP	july
+,	,	,
+Aug	NNP	aug
+/	/	/
+Sept	NNP	sept
+at	IN	at
+2	CD	2
+dlrs	NNS	dlr
+and	CC	and
+at	IN	at
+1	CD	1
+times	NNS	time
+New	NNP	new
+York	NNP	york
+Sept	NNP	sept
+and	CC	and
+Oct	NNP	oct
+/	/	/
+Dec	NNP	dec
+at	IN	at
+1	CD	1
+times	NNS	time
+New	NNP	new
+York	NNP	york
+Dec	NNP	dec
+,	,	,
+Comissaria	NNP	comissaria
+Smith	NNP	smith
+said	VBD	say
+.	.	.
+
+Total	JJ	total
+Bahia	NN	bahia
+sales	NNS	sale
+are	VBP	be
+currently	RB	currently
+estimated	VBN	estimate
+at	IN	at
+6	CD	6
+mln	NN	mln
+bags	NNS	bag
+against	IN	against
+the	DT	the
+1986/87	CD	1986/87
+crop	NN	crop
+and	CC	and
+1	CD	1
+mln	NN	mln
+bags	NNS	baga
+against	IN	against
+the	DT	the
+1987/88	CD	1987/88
+crop	NN	crop
+.	.	.
+
+Final	JJ	final
+figures	NNS	figure
+for	IN	for
+the	DT	the
+period	NN	period
+to	TO	to
+February	NNP	february
+28	CD	28
+are	VBP	be
+expected	VBN	expect
+to	TO	to
+be	VB	be
+published	VBN	publish
+by	IN	by
+the	DT	the
+Brazilian	JJ	brazilian
+Cocoa	NNP	cocoa
+Trade	NNP	trade
+Commission	NNP	commission
+after	IN	after
+carnival	NN	carnival
+which	WDT	which
+ends	VBZ	end
+midday	NN	midday
+on	IN	on
+February	NNP	february
+27	CD	27
+.	.	.
+
+Iran	NNP	iran
+announced	VBD	announce
+tonight	NN	tonight
+that	IN	that
+its	PRP$	its
+major	JJ	major
+offensive	NN	offensive
+against	IN	against
+Iraq	NNP	iraq
+in	IN	in
+the	DT	the
+Gulf	NNP	gulf
+war	NN	war
+had	VBD	have
+ended	VBN	end
+after	IN	after
+dealing	VBG	deal
+savage	JJ	savage
+blows	NNS	blow
+against	IN	against
+the	DT	the
+Baghdad	NNP	baghdad
+government	NN	government
+.	.	.
+
+The	DT	the
+Iranian	JJ	iranian
+news	NN	news
+agency	NN	agency
+IRNA	NNP	irna
+,	,	,
+in	IN	in
+a	DT	a
+report	NN	report
+received	VBN	receive
+in	IN	in
+London	NNP	London
+,	,	,
+said	VBD	say
+the	DT	the
+operation	NN	operation
+code	NNP-named	code
+Karbala-5	NNP	karbala-5
+launched	VBD	launch
+into	IN	into
+Iraq	NNP	iraq
+on	IN	on
+January	NNP	january
+9	CD	9
+was	VBD	be
+now	RB	now
+over	RP	over
+.	.	.
+
+It	PRP	it
+quoted	VBD	quote
+a	DT	a
+joint	NN	joint
+statewment	NN	statement
+by	IN	by
+the	DT	the
+Iranian	JJ	iranian
+Army	NNP	army
+and	CC	and
+Revolutionary	NNP	revolutionary
+Guards	NNPS	guards
+Corps	NNP	corps
+as	IN	as
+saying	VBG	say
+that	IN	that
+their	PRP$	their
+forces	NNS	force
+had	VBD	have
+dealt	VBD	deal
+one	CD	one
+of	IN	of
+the	DT	the
+severest	JJS	severe
+blows	NNS	blow
+on	IN	on
+the	DT	the
+Iraqi	JJ	iraqi
+war	NN	war
+machine	NN	machine
+in	IN	in
+the	DT	the
+history	NN	history
+of	IN	of
+the	DT	the
+Iraq-imposed	JJ	iraq-imposed
+war	NN	war
+.	.	.
+
+The	DT	the
+statement	NN	statement
+by	IN	by
+the	DT	the
+Iranian	JJ	iranian
+High	NNP	high
+Command	NNP	command
+appeared	VBD	appear
+to	TO	to
+herald	VB	herald
+the	DT	the
+close	NN	close
+of	IN	of
+an	DT	an
+assault	NN	assault
+on	IN	on
+the	DT	the
+port	JJ	port
+city	NN	city
+of	IN	of
+Basra	NNP	basra
+in	IN	in
+southern	JJ	southern
+Iraq	NNP	iraq
+.	.	.
+
+The	DT	the
+operation	NN	operation
+was	VBD	be
+launched	VBN	launch
+at	IN	at
+a	DT	a
+time	NN	time
+when	WRB	when
+the	DT	the
+Baghdad	NNP	baghdad
+government	NN	government
+was	VBD	be
+spreading	VBG	spread
+extensive	JJ	extensive
+propaganda	NN	propaganda
+on	IN	on
+the	DT	the
+resistance	NN	resistance
+power	NN	power
+of	IN	of
+its	PRP$	its
+army	NN	army
+:	...	:
+,	,	,
+said	VBD	say
+the	DT	the
+statement	NN	statement
+quoted	VBN	quot
+by	IN	by
+IRNA	NNP	irna
+.	.	.
+
+It	PRP	it
+claimed	VBD	claim
+massive	JJ	massive
+victories	NNS	victory
+in	IN	in
+the	DT	the
+seven-week	NN	seven-week
+offensive	JJ	offensive
+and	CC	and
+called	VBN	call
+on	IN	on
+supporters	NNS	supporter
+of	IN	of
+Baghdad	NNP	baghdad
+to	TO	to
+come	VB	come
+to	TO	to
+their	PRP$	their
+senses	NNS	sense
+and	CC	and
+discontinue	VB	discontinue
+support	NN	support
+for	IN	for
+what	WP	what
+it	PRP	it
+called	VBD	called
+the	DT	the
+tottering	VBG	totter
+regime	NN	regime
+in	IN	in
+Iraq	NNP	iraq
+.	.	.
+
+Iran	NNP	iran
+said	VBD	say
+its	PRP$	its
+forces	NNS	force
+had	VBD	have
+liberated	JJ	liberate
+155	CD	155
+square	JJ	square
+kilometers	NNS	kilometer
+of	IN	of
+enemy-occupied	JJ	enemy-occupied
+territory	NN	territory
+during	IN	during
+the	DT	the
+1987	CD	1987
+offensive	NN	offensive
+and	CC	and
+taken	VBN	take
+over	IN	over
+islands	NNS	island
+,	,	,
+townships	NNS	township
+,	,	,
+rivers	NNS	river
+and	CC	and
+part	NN	part
+of	IN	of
+a	DT	a
+road	NN	road
+leading	VBG	lead
+into	IN	into
+Basra	NNP	basra
+.	.	.
+
+The	DT	the
+Iranian	JJ	iranian
+forces	NNS	force
+are	VBP	be
+in	IN	in
+full	JJ	full
+control	NN	control
+of	IN	of
+these	DT	these
+areas	NNS	area
+,	,	,
+the	DT	the
+statement	NN	statement
+said	VBD	say
+.	.	.
+
+It	PRP	it
+said	VBD	say
+81	CD	81
+Iraqi	JJ	iraqi
+brigades	NNS	brigade
+and	CC	and
+battalions	NNS	battalion
+were	VBD	be
+totally	RB	totally
+destroyed	VBN	destroy
+,	,	,
+along	IN	along
+with	IN	with
+700	CD	700
+tanks	NNS	tank
+and	CC	and
+1	CD	1
+other	JJ	other
+vehicles	NNS	vehicle
+.	.	.
+
+The	DT	the
+victory	NN	victory
+list	NN	list
+also	RB	also
+included	VBD	include
+80	CD	80
+warplanes	NNS	warplane
+downed	VBD	down
+,	,	,
+250	CD	250
+anti	NN	anti
+:	-	:
+aircraft	NN	aircraft
+guns	NNS	gun
+and	CC	and
+400	CD	400
+pieces	NNS	piece
+of	IN	of
+military	JJ	military
+hardware	NN	hardware
+destroyed	VBN	destroy
+and	CC	and
+the	DT	the
+seizure	NN	seizure
+of	IN	of
+220	CD	220
+tanks	NNS	tank
+and	CC	and
+armoured	JJ	armoured
+personnel	NNS	personnel
+carriers	NNS	carrier
+.	.	.
+They	NNP	they
+sent	VBD	send
+him	PRP	he
+running	VBG	run
+in	IN	in
+the	DT	the
+evening	NN	evening
+.	.	.
+He	PRP	he
+did	VBD	do
+not	RB	not
+come	VB	come
+back	RB	back
+.	.	.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/ner_TrainerParams.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/ner_TrainerParams.txt b/lucene/analysis/opennlp/src/tools/test-model-data/ner_TrainerParams.txt
new file mode 100644
index 0000000..e01ad50
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/ner_TrainerParams.txt
@@ -0,0 +1,21 @@
+# 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.
+
+# maxent won't work on small training set. Use perceptron, train on one word.
+
+Algorithm=PERCEPTRON
+Iterations=200
+Cutoff=5
+Threads=2


[04/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
new file mode 100644
index 0000000..a96a1d5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
@@ -0,0 +1,267 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simulated {@link NodeStateProvider}.
+ * Note: in order to setup node-level metrics use {@link #simSetNodeValues(String, Map)}. However, in order
+ * to setup core-level metrics use {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)}.
+ */
+public class SimNodeStateProvider implements NodeStateProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, Map<String, Object>> nodeValues = new ConcurrentHashMap<>();
+  private final SimClusterStateProvider clusterStateProvider;
+  private final SimDistribStateManager stateManager;
+  private final LiveNodesSet liveNodesSet;
+
+  public SimNodeStateProvider(LiveNodesSet liveNodesSet, SimDistribStateManager stateManager,
+                              SimClusterStateProvider clusterStateProvider,
+                              Map<String, Map<String, Object>> nodeValues) {
+    this.liveNodesSet = liveNodesSet;
+    this.stateManager = stateManager;
+    this.clusterStateProvider = clusterStateProvider;
+    if (nodeValues != null) {
+      this.nodeValues.putAll(nodeValues);
+    }
+  }
+
+  // -------- simulator setup methods ------------
+
+  /**
+   * Get a node value
+   * @param node node id
+   * @param key property name
+   * @return property value or null if property or node doesn't exist.
+   */
+  public Object simGetNodeValue(String node, String key) {
+    Map<String, Object> values = nodeValues.get(node);
+    if (values == null) {
+      return null;
+    }
+    return values.get(key);
+  }
+
+  /**
+   * Set node values.
+   * NOTE: if values contain 'nodeRole' key then /roles.json is updated.
+   * @param node node id
+   * @param values values.
+   */
+  public void simSetNodeValues(String node, Map<String, Object> values) {
+    Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
+    existing.clear();
+    if (values != null) {
+      existing.putAll(values);
+    }
+    if (values == null || values.isEmpty() || values.containsKey("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Set a node value, replacing any previous value.
+   * NOTE: if key is 'nodeRole' then /roles.json is updated.
+   * @param node node id
+   * @param key property name
+   * @param value property value
+   */
+  public void simSetNodeValue(String node, String key, Object value) {
+    Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
+    if (value == null) {
+      existing.remove(key);
+    } else {
+      existing.put(key, value);
+    }
+    if (key.equals("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Add a node value, creating a list of values if necessary.
+   * NOTE: if key is 'nodeRole' then /roles.json is updated.
+   * @param node node id
+   * @param key property name
+   * @param value property value.
+   */
+  public void simAddNodeValue(String node, String key, Object value) {
+    Map<String, Object> values = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
+    Object existing = values.get(key);
+    if (existing == null) {
+      values.put(key, value);
+    } else if (existing instanceof Set) {
+      ((Set)existing).add(value);
+    } else {
+      Set<Object> vals = new HashSet<>();
+      vals.add(existing);
+      vals.add(value);
+      values.put(key, vals);
+    }
+    if (key.equals("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Remove node values. If values contained a 'nodeRole' key then
+   * /roles.json is updated.
+   * @param node node id
+   */
+  public void simRemoveNodeValues(String node) {
+    Map<String, Object> values = nodeValues.remove(node);
+    if (values != null && values.containsKey("nodeRole")) {
+      saveRoles();
+    }
+  }
+
+  /**
+   * Get all node values.
+   */
+  public Map<String, Map<String, Object>> simGetAllNodeValues() {
+    return nodeValues;
+  }
+
+  private synchronized void saveRoles() {
+    final Map<String, Set<String>> roles = new HashMap<>();
+    nodeValues.forEach((n, values) -> {
+      String nodeRole = (String)values.get("nodeRole");
+      if (nodeRole != null) {
+        roles.computeIfAbsent(nodeRole, role -> new HashSet<>()).add(n);
+      }
+    });
+    try {
+      stateManager.setData(ZkStateReader.ROLES, Utils.toJSON(roles), -1);
+    } catch (Exception e) {
+      throw new RuntimeException("Unexpected exception saving roles " + roles, e);
+    }
+  }
+
+  /**
+   * Simulate getting replica metrics values. This uses per-replica properties set in
+   * {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)} and
+   * similar methods.
+   * @param node node id
+   * @param tags metrics names
+   * @return map of metrics names / values
+   */
+  public Map<String, Object> getReplicaMetricsValues(String node, Collection<String> tags) {
+    List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
+    if (replicas == null || replicas.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> values = new HashMap<>();
+    for (String tag : tags) {
+      String[] parts = tag.split(":");
+      if (parts.length < 3 || !parts[0].equals("metrics")) {
+        LOG.warn("Invalid metrics: tag: " + tag);
+        continue;
+      }
+      if (!parts[1].startsWith("solr.core.")) {
+        // skip - this is probably solr.node or solr.jvm metric
+        continue;
+      }
+      String[] collParts = parts[1].substring(10).split("\\.");
+      if (collParts.length != 3) {
+        LOG.warn("Invalid registry name: " + parts[1]);
+        continue;
+      }
+      String collection = collParts[0];
+      String shard = collParts[1];
+      String replica = collParts[2];
+      String key = parts.length > 3 ? parts[2] + ":" + parts[3] : parts[2];
+      replicas.forEach(r -> {
+        if (r.getCollection().equals(collection) && r.getShard().equals(shard) && r.getCore().endsWith(replica)) {
+          Object value = r.getVariables().get(key);
+          if (value != null) {
+            values.put(tag, value);
+          } else {
+            value = r.getVariables().get(tag);
+            if (value != null) {
+              values.put(tag, value);
+            }
+          }
+        }
+      });
+    }
+    return values;
+  }
+
+  // ---------- interface methods -------------
+
+  @Override
+  public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
+    LOG.trace("-- requested values for " + node + ": " + tags);
+    if (!liveNodesSet.contains(node)) {
+      nodeValues.remove(node);
+      return Collections.emptyMap();
+    }
+    if (tags.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> result = new HashMap<>();
+    Map<String, Object> metrics = getReplicaMetricsValues(node, tags.stream().filter(s -> s.startsWith("metrics:solr.core.")).collect(Collectors.toList()));
+    result.putAll(metrics);
+    Map<String, Object> values = nodeValues.get(node);
+    if (values == null) {
+      return result;
+    }
+    result.putAll(values.entrySet().stream().filter(e -> tags.contains(e.getKey())).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
+    return result;
+  }
+
+  @Override
+  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+    List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
+    if (replicas == null || replicas.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map<String, Map<String, List<ReplicaInfo>>> res = new HashMap<>();
+    // TODO: probably needs special treatment for "metrics:solr.core..." tags
+    for (ReplicaInfo r : replicas) {
+      Map<String, List<ReplicaInfo>> perCollection = res.computeIfAbsent(r.getCollection(), s -> new HashMap<>());
+      List<ReplicaInfo> perShard = perCollection.computeIfAbsent(r.getShard(), s -> new ArrayList<>());
+      perShard.add(r);
+    }
+    return res;
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
new file mode 100644
index 0000000..be7209b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
@@ -0,0 +1,251 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.CollectionStatePredicate;
+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.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
+
+/**
+ * Base class for simulated test cases. Tests that use this class should configure the simulated cluster
+ * in <code>@BeforeClass</code> like this:
+ * <pre>
+ *   @BeforeClass
+ *   public static void setupCluster() throws Exception {
+ *     cluster = configureCluster(5, TimeSource.get("simTime:50"));
+ *   }
+ * </pre>
+ */
+public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int DEFAULT_TIMEOUT = 90;
+
+  /** The cluster. */
+  protected static SimCloudManager cluster;
+
+  protected static void configureCluster(int nodeCount, TimeSource timeSource) throws Exception {
+    cluster = SimCloudManager.createCluster(nodeCount, timeSource);
+  }
+
+  @AfterClass
+  public static void shutdownCluster() throws Exception {
+    if (cluster != null) {
+      cluster.close();
+    }
+    cluster = null;
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    if (cluster != null) {
+      // clear any persisted auto scaling configuration
+      cluster.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), -1);
+      // clean any persisted trigger state or events
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
+      removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+      cluster.getSimClusterStateProvider().simDeleteAllCollections();
+      cluster.simClearSystemCollection();
+      cluster.getSimClusterStateProvider().simResetLeaderThrottle();
+      cluster.simRestartOverseer(null);
+    }
+  }
+
+  @Before
+  public void checkClusterConfiguration() {
+    if (cluster == null)
+      throw new RuntimeException("SimCloudManager not configured - have you called configureCluster()?");
+  }
+
+  protected void removeChildren(String path) throws Exception {
+    if (!cluster.getDistribStateManager().hasData(path)) {
+      return;
+    }
+    List<String> children = cluster.getDistribStateManager().listData(path);
+    for (String c : children) {
+      if (cluster.getDistribStateManager().hasData(path + "/" + c)) {
+        try {
+          cluster.getDistribStateManager().removeData(path + "/" + c, -1);
+        } catch (NoSuchElementException e) {
+          // ignore
+        }
+      }
+    }
+  }
+
+  /* Cluster helper methods ************************************/
+
+  /**
+   * Get the collection state for a particular collection
+   */
+  protected DocCollection getCollectionState(String collectionName) throws IOException {
+    return cluster.getClusterStateProvider().getClusterState().getCollection(collectionName);
+  }
+
+  /**
+   * Wait for a particular collection state to appear in the cluster client's state reader
+   *
+   * This is a convenience method using the {@link #DEFAULT_TIMEOUT}
+   *
+   * @param message     a message to report on failure
+   * @param collection  the collection to watch
+   * @param predicate   a predicate to match against the collection state
+   */
+  protected long waitForState(String message, String collection, CollectionStatePredicate predicate) {
+    AtomicReference<DocCollection> state = new AtomicReference<>();
+    AtomicReference<Set<String>> liveNodesLastSeen = new AtomicReference<>();
+    try {
+      return waitForState(collection, DEFAULT_TIMEOUT, TimeUnit.SECONDS, (n, c) -> {
+        state.set(c);
+        liveNodesLastSeen.set(n);
+        return predicate.matches(n, c);
+      });
+    } catch (Exception e) {
+      throw new AssertionError(message + "\n" + "Live Nodes: " + liveNodesLastSeen.get() + "\nLast available state: " + state.get(), e);
+    }
+  }
+
+  /**
+   * Block until a CollectionStatePredicate returns true, or the wait times out
+   *
+   * Note that the predicate may be called again even after it has returned true, so
+   * implementors should avoid changing state within the predicate call itself.
+   *
+   * @param collection the collection to watch
+   * @param wait       how long to wait
+   * @param unit       the units of the wait parameter
+   * @param predicate  the predicate to call on state changes
+   * @return number of milliseconds elapsed
+   * @throws InterruptedException on interrupt
+   * @throws TimeoutException on timeout
+   * @throws IOException on watcher register / unregister error
+   */
+  public long waitForState(final String collection, long wait, TimeUnit unit, CollectionStatePredicate predicate)
+      throws InterruptedException, TimeoutException, IOException {
+    TimeOut timeout = new TimeOut(wait, unit, cluster.getTimeSource());
+    long timeWarn = timeout.timeLeft(TimeUnit.MILLISECONDS) / 4;
+    while (!timeout.hasTimedOut()) {
+      ClusterState state = cluster.getClusterStateProvider().getClusterState();
+      DocCollection coll = state.getCollectionOrNull(collection);
+      // due to the way we manage collections in SimClusterStateProvider a null here
+      // can mean that a collection is still being created but has no replicas
+      if (coll == null) { // does not yet exist?
+        timeout.sleep(50);
+        continue;
+      }
+      if (predicate.matches(state.getLiveNodes(), coll)) {
+        log.trace("-- predicate matched with state {}", state);
+        return timeout.timeElapsed(TimeUnit.MILLISECONDS);
+      }
+      timeout.sleep(50);
+      if (timeout.timeLeft(TimeUnit.MILLISECONDS) < timeWarn) {
+        log.trace("-- still not matching predicate: {}", state);
+      }
+    }
+    throw new TimeoutException();
+  }
+
+  /**
+   * Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
+   * number of shards and replicas
+   */
+  public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) {
+    return (liveNodes, collectionState) -> {
+      if (collectionState == null)
+        return false;
+      if (collectionState.getSlices().size() != expectedShards)
+        return false;
+      for (Slice slice : collectionState) {
+        int activeReplicas = 0;
+        for (Replica replica : slice) {
+          if (replica.isActive(liveNodes))
+            activeReplicas++;
+        }
+        if (activeReplicas != expectedReplicas)
+          return false;
+      }
+      return true;
+    };
+  }
+
+  /**
+   * Get a (reproducibly) random shard from a {@link DocCollection}
+   */
+  protected static Slice getRandomShard(DocCollection collection) {
+    List<Slice> shards = new ArrayList<>(collection.getActiveSlices());
+    if (shards.size() == 0)
+      fail("Couldn't get random shard for collection as it has no shards!\n" + collection.toString());
+    Collections.shuffle(shards, random());
+    return shards.get(0);
+  }
+
+  /**
+   * Get a (reproducibly) random replica from a {@link Slice}
+   */
+  protected static Replica getRandomReplica(Slice slice) {
+    List<Replica> replicas = new ArrayList<>(slice.getReplicas());
+    if (replicas.size() == 0)
+      fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
+    Collections.shuffle(replicas, random());
+    return replicas.get(0);
+  }
+
+  /**
+   * Get a (reproducibly) random replica from a {@link Slice} matching a predicate
+   */
+  protected static Replica getRandomReplica(Slice slice, Predicate<Replica> matchPredicate) {
+    List<Replica> replicas = new ArrayList<>(slice.getReplicas());
+    if (replicas.size() == 0)
+      fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
+    Collections.shuffle(replicas, random());
+    for (Replica replica : replicas) {
+      if (matchPredicate.test(replica))
+        return replica;
+    }
+    fail("Couldn't get random replica that matched conditions\n" + slice.toString());
+    return null;  // just to keep the compiler happy - fail will always throw an Exception
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java
new file mode 100644
index 0000000..396edea
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestClusterStateProvider.java
@@ -0,0 +1,221 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.ClusterState;
+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.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.Watcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test compares the cluster state of a real cluster and a simulated one.
+ */
+public class TestClusterStateProvider extends SolrCloudTestCase {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static int NODE_COUNT = 3;
+  private static boolean simulated;
+
+  private static SolrCloudManager cloudManager;
+
+  private static Collection<String> liveNodes;
+  private static Map<String, Object> clusterProperties;
+  private static AutoScalingConfig autoScalingConfig;
+  private static Map<String, Map<String, Map<String, List<ReplicaInfo>>>> replicas;
+  private static Map<String, Map<String, Object>> nodeValues;
+  private static ClusterState realState;
+
+  // set up a real cluster as the source of test data
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    simulated = random().nextBoolean();
+    LOG.info("####### Using simulated components? " + simulated);
+
+    configureCluster(NODE_COUNT)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 2, 0, 1)
+        .process(cluster.getSolrClient());
+    init();
+  }
+
+  @AfterClass
+  public static void closeCloudManager() throws Exception {
+    if (simulated && cloudManager != null) {
+      cloudManager.close();
+    }
+  }
+
+  private static void init() throws Exception {
+    SolrCloudManager realManager = cluster.getJettySolrRunner(cluster.getJettySolrRunners().size() - 1).getCoreContainer()
+        .getZkController().getSolrCloudManager();
+    liveNodes = realManager.getClusterStateProvider().getLiveNodes();
+    clusterProperties = realManager.getClusterStateProvider().getClusterProperties();
+    autoScalingConfig = realManager.getDistribStateManager().getAutoScalingConfig();
+    replicas = new HashMap<>();
+    nodeValues = new HashMap<>();
+    liveNodes.forEach(n -> {
+      replicas.put(n, realManager.getNodeStateProvider().getReplicaInfo(n, Collections.emptySet()));
+      nodeValues.put(n, realManager.getNodeStateProvider().getNodeValues(n, ImplicitSnitch.tags));
+    });
+    realState = realManager.getClusterStateProvider().getClusterState();
+
+    if (simulated) {
+      // initialize simulated provider
+      SimCloudManager simCloudManager = new SimCloudManager(TimeSource.get("simTime:10"));
+      simCloudManager.getSimClusterStateProvider().simSetClusterProperties(clusterProperties);
+      simCloudManager.getSimDistribStateManager().simSetAutoScalingConfig(autoScalingConfig);
+      nodeValues.forEach((n, values) -> {
+        simCloudManager.getSimNodeStateProvider().simSetNodeValues(n, values);
+      });
+      simCloudManager.getSimClusterStateProvider().simSetClusterState(realState);
+      ClusterState simState = simCloudManager.getClusterStateProvider().getClusterState();
+      assertClusterStateEquals(realState, simState);
+      cloudManager = simCloudManager;
+    } else {
+      cloudManager = realManager;
+    }
+  }
+
+  private static void assertClusterStateEquals(ClusterState one, ClusterState two) {
+    assertEquals(one.getLiveNodes(), two.getLiveNodes());
+    assertEquals(one.getCollectionsMap().keySet(), two.getCollectionsMap().keySet());
+    one.forEachCollection(oneColl -> {
+      DocCollection twoColl = two.getCollection(oneColl.getName());
+      Map<String, Slice> oneSlices = oneColl.getSlicesMap();
+      Map<String, Slice> twoSlices = twoColl.getSlicesMap();
+      assertEquals(oneSlices.keySet(), twoSlices.keySet());
+      oneSlices.forEach((s, slice) -> {
+        Slice sTwo = twoSlices.get(s);
+        for (Replica oneReplica : slice.getReplicas()) {
+          Replica twoReplica = sTwo.getReplica(oneReplica.getName());
+          assertNotNull(twoReplica);
+          assertEquals(oneReplica, twoReplica);
+        }
+      });
+    });
+  }
+
+  private String addNode() throws Exception {
+    JettySolrRunner solr = cluster.startJettySolrRunner();
+    String nodeId = solr.getNodeName();
+    if (simulated) {
+      ((SimCloudManager) cloudManager).getSimClusterStateProvider().simAddNode(nodeId);
+    }
+    return nodeId;
+  }
+
+  private String deleteNode() throws Exception {
+    String nodeId = cluster.getJettySolrRunner(0).getNodeName();
+    cluster.stopJettySolrRunner(0);
+    if (simulated) {
+      ((SimCloudManager) cloudManager).getSimClusterStateProvider().simRemoveNode(nodeId);
+    }
+    return nodeId;
+  }
+
+  private void setAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
+    cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getZkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH,
+        Utils.toJSON(cfg), -1, true);
+    if (simulated) {
+      ((SimCloudManager) cloudManager).getSimDistribStateManager().simSetAutoScalingConfig(cfg);
+    }
+  }
+
+  @Test
+  public void testAddRemoveNode() throws Exception {
+    Set<String> lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    List<String> liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
+    assertEquals(lastNodes.size(), liveNodes.size());
+    liveNodes.removeAll(lastNodes);
+    assertTrue(liveNodes.isEmpty());
+
+    String node = addNode();
+    cloudManager.getTimeSource().sleep(2000);
+    assertFalse(lastNodes.contains(node));
+    lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    assertTrue(lastNodes.contains(node));
+    liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
+    assertEquals(lastNodes.size(), liveNodes.size());
+    liveNodes.removeAll(lastNodes);
+    assertTrue(liveNodes.isEmpty());
+
+    node = deleteNode();
+    cloudManager.getTimeSource().sleep(2000);
+    assertTrue(lastNodes.contains(node));
+    lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    assertFalse(lastNodes.contains(node));
+    liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
+    assertEquals(lastNodes.size(), liveNodes.size());
+    liveNodes.removeAll(lastNodes);
+    assertTrue(liveNodes.isEmpty());  }
+
+  @Test
+  public void testAutoScalingConfig() throws Exception {
+    final CountDownLatch triggered = new CountDownLatch(1);
+    Watcher w = ev -> {
+      if (triggered.getCount() == 0) {
+        fail("already triggered once!");
+      }
+      triggered.countDown();
+    };
+    AutoScalingConfig cfg = cloudManager.getDistribStateManager().getAutoScalingConfig(w);
+    assertEquals(autoScalingConfig, cfg);
+    Preference p = new Preference(Collections.singletonMap("maximize", "freedisk"));
+    cfg = cfg.withPolicy(cfg.getPolicy().withClusterPreferences(Collections.singletonList(p)));
+    setAutoScalingConfig(cfg);
+    if (!triggered.await(10, TimeUnit.SECONDS)) {
+      fail("Watch should be triggered on update!");
+    }
+    AutoScalingConfig cfg1 = cloudManager.getDistribStateManager().getAutoScalingConfig(null);
+    assertEquals(cfg, cfg1);
+
+    // restore
+    setAutoScalingConfig(autoScalingConfig);
+    cfg1 = cloudManager.getDistribStateManager().getAutoScalingConfig(null);
+    assertEquals(autoScalingConfig, cfg1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
new file mode 100644
index 0000000..b7053d7d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
@@ -0,0 +1,357 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.ComputePlanAction;
+import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
+import org.apache.solr.cloud.autoscaling.TriggerAction;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.LogLevel;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
+
+/**
+ * Test for {@link ComputePlanAction}
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG;")
+public class TestComputePlanAction extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final AtomicBoolean fired = new AtomicBoolean(false);
+  private static final int NODE_COUNT = 1;
+  private static CountDownLatch triggerFiredLatch = new CountDownLatch(1);
+  private static final AtomicReference<Map> actionContextPropsRef = new AtomicReference<>();
+  private static final AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1, TimeSource.get("simTime:50"));
+  }
+
+  @Before
+  public void init() throws Exception {
+
+    fired.set(false);
+    triggerFiredLatch = new CountDownLatch(1);
+    actionContextPropsRef.set(null);
+
+    if (cluster.getClusterStateProvider().getLiveNodes().size() > NODE_COUNT) {
+      // stop some to get to original state
+      int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
+      for (int i = 0; i < numJetties - NODE_COUNT; i++) {
+        String node = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+        cluster.getSimClusterStateProvider().simRemoveNode(node);
+      }
+    }
+
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    SolrResponse rsp = cluster.request(req);
+    NamedList<Object> response = rsp.getResponse();
+    assertEquals(response.get("result").toString(), "success");
+
+    String setClusterPreferencesCommand = "{" +
+        "'set-cluster-preferences': [" +
+        "{'minimize': 'cores'}," +
+        "{'maximize': 'freedisk','precision': 100}]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
+    rsp = cluster.request(req);
+    response = rsp.getResponse();
+    assertEquals(response.get("result").toString(), "success");
+    cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_COOLDOWN_PERIOD_SECONDS));
+  }
+
+  @After
+  public void printState() throws Exception {
+    log.info("-------------_ FINAL STATE --------------");
+    log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
+    log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cluster.getClusterStateProvider().getClusterState();
+    for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
+      log.info("* Collection " + coll + " state: " + state.getCollection(coll));
+    }
+
+  }
+
+  @Test
+  public void testNodeLost() throws Exception  {
+    // let's start a node so that we have at least two
+    String node = cluster.simAddNode();
+    AssertingTriggerAction.expectedNode = node;
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '7s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeLost",
+        "conf",1, 2);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        "testNodeLost", clusterShape(1, 2));
+
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    log.debug("-- cluster state: {}", clusterState);
+    DocCollection collection = clusterState.getCollection("testNodeLost");
+    List<Replica> replicas = collection.getReplicas(node);
+    assertNotNull(replicas);
+    assertFalse(replicas.isEmpty());
+
+    // start another node because because when the other node goes away, the cluster policy requires only
+    // 1 replica per node and none on the overseer
+    String node2 = cluster.simAddNode();
+    assertTrue(node2 + "is not live yet", cluster.getClusterStateProvider().getClusterState().liveNodesContain(node2) );
+
+    // stop the original node
+    cluster.simRemoveNode(node, false);
+    log.info("Stopped_node : {}", node);
+
+    assertTrue("Trigger was not fired even after 10 seconds", triggerFiredLatch.await(10, TimeUnit.SECONDS));
+    assertTrue(fired.get());
+    Map context = actionContextPropsRef.get();
+    assertNotNull(context);
+    List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
+    assertNotNull("The operations computed by ComputePlanAction should not be null , " + eventRef.get(), operations);
+    assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
+    SolrRequest solrRequest = operations.get(0);
+    SolrParams params = solrRequest.getParams();
+    assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
+    String replicaToBeMoved = params.get("replica");
+    assertEquals("Unexpected node in computed operation", replicas.get(0).getName(), replicaToBeMoved);
+
+    // shutdown the extra node that we had started
+    cluster.simRemoveNode(node2, false);
+  }
+
+  public void testNodeWithMultipleReplicasLost() throws Exception {
+    AssertingTriggerAction.expectedNode = null;
+
+    // start 3 more nodes
+    cluster.simAddNode();
+    cluster.simAddNode();
+    cluster.simAddNode();
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeWithMultipleReplicasLost",
+        "conf",2, 3);
+//    create.setMaxShardsPerNode(2);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        "testNodeWithMultipleReplicasLost", clusterShape(2, 3));
+
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    log.debug("-- cluster state: {}", clusterState);
+    DocCollection docCollection = clusterState.getCollection("testNodeWithMultipleReplicasLost");
+
+    // lets find a node with at least 2 replicas
+    String stoppedNodeName = null;
+    List<Replica> replicasToBeMoved = null;
+    for (String node : cluster.getClusterStateProvider().getLiveNodes()) {
+      List<Replica> replicas = docCollection.getReplicas(node);
+      if (replicas != null && replicas.size() == 2) {
+        stoppedNodeName = node;
+        replicasToBeMoved = replicas;
+        cluster.simRemoveNode(node, false);
+        break;
+      }
+    }
+    assertNotNull(stoppedNodeName);
+
+    assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
+    assertTrue(fired.get());
+
+    TriggerEvent triggerEvent = eventRef.get();
+    assertNotNull(triggerEvent);
+    assertEquals(TriggerEventType.NODELOST, triggerEvent.getEventType());
+    // TODO assertEquals(stoppedNodeName, triggerEvent.getProperty(TriggerEvent.NODE_NAME));
+
+    Map context = actionContextPropsRef.get();
+    assertNotNull(context);
+    List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
+    assertNotNull("The operations computed by ComputePlanAction should not be null " + actionContextPropsRef.get(), operations);
+    operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
+    assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
+
+    for (SolrRequest solrRequest : operations) {
+      SolrParams params = solrRequest.getParams();
+      assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
+      String moved = params.get("replica");
+      assertTrue(replicasToBeMoved.stream().anyMatch(replica -> replica.getName().equals(moved)));
+    }
+  }
+
+  @Test
+  public void testNodeAdded() throws Exception {
+    AssertingTriggerAction.expectedNode = null;
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // the default policy limits 1 replica per node, we need more right now
+    String setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<3', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeAdded",
+        "conf",1, 2);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        "testNodeAdded", (liveNodes, collectionState) -> collectionState.getReplicas().stream().allMatch(replica -> replica.isActive(liveNodes)));
+
+    // reset to the original policy which has only 1 replica per shard per node
+    setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'nodeRole':'overseer', 'replica':0}" +
+        "    ]" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // start a node so that the 'violation' created by the previous policy update is fixed
+    String newNode = cluster.simAddNode();
+    assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
+    assertTrue(fired.get());
+    Map context = actionContextPropsRef.get();
+    assertNotNull(context);
+    log.info("Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
+    log.info("Live nodes: " + cluster.getClusterStateProvider().getLiveNodes() + ", collection state: " + cluster.getClusterStateProvider().getClusterState().getCollection("testNodeAdded"));
+    List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
+    assertNotNull("The operations computed by ComputePlanAction should not be null" + context, operations);
+    assertEquals("ComputePlanAction should have computed exactly 1 operation, but was: " + operations, 1, operations.size());
+    SolrRequest request = operations.get(0);
+    SolrParams params = request.getParams();
+    assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
+    String nodeAdded = params.get("targetNode");
+    assertEquals("Unexpected node in computed operation", newNode, nodeAdded);
+  }
+
+  public static class AssertingTriggerAction implements TriggerAction {
+    static String expectedNode;
+
+    @Override
+    public String getName() {
+      return null;
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext context) {
+      if (expectedNode != null) {
+        Collection nodes = (Collection) event.getProperty(TriggerEvent.NODE_NAMES);
+        if (nodes == null || !nodes.contains(expectedNode)) return;//this is not the event we are looking for
+      }
+      if (fired.compareAndSet(false, true)) {
+        eventRef.set(event);
+        actionContextPropsRef.set(context.getProperties());
+        triggerFiredLatch.countDown();
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java
new file mode 100644
index 0000000..a9c5140
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestDistribStateManager.java
@@ -0,0 +1,284 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.impl.ZkDistribStateManager;
+import org.apache.solr.cloud.ZkTestServer;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test compares a ZK-based {@link DistribStateManager} to the simulated one.
+ */
+public class TestDistribStateManager extends SolrTestCaseJ4 {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private DistribStateManager stateManager;
+  private ZkTestServer zkTestServer;
+  private SolrZkClient solrZkClient;
+  private boolean simulated;
+  private SimDistribStateManager.Node root;
+
+  @Before
+  public void setup() throws Exception {
+    simulated = random().nextBoolean();
+    if (simulated) {
+      root = SimDistribStateManager.createNewRootNode();
+    } else {
+      zkTestServer = new ZkTestServer(createTempDir("zkDir").toString());
+      zkTestServer.run();
+    }
+    reInit();
+  }
+
+  private void reInit() throws Exception {
+    if (stateManager != null) {
+      stateManager.close();
+    }
+    if (simulated) {
+      stateManager = new SimDistribStateManager(root);
+    } else {
+      if (solrZkClient != null) {
+        solrZkClient.close();
+      }
+      solrZkClient = new SolrZkClient(zkTestServer.getZkHost(), 30000);
+      stateManager = new ZkDistribStateManager(solrZkClient);
+    }
+    LOG.info("Using " + stateManager.getClass().getName());
+  }
+
+  @After
+  public void teardown() throws Exception {
+    if (solrZkClient != null) {
+      solrZkClient.close();
+      solrZkClient = null;
+    }
+    if (zkTestServer != null) {
+      zkTestServer.shutdown();
+      zkTestServer = null;
+    }
+    if (stateManager != null) {
+      stateManager.close();
+    }
+    stateManager = null;
+  }
+
+  @Test
+  public void testHasData() throws Exception {
+    assertFalse(stateManager.hasData("/hasData/foo"));
+    assertFalse(stateManager.hasData("/hasData/bar"));
+    try {
+      stateManager.createData("/hasData/foo", new byte[0], CreateMode.PERSISTENT);
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+    stateManager.makePath("/hasData");
+    stateManager.createData("/hasData/foo", new byte[0], CreateMode.PERSISTENT);
+    stateManager.createData("/hasData/bar", new byte[0], CreateMode.PERSISTENT);
+    assertTrue(stateManager.hasData("/hasData/foo"));
+    assertTrue(stateManager.hasData("/hasData/bar"));
+  }
+
+  @Test
+  public void testListData() throws Exception {
+    assertFalse(stateManager.hasData("/listData/foo"));
+    assertFalse(stateManager.hasData("/listData/foo/bar"));
+    try {
+      stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
+      fail("should not succeed");
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+    try {
+      stateManager.listData("/listData/foo");
+      fail("should not succeed");
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+    stateManager.makePath("/listData");
+    List<String> kids = stateManager.listData("/listData");
+    assertEquals(0, kids.size());
+    stateManager.makePath("/listData/foo");
+    kids = stateManager.listData("/listData");
+    assertEquals(1, kids.size());
+    assertEquals("foo", kids.get(0));
+    stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
+    stateManager.createData("/listData/foo/baz", new byte[0], CreateMode.PERSISTENT);
+    kids = stateManager.listData("/listData/foo");
+    assertEquals(2, kids.size());
+    assertTrue(kids.contains("bar"));
+    assertTrue(kids.contains("baz"));
+    try {
+      stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
+      fail("should not succeed");
+    } catch (AlreadyExistsException e) {
+      // expected
+    }
+  }
+
+  static final byte[] firstData = new byte[] {
+      (byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe
+  };
+
+  static final byte[] secondData = new byte[] {
+      (byte)0xbe, (byte)0xba, (byte)0xfe, (byte)0xca
+  };
+
+  @Test
+  public void testCreateMode() throws Exception {
+    stateManager.makePath("/createMode");
+    stateManager.createData("/createMode/persistent", firstData, CreateMode.PERSISTENT);
+    stateManager.createData("/createMode/persistent_seq", firstData, CreateMode.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
+    }
+    // check what happens with gaps
+    stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
+    stateManager.removeData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 10), -1);
+    stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
+
+    stateManager.createData("/createMode/ephemeral", firstData, CreateMode.EPHEMERAL);
+    stateManager.createData("/createMode/ephemeral_seq", firstData, CreateMode.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      stateManager.createData("/createMode/ephemeral_seq/data", firstData, CreateMode.EPHEMERAL_SEQUENTIAL);
+    }
+    assertTrue(stateManager.hasData("/createMode"));
+    assertTrue(stateManager.hasData("/createMode/persistent"));
+    assertTrue(stateManager.hasData("/createMode/ephemeral"));
+    List<String> kids = stateManager.listData("/createMode/persistent_seq");
+    assertEquals(11, kids.size());
+    kids = stateManager.listData("/createMode/ephemeral_seq");
+    assertEquals(10, kids.size());
+    for (int i = 0; i < 10; i++) {
+      assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", i)));
+    }
+    assertFalse(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 10)));
+    assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 11)));
+
+    for (int i = 0; i < 10; i++) {
+      assertTrue(stateManager.hasData("/createMode/ephemeral_seq/data" + String.format(Locale.ROOT, "%010d", i)));
+    }
+    // check that ephemeral nodes disappear on disconnect
+    reInit();
+    assertTrue(stateManager.hasData("/createMode/persistent"));
+    for (int i = 0; i < 10; i++) {
+      assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", i)));
+    }
+    assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 11)));
+
+    assertFalse(stateManager.hasData("/createMode/ephemeral"));
+    assertTrue(stateManager.hasData("/createMode/ephemeral_seq"));
+    kids = stateManager.listData("/createMode/ephemeral_seq");
+    assertEquals(0, kids.size());
+  }
+
+  static class OnceWatcher implements Watcher {
+    CountDownLatch triggered = new CountDownLatch(1);
+    WatchedEvent event;
+
+    @Override
+    public void process(WatchedEvent event) {
+      if (triggered.getCount() == 0) {
+        fail("Watch was already triggered once!");
+      }
+      triggered.countDown();
+      this.event = event;
+    }
+  }
+
+  @Test
+  public void testGetSetRemoveData() throws Exception {
+    stateManager.makePath("/getData");
+    stateManager.createData("/getData/persistentData", firstData, CreateMode.PERSISTENT);
+    OnceWatcher nodeWatcher = new OnceWatcher();
+    VersionedData vd = stateManager.getData("/getData/persistentData", nodeWatcher);
+    assertNotNull(vd);
+    assertEquals(0, vd.getVersion());
+    assertTrue(Arrays.equals(firstData, vd.getData()));
+
+    // update data, test versioning
+    try {
+      stateManager.setData("/getData/persistentData", secondData, 1);
+      fail("should have failed");
+    } catch (BadVersionException e) {
+      // expected
+    }
+    // watch should not have fired
+    assertEquals(1, nodeWatcher.triggered.getCount());
+
+    stateManager.setData("/getData/persistentData", secondData, 0);
+    if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
+      fail("Node watch should have fired!");
+    }
+    // watch should not fire now because it needs to be reset
+    stateManager.setData("/getData/persistentData", secondData, -1);
+
+    nodeWatcher = new OnceWatcher();
+    stateManager.createData("/getData/ephemeralData", firstData, CreateMode.EPHEMERAL);
+    vd = stateManager.getData("/getData/ephemeralData", nodeWatcher);
+    reInit();
+    if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
+      fail("Node watch should have fired!");
+    }
+    assertTrue(stateManager.hasData("/getData/persistentData"));
+    assertFalse(stateManager.hasData("/getData/ephemeralData"));
+
+    nodeWatcher = new OnceWatcher();
+    vd = stateManager.getData("/getData/persistentData", nodeWatcher);
+    // try wrong version
+    try {
+      stateManager.removeData("/getData/persistentData", vd.getVersion() - 1);
+      fail("should have failed");
+    } catch (BadVersionException e) {
+      // expected
+    }
+    // watch should not have fired
+    assertEquals(1, nodeWatcher.triggered.getCount());
+
+    stateManager.removeData("/getData/persistentData", vd.getVersion());
+    if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
+      fail("Node watch should have fired!");
+    }
+  }
+
+  @Test
+  public void testMulti() throws Exception {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
new file mode 100644
index 0000000..18d76dc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
@@ -0,0 +1,216 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest;
+import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
+import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.LogLevel;
+import org.apache.solr.common.util.TimeSource;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test for {@link ExecutePlanAction}
+ */
+@LogLevel("org.apache.solr.cloud=DEBUG")
+public class TestExecutePlanAction extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final int NODE_COUNT = 2;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(NODE_COUNT, TimeSource.get("simTime:50"));
+  }
+
+  @Before
+  public void setUp() throws Exception  {
+    super.setUp();
+
+    if (cluster.getClusterStateProvider().getLiveNodes().size() < NODE_COUNT) {
+      // start some to get to original state
+      int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
+      for (int i = 0; i < NODE_COUNT - numJetties; i++) {
+        cluster.simAddNode();
+      }
+    }
+  }
+
+  @After
+  public void printState() throws Exception {
+    log.info("-------------_ FINAL STATE --------------");
+    log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
+    log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
+    ClusterState state = cluster.getClusterStateProvider().getClusterState();
+    for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
+      log.info("* Collection " + coll + " state: " + state.getCollection(coll));
+    }
+
+  }
+
+  @Test
+  public void testExecute() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String collectionName = "testExecute";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 1, 2);
+    create.setMaxShardsPerNode(1);
+    create.process(solrClient);
+
+    log.info("Collection ready after " + waitForState(collectionName, 120, TimeUnit.SECONDS, clusterShape(1, 2)) + "ms");
+
+    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    DocCollection docCollection = clusterState.getCollection(collectionName);
+    List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
+    assertNotNull(replicas);
+    assertFalse(replicas.isEmpty());
+
+    List<String> otherNodes = cluster.getClusterStateProvider().getLiveNodes().stream()
+        .filter(node -> !node.equals(sourceNodeName)).collect(Collectors.toList());
+    assertFalse(otherNodes.isEmpty());
+    String survivor = otherNodes.get(0);
+
+    try (ExecutePlanAction action = new ExecutePlanAction()) {
+      action.init(Collections.singletonMap("name", "execute_plan"));
+
+      // used to signal if we found that ExecutePlanAction did in fact create the right znode before executing the operation
+      AtomicBoolean znodeCreated = new AtomicBoolean(false);
+
+      CollectionAdminRequest.AsyncCollectionAdminRequest moveReplica = new CollectionAdminRequest.MoveReplica(collectionName, replicas.get(0).getName(), survivor);
+      CollectionAdminRequest.AsyncCollectionAdminRequest mockRequest = new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.OVERSEERSTATUS) {
+        @Override
+        public void setAsyncId(String asyncId) {
+          super.setAsyncId(asyncId);
+          String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/xyz/execute_plan";
+          try {
+            if (cluster.getDistribStateManager().hasData(parentPath)) {
+              java.util.List<String> children = cluster.getDistribStateManager().listData(parentPath);
+              if (!children.isEmpty()) {
+                String child = children.get(0);
+                VersionedData data = cluster.getDistribStateManager().getData(parentPath + "/" + child);
+                Map m = (Map) Utils.fromJSON(data.getData());
+                if (m.containsKey("requestid")) {
+                  znodeCreated.set(m.get("requestid").equals(asyncId));
+                }
+              }
+            }
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+
+        }
+      };
+      List<CollectionAdminRequest.AsyncCollectionAdminRequest> operations = Lists.asList(moveReplica, new CollectionAdminRequest.AsyncCollectionAdminRequest[]{mockRequest});
+      NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST,
+          "mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTime()),
+          Collections.singletonList(sourceNodeName));
+      ActionContext actionContext = new ActionContext(cluster, null,
+          new HashMap<>(Collections.singletonMap("operations", operations)));
+      action.process(nodeLostEvent, actionContext);
+
+//      assertTrue("ExecutePlanAction should have stored the requestid in ZK before executing the request", znodeCreated.get());
+      List<NamedList<Object>> responses = (List<NamedList<Object>>) actionContext.getProperty("responses");
+      assertNotNull(responses);
+      assertEquals(2, responses.size());
+      NamedList<Object> response = responses.get(0);
+      assertNull(response.get("failure"));
+      assertNotNull(response.get("success"));
+    }
+
+    log.info("Collection ready after " + waitForState(collectionName, 300, TimeUnit.SECONDS, clusterShape(1, 2)) + "ms");
+  }
+
+  @Test
+  public void testIntegration() throws Exception  {
+    SolrClient solrClient = cluster.simGetSolrClient();
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
+        "{'name':'execute_plan','class':'solr.ExecutePlanAction'}]" +
+        "}}";
+    SolrRequest req = AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String collectionName = "testIntegration";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 1, 2);
+    create.setMaxShardsPerNode(1);
+    create.process(solrClient);
+
+    waitForState("Timed out waiting for replicas of new collection to be active",
+        collectionName, clusterShape(1, 2));
+
+    String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
+    DocCollection docCollection = clusterState.getCollection(collectionName);
+    List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
+    assertNotNull(replicas);
+    assertFalse(replicas.isEmpty());
+
+    List<String> otherNodes = cluster.getClusterStateProvider().getLiveNodes().stream()
+        .filter(node -> !node.equals(sourceNodeName)).collect(Collectors.toList());
+    assertFalse(otherNodes.isEmpty());
+    String survivor = otherNodes.get(0);
+
+    cluster.simRemoveNode(sourceNodeName, false);
+
+    waitForState("Timed out waiting for replicas of collection to be 2 again",
+        collectionName, clusterShape(1, 2));
+
+    clusterState = cluster.getClusterStateProvider().getClusterState();
+    docCollection = clusterState.getCollection(collectionName);
+    List<Replica> replicasOnSurvivor = docCollection.getReplicas(survivor);
+    assertNotNull(replicasOnSurvivor);
+    assertEquals(2, replicasOnSurvivor.size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java
new file mode 100644
index 0000000..cba700b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestGenericDistributedQueue.java
@@ -0,0 +1,32 @@
+/*
+ * 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.autoscaling.sim;
+
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+
+/**
+ *
+ */
+public class TestGenericDistributedQueue extends TestSimDistributedQueue {
+  DistribStateManager stateManager = new SimDistribStateManager();
+
+  @Override
+  protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
+    return new GenericDistributedQueue(stateManager, dqZNode);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
new file mode 100644
index 0000000..034a039
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
@@ -0,0 +1,266 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.ComputePlanAction;
+import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
+import org.apache.solr.cloud.autoscaling.TriggerActionBase;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
+import org.apache.solr.cloud.autoscaling.CapturedEvent;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.LogLevel;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
+/**
+ *
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
+public class TestLargeCluster extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int SPEED = 50;
+
+  public static final int NUM_NODES = 100;
+
+  static Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
+  static AtomicInteger triggerFiredCount = new AtomicInteger();
+  static CountDownLatch triggerFiredLatch;
+  static int waitForSeconds;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
+  }
+
+  @Before
+  public void setupTest() throws Exception {
+
+    waitForSeconds = 1 + random().nextInt(3);
+    triggerFiredCount.set(0);
+    triggerFiredLatch = new CountDownLatch(1);
+    listenerEvents.clear();
+    while (cluster.getClusterStateProvider().getLiveNodes().size() < NUM_NODES) {
+      // perhaps a test stopped a node but didn't start it back
+      // lets start a node
+      cluster.simAddNode();
+    }
+  }
+
+  public static class TestTriggerListener extends TriggerListenerBase {
+    @Override
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
+    }
+
+    @Override
+    public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
+                                     ActionContext context, Throwable error, String message) {
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(cluster.getTimeSource().getTime(), context, config, stage, actionName, event, message));
+    }
+  }
+
+  public static class TestTriggerAction extends TriggerActionBase {
+    @Override
+    public void process(TriggerEvent event, ActionContext context) throws Exception {
+      triggerFiredCount.incrementAndGet();
+      triggerFiredLatch.countDown();
+    }
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'foo'," +
+        "'trigger' : 'node_lost_trigger'," +
+        "'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
+        "'beforeAction' : ['compute', 'execute']," +
+        "'afterAction' : ['compute', 'execute']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    cluster.getTimeSource().sleep(5000);
+
+    // pick a few random nodes
+    List<String> nodes = new ArrayList<>();
+    int limit = 75;
+    for (String node : cluster.getClusterStateProvider().getLiveNodes()) {
+      nodes.add(node);
+      if (nodes.size() > limit) {
+        break;
+      }
+    }
+    Collections.shuffle(nodes, random());
+    // create collection on these nodes
+    String collectionName = "testBasic";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 5, 5, 5, 5);
+    create.setMaxShardsPerNode(1);
+    create.setCreateNodeSet(String.join(",", nodes));
+    create.process(solrClient);
+
+    log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
+
+    int KILL_NODES = 8;
+    // kill off a number of nodes
+    for (int i = 0; i < KILL_NODES; i++) {
+      cluster.simRemoveNode(nodes.get(i), false);
+    }
+    // should fully recover
+    log.info("Ready after " + waitForState(collectionName, 90 * KILL_NODES, TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
+
+    log.info("OP COUNTS: " + cluster.simGetOpCounts());
+    long moveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
+
+    // simulate a number of flaky nodes
+    int FLAKY_NODES = 10;
+    int flakyReplicas = 0;
+    for (int cnt = 0; cnt < 10; cnt++) {
+      for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
+        flakyReplicas += cluster.getSimClusterStateProvider().simGetReplicaInfos(nodes.get(i))
+            .stream().filter(r -> r.getState().equals(Replica.State.ACTIVE)).count();
+        cluster.simRemoveNode(nodes.get(i), false);
+      }
+      cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(waitForSeconds) * 2);
+      for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
+        final String nodeId = nodes.get(i);
+        cluster.submit(() -> cluster.getSimClusterStateProvider().simRestoreNode(nodeId));
+      }
+    }
+
+    log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
+    log.info("OP COUNTS: " + cluster.simGetOpCounts());
+    long newMoveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
+    log.info("==== Flaky replicas: {}. Additional MOVEREPLICA count: {}", flakyReplicas, (newMoveReplicaOps - moveReplicaOps));
+    // flaky nodes lead to a number of MOVEREPLICA that is non-zero but lower than the number of flaky replicas
+    assertTrue("there should be new MOVERPLICA ops", newMoveReplicaOps - moveReplicaOps > 0);
+    assertTrue("there should be less than flakyReplicas=" + flakyReplicas + " MOVEREPLICA ops",
+        newMoveReplicaOps - moveReplicaOps < flakyReplicas);
+  }
+
+  @Test
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
+  public void testSearchRate() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'search_rate_trigger'," +
+        "'event' : 'searchRate'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'rate' : 1.0," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
+        "{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'srt'," +
+        "'trigger' : 'search_rate_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String collectionName = "testSearchRate";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
+        "conf", 2, 10);
+    create.process(solrClient);
+
+    log.info("Ready after " + waitForState(collectionName, 300, TimeUnit.SECONDS, clusterShape(2, 10)) + " ms");
+
+    // collect the node names
+    Set<String> nodes = new HashSet<>();
+    cluster.getSimClusterStateProvider().getClusterState().getCollection(collectionName)
+        .getReplicas()
+        .forEach(r -> nodes.add(r.getNodeName()));
+
+    String metricName = "QUERY./select.requestTimes:1minRate";
+    // simulate search traffic
+    cluster.getSimClusterStateProvider().simSetShardValue(collectionName, "shard1", metricName, 40, true);
+
+    Thread.sleep(1000000000);
+//    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+//    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(2000);
+    assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
+  }
+}


[13/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Add OpenNLP Analysis capabilities as a module

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/licenses/opennlp-tools-NOTICE.txt
----------------------------------------------------------------------
diff --git a/lucene/licenses/opennlp-tools-NOTICE.txt b/lucene/licenses/opennlp-tools-NOTICE.txt
new file mode 100644
index 0000000..68a08dc
--- /dev/null
+++ b/lucene/licenses/opennlp-tools-NOTICE.txt
@@ -0,0 +1,6 @@
+
+Apache OpenNLP Tools
+Copyright 2015 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/module-build.xml
----------------------------------------------------------------------
diff --git a/lucene/module-build.xml b/lucene/module-build.xml
index d48ae37..c2159b6 100644
--- a/lucene/module-build.xml
+++ b/lucene/module-build.xml
@@ -285,6 +285,28 @@
     <property name="analyzers-icu-javadocs.uptodate" value="true"/>
   </target>
 
+  <property name="analyzers-opennlp.jar" value="${common.dir}/build/analysis/opennlp/lucene-analyzers-opennlp-${version}.jar"/>
+  <target name="check-analyzers-opennlp-uptodate" unless="analyzers-opennlp.uptodate">
+    <module-uptodate name="analysis/opennlp" jarfile="${analyzers-opennlp.jar}" property="analyzers-opennlp.uptodate"/>
+  </target>
+  <target name="jar-analyzers-opennlp" unless="analyzers-opennlp.uptodate" depends="check-analyzers-opennlp-uptodate">
+    <ant dir="${common.dir}/analysis/opennlp" target="jar-core" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+    <property name="analyzers-opennlp.uptodate" value="true"/>
+  </target>
+
+  <property name="analyzers-opennlp-javadoc.jar" value="${common.dir}/build/analysis/opennlp/lucene-analyzers-opennlp-${version}-javadoc.jar"/>
+  <target name="check-analyzers-opennlp-javadocs-uptodate" unless="analyzers-opennlp-javadocs.uptodate">
+    <module-uptodate name="analysis/opennlp" jarfile="${analyzers-opennlp-javadoc.jar}" property="analyzers-opennlp-javadocs.uptodate"/>
+  </target>
+  <target name="javadocs-analyzers-opennlp" unless="analyzers-opennlp-javadocs.uptodate" depends="check-analyzers-opennlp-javadocs-uptodate">
+    <ant dir="${common.dir}/analysis/opennlp" target="javadocs" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+    <property name="analyzers-opennlp-javadocs.uptodate" value="true"/>
+  </target>
+
   <property name="analyzers-phonetic.jar" value="${common.dir}/build/analysis/phonetic/lucene-analyzers-phonetic-${version}.jar"/>
   <target name="check-analyzers-phonetic-uptodate" unless="analyzers-phonetic.uptodate">
     <module-uptodate name="analysis/phonetic" jarfile="${analyzers-phonetic.jar}" property="analyzers-phonetic.uptodate"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
index 070eab2..3e1e375 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
@@ -41,6 +41,7 @@ import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeFactory;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
@@ -127,7 +128,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   //     lastStartOffset)
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[],
                                                int posLengths[], Integer finalOffset, Integer finalPosInc, boolean[] keywordAtts,
-                                               boolean offsetsAreCorrect) throws IOException {
+                                               boolean offsetsAreCorrect, byte[][] payloads) throws IOException {
     assertNotNull(output);
     CheckClearAttributesAttribute checkClearAtt = ts.addAttribute(CheckClearAttributesAttribute.class);
     
@@ -166,6 +167,12 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       assertTrue("has no KeywordAttribute", ts.hasAttribute(KeywordAttribute.class));
       keywordAtt = ts.getAttribute(KeywordAttribute.class);
     }
+
+    PayloadAttribute payloadAtt = null;
+    if (payloads != null) {
+      assertTrue("has no PayloadAttribute", ts.hasAttribute(PayloadAttribute.class));
+      payloadAtt = ts.getAttribute(PayloadAttribute.class);
+    }
     
     // Maps position to the start/end offset:
     final Map<Integer,Integer> posToStartOffset = new HashMap<>();
@@ -185,6 +192,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       if (posIncrAtt != null) posIncrAtt.setPositionIncrement(45987657);
       if (posLengthAtt != null) posLengthAtt.setPositionLength(45987653);
       if (keywordAtt != null) keywordAtt.setKeyword((i&1) == 0);
+      if (payloadAtt != null) payloadAtt.setPayload(new BytesRef(new byte[] { 0x00, -0x21, 0x12, -0x43, 0x24 }));
       
       checkClearAtt.getAndResetClearCalled(); // reset it, because we called clearAttribute() before
       assertTrue("token "+i+" does not exist", ts.incrementToken());
@@ -209,7 +217,14 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       if (keywordAtts != null) {
         assertEquals("keywordAtt " + i + " term=" + termAtt, keywordAtts[i], keywordAtt.isKeyword());
       }
-      
+      if (payloads != null) {
+        if (payloads[i] != null) {
+          assertEquals("payloads " + i, new BytesRef(payloads[i]), payloadAtt.getPayload());
+        } else {
+          assertNull("payloads " + i, payloads[i]);
+        }
+      }
+
       // we can enforce some basic things about a few attributes even if the caller doesn't check:
       if (offsetAtt != null) {
         final int startOffset = offsetAtt.startOffset();
@@ -283,7 +298,9 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     if (typeAtt != null) typeAtt.setType("bogusType");
     if (posIncrAtt != null) posIncrAtt.setPositionIncrement(45987657);
     if (posLengthAtt != null) posLengthAtt.setPositionLength(45987653);
-    
+    if (keywordAtt != null) keywordAtt.setKeyword(true);
+    if (payloadAtt != null) payloadAtt.setPayload(new BytesRef(new byte[] { 0x00, -0x21, 0x12, -0x43, 0x24 }));
+
     checkClearAtt.getAndResetClearCalled(); // reset it, because we called clearAttribute() before
 
     ts.end();
@@ -305,7 +322,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[],
                                                int posLengths[], Integer finalOffset, boolean[] keywordAtts,
                                                boolean offsetsAreCorrect) throws IOException {
-    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, null, null, offsetsAreCorrect);
+    assertTokenStreamContents(ts, output, startOffsets, endOffsets, types, posIncrements, posLengths, finalOffset, null, keywordAtts, offsetsAreCorrect, null);
   }
 
   public static void assertTokenStreamContents(TokenStream ts, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], Integer finalOffset, boolean offsetsAreCorrect) throws IOException {
@@ -373,7 +390,12 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
     checkAnalysisConsistency(random(), a, true, input, offsetsAreCorrect);
     assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), offsetsAreCorrect);
   }
-  
+
+  public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], boolean offsetsAreCorrect, byte[][] payloads) throws IOException {
+    checkResetException(a, input);
+    assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), null, null, offsetsAreCorrect, payloads);
+  }
+
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
     assertAnalyzesTo(a, input, output, null, null, null, null, null);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a3f6f75..e60262d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -53,6 +53,13 @@ New Features
 ----------------------
 * SOLR-11285: Simulation framework for autoscaling. (ab)
 
+* LUCENE-2899: In the Solr analysis-extras contrib, added support for the
+  OpenNLP-based analysis components in the Lucene analysis/opennlp module:
+  tokenization, part-of-speech tagging, phrase chunking, and lemmatization.
+  Also added OpenNLP-based named entity extraction as a Solr update request
+  processor.  (Lance Norskog, Grant Ingersoll, Joern Kottmann, Em, Kai Gülzau,
+  Rene Nederhand, Robert Muir, Steven Bower, Steve Rowe)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/README.txt
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/README.txt b/solr/contrib/analysis-extras/README.txt
index 3863420..fb8381a 100644
--- a/solr/contrib/analysis-extras/README.txt
+++ b/solr/contrib/analysis-extras/README.txt
@@ -1,8 +1,10 @@
 The analysis-extras plugin provides additional analyzers that rely
 upon large dependencies/dictionaries.
 
-It includes integration with ICU for multilingual support, and 
-analyzers for Chinese and Polish.
+It includes integration with ICU for multilingual support,
+analyzers for Chinese and Polish, and integration with
+OpenNLP for multilingual tokenization, part-of-speech tagging
+lemmatization, phrase chunking, and named-entity recognition.
 
 ICU relies upon lucene-libs/lucene-analyzers-icu-X.Y.jar
 and lib/icu4j-X.Y.jar
@@ -13,4 +15,6 @@ Stempel relies on lucene-libs/lucene-analyzers-stempel-X.Y.jar
 
 Morfologik relies on lucene-libs/lucene-analyzers-morfologik-X.Y.jar
 and lib/morfologik-*.jar
- 
+
+OpenNLP relies on lucene-libs/lucene-analyzers-opennlp-X.Y.jar
+and lib/opennlp-*.jar

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/build.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/build.xml b/solr/contrib/analysis-extras/build.xml
index 38d67dd..68a88ad 100644
--- a/solr/contrib/analysis-extras/build.xml
+++ b/solr/contrib/analysis-extras/build.xml
@@ -7,9 +7,9 @@
     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.
@@ -24,19 +24,20 @@
   </description>
 
   <import file="../contrib-build.xml"/>
-  
+
   <target name="compile-test" depends="-compile-test-lucene-analysis,common-solr.compile-test"/>
 
   <path id="analysis.extras.lucene.libs">
     <pathelement location="${analyzers-icu.jar}"/>
-    <!-- 
-      Although the smartcn, stempel, and morfologik jars are not dependencies of
+    <!--
+      Although the smartcn, stempel, morfologik and opennlp jars are not dependencies of
       code in the analysis-extras contrib, they must remain here in order to
       populate the Solr distribution
      -->
     <pathelement location="${analyzers-smartcn.jar}"/>
     <pathelement location="${analyzers-stempel.jar}"/>
     <pathelement location="${analyzers-morfologik.jar}"/>
+    <pathelement location="${analyzers-opennlp.jar}"/>
   </path>
 
   <path id="classpath">
@@ -53,12 +54,12 @@
     </dirset>
   </path>
 
-  <!-- 
-    Although the smartcn, stempel, and morfologik jars are not dependencies of
+  <!--
+    Although the smartcn, stempel, morfologik and opennlp jars are not dependencies of
     code in the analysis-extras contrib, they must remain here in order to
     populate the Solr distribution
    -->
-  <target name="module-jars-to-solr" 
+  <target name="module-jars-to-solr"
           depends="-module-jars-to-solr-not-for-package,-module-jars-to-solr-package"/>
   <target name="-module-jars-to-solr-not-for-package" unless="called.from.create-package">
     <antcall inheritall="true">
@@ -66,6 +67,7 @@
       <target name="jar-analyzers-smartcn"/>
       <target name="jar-analyzers-stempel"/>
       <target name="jar-analyzers-morfologik"/>
+      <target name="jar-analyzers-opennlp"/>
     </antcall>
     <property name="analyzers-icu.uptodate" value="true"/> <!-- compile-time dependency -->
     <mkdir dir="${build.dir}/lucene-libs"/>
@@ -85,6 +87,6 @@
     </copy>
   </target>
 
-  <target name="compile-core" depends="jar-analyzers-icu, solr-contrib-build.compile-core"/>
+  <target name="compile-core" depends="jar-analyzers-icu, jar-analyzers-opennlp, solr-contrib-build.compile-core"/>
   <target name="dist" depends="module-jars-to-solr, common-solr.dist"/>
 </project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/ivy.xml b/solr/contrib/analysis-extras/ivy.xml
index 0c71701..cfc30c1 100644
--- a/solr/contrib/analysis-extras/ivy.xml
+++ b/solr/contrib/analysis-extras/ivy.xml
@@ -24,6 +24,9 @@
   </configurations>
   <dependencies>
     <dependency org="com.ibm.icu" name="icu4j" rev="${/com.ibm.icu/icu4j}" conf="compile"/>
+    <dependency org="org.apache.opennlp" name="opennlp-tools" rev="${/org.apache.opennlp/opennlp-tools}" conf="compile" />
+    <dependency org="org.apache.opennlp" name="opennlp-maxent" rev="${/org.apache.opennlp/opennlp-maxent}" conf="compile" />
+
     <!--
       Although the 3rd party morfologik jars are not dependencies of code in
       the analysis-extras contrib, they must remain here in order to

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
new file mode 100644
index 0000000..d00df2b
--- /dev/null
+++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
@@ -0,0 +1,571 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update.processor;
+
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+import opennlp.tools.util.Span;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.opennlp.OpenNLPTokenizer;
+import org.apache.lucene.analysis.opennlp.tools.NLPNERTaggerOp;
+import org.apache.lucene.analysis.opennlp.tools.OpenNLPOpsFactory;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.FieldMutatingUpdateProcessor.FieldNameSelector;
+import org.apache.solr.update.processor.FieldMutatingUpdateProcessorFactory.SelectorParams;
+import org.apache.solr.util.plugin.SolrCoreAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extracts named entities using an OpenNLP NER <code>modelFile</code> from the values found in
+ * any matching <code>source</code> field into a configured <code>dest</code> field, after
+ * first tokenizing the source text using the index analyzer on the configured
+ * <code>analyzerFieldType</code>, which must include <code>solr.OpenNLPTokenizerFactory</code>
+ * as the tokenizer. E.g.:
+ *
+ * <pre class="prettyprint">
+ *   &lt;fieldType name="opennlp-en-tokenization" class="solr.TextField"&gt;
+ *     &lt;analyzer&gt;
+ *       &lt;tokenizer class="solr.OpenNLPTokenizerFactory"
+ *                  sentenceModel="en-sent.bin"
+ *                  tokenizerModel="en-tokenizer.bin"/&gt;
+ *     &lt;/analyzer&gt;
+ *   &lt;/fieldType&gt;
+ * </pre>
+ * 
+ * <p>See the <a href="OpenNLP website">http://opennlp.apache.org/models.html</a>
+ * for information on downloading pre-trained models.</p>
+ *
+ * <p>
+ * The <code>source</code> field(s) can be configured as either:
+ * </p>
+ * <ul>
+ *  <li>One or more <code>&lt;str&gt;</code></li>
+ *  <li>An <code>&lt;arr&gt;</code> of <code>&lt;str&gt;</code></li>
+ *  <li>A <code>&lt;lst&gt;</code> containing
+ *   {@link FieldMutatingUpdateProcessor FieldMutatingUpdateProcessorFactory style selector arguments}</li>
+ * </ul>
+ *
+ * <p>The <code>dest</code> field can be a single <code>&lt;str&gt;</code>
+ * containing the literal name of a destination field, or it may be a <code>&lt;lst&gt;</code> specifying a
+ * regex <code>pattern</code> and a <code>replacement</code> string. If the pattern + replacement option
+ * is used the pattern will be matched against all fields matched by the source selector, and the replacement
+ * string (including any capture groups specified from the pattern) will be evaluated a using
+ * {@link Matcher#replaceAll(String)} to generate the literal name of the destination field.  Additionally,
+ * an occurrence of the string "{EntityType}" in the <code>dest</code> field specification, or in the
+ * <code>replacement</code> string, will be replaced with the entity type(s) returned for each entity by
+ * the OpenNLP NER model; as a result, if the model extracts more than one entity type, then more than one
+ * <code>dest</code> field will be populated.
+ * </p>
+ *
+ * <p>If the resolved <code>dest</code> field already exists in the document, then the
+ * named entities extracted from the <code>source</code> fields will be added to it.
+ * </p>
+ * <p>
+ * In the example below:
+ * </p>
+ * <ul>
+ *   <li>Named entities will be extracted from the <code>text</code> field and added
+ *       to the <code>names_ss</code> field</li>
+ *   <li>Named entities will be extracted from both the <code>title</code> and
+ *       <code>subtitle</code> fields and added into the <code>titular_people</code> field</li>
+ *   <li>Named entities will be extracted from any field with a name ending in <code>_txt</code>
+ *       -- except for <code>notes_txt</code> -- and added into the <code>people_ss</code> field</li>
+ *   <li>Named entities will be extracted from any field with a name beginning with "desc" and
+ *       ending in "s" (e.g. "descs" and "descriptions") and added to a field prefixed with "key_",
+ *       not ending in "s", and suffixed with "_people". (e.g. "key_desc_people" or
+ *       "key_description_people")</li>
+ *   <li>Named entities will be extracted from the <code>summary</code> field and added
+ *       to the <code>summary_person_ss</code> field, assuming that the modelFile only extracts
+ *       entities of type "person".</li>
+ * </ul>
+ *
+ * <pre class="prettyprint">
+ * &lt;updateRequestProcessorChain name="multiple-extract"&gt;
+ *   &lt;processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory"&gt;
+ *     &lt;str name="modelFile"&gt;en-test-ner-person.bin&lt;/str&gt;
+ *     &lt;str name="analyzerFieldType"&gt;opennlp-en-tokenization&lt;/str&gt;
+ *     &lt;str name="source"&gt;text&lt;/str&gt;
+ *     &lt;str name="dest"&gt;people_s&lt;/str&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory"&gt;
+ *     &lt;str name="modelFile"&gt;en-test-ner-person.bin&lt;/str&gt;
+ *     &lt;str name="analyzerFieldType"&gt;opennlp-en-tokenization&lt;/str&gt;
+ *     &lt;arr name="source"&gt;
+ *       &lt;str&gt;title&lt;/str&gt;
+ *       &lt;str&gt;subtitle&lt;/str&gt;
+ *     &lt;/arr&gt;
+ *     &lt;str name="dest"&gt;titular_people&lt;/str&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory"&gt;
+ *     &lt;str name="modelFile"&gt;en-test-ner-person.bin&lt;/str&gt;
+ *     &lt;str name="analyzerFieldType"&gt;opennlp-en-tokenization&lt;/str&gt;
+ *     &lt;lst name="source"&gt;
+ *       &lt;str name="fieldRegex"&gt;.*_txt$&lt;/str&gt;
+ *       &lt;lst name="exclude"&gt;
+ *         &lt;str name="fieldName"&gt;notes_txt&lt;/str&gt;
+ *       &lt;/lst&gt;
+ *     &lt;/lst&gt;
+ *     &lt;str name="dest"&gt;people_s&lt;/str&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.processor.OpenNLPExtractNamedEntitiesUpdateProcessorFactory"&gt;
+ *     &lt;str name="modelFile"&gt;en-test-ner-person.bin&lt;/str&gt;
+ *     &lt;str name="analyzerFieldType"&gt;opennlp-en-tokenization&lt;/str&gt;
+ *     &lt;lst name="source"&gt;
+ *       &lt;str name="fieldRegex"&gt;^desc(.*)s$&lt;/str&gt;
+ *     &lt;/lst&gt;
+ *     &lt;lst name="dest"&gt;
+ *       &lt;str name="pattern"&gt;^desc(.*)s$&lt;/str&gt;
+ *       &lt;str name="replacement"&gt;key_desc$1_people&lt;/str&gt;
+ *     &lt;/lst&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory"&gt;
+ *     &lt;str name="modelFile"&gt;en-test-ner-person.bin&lt;/str&gt;
+ *     &lt;str name="analyzerFieldType"&gt;opennlp-en-tokenization&lt;/str&gt;
+ *     &lt;str name="source"&gt;summary&lt;/str&gt;
+ *     &lt;str name="dest"&gt;summary_{EntityType}_s&lt;/str&gt;
+ *   &lt;/processor&gt;
+ * &lt;/updateRequestProcessorChain&gt;
+ * </pre>
+ *
+ * @since 7.3.0
+ */
+public class OpenNLPExtractNamedEntitiesUpdateProcessorFactory
+    extends UpdateRequestProcessorFactory implements SolrCoreAware {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String SOURCE_PARAM = "source";
+  public static final String DEST_PARAM = "dest";
+  public static final String PATTERN_PARAM = "pattern";
+  public static final String REPLACEMENT_PARAM = "replacement";
+  public static final String MODEL_PARAM = "modelFile";
+  public static final String ANALYZER_FIELD_TYPE_PARAM = "analyzerFieldType";
+  public static final String ENTITY_TYPE = "{EntityType}";
+
+  private SelectorParams srcInclusions = new SelectorParams();
+  private Collection<SelectorParams> srcExclusions = new ArrayList<>();
+
+  private FieldNameSelector srcSelector = null;
+
+  private String modelFile = null;
+  private String analyzerFieldType = null;
+
+  /**
+   * If pattern is null, this this is a literal field name.  If pattern is non-null then this
+   * is a replacement string that may contain meta-characters (ie: capture group identifiers)
+   * @see #pattern
+   */
+  private String dest = null;
+  /** @see #dest */
+  private Pattern pattern = null;
+
+  protected final FieldNameSelector getSourceSelector() {
+    if (null != srcSelector) return srcSelector;
+
+    throw new SolrException(SERVER_ERROR, "selector was never initialized, inform(SolrCore) never called???");
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void init(NamedList args) {
+
+    // high level (loose) check for which type of config we have.
+    //
+    // individual init methods do more strict syntax checking
+    if (0 <= args.indexOf(SOURCE_PARAM, 0) && 0 <= args.indexOf(DEST_PARAM, 0) ) {
+      initSourceSelectorSyntax(args);
+    } else if (0 <= args.indexOf(PATTERN_PARAM, 0) && 0 <= args.indexOf(REPLACEMENT_PARAM, 0)) {
+      initSimpleRegexReplacement(args);
+    } else {
+      throw new SolrException(SERVER_ERROR, "A combination of either '" + SOURCE_PARAM + "' + '"+
+          DEST_PARAM + "', or '" + REPLACEMENT_PARAM + "' + '" +
+          PATTERN_PARAM + "' init params are mandatory");
+    }
+
+    Object modelParam = args.remove(MODEL_PARAM);
+    if (null == modelParam) {
+      throw new SolrException(SERVER_ERROR, "Missing required init param '" + MODEL_PARAM + "'");
+    }
+    if ( ! (modelParam instanceof CharSequence)) {
+      throw new SolrException(SERVER_ERROR, "Init param '" + MODEL_PARAM + "' must be a <str>");
+    }
+    modelFile = modelParam.toString();
+
+    Object analyzerFieldTypeParam = args.remove(ANALYZER_FIELD_TYPE_PARAM);
+    if (null == analyzerFieldTypeParam) {
+      throw new SolrException(SERVER_ERROR, "Missing required init param '" + ANALYZER_FIELD_TYPE_PARAM + "'");
+    }
+    if ( ! (analyzerFieldTypeParam instanceof CharSequence)) {
+      throw new SolrException(SERVER_ERROR, "Init param '" + ANALYZER_FIELD_TYPE_PARAM + "' must be a <str>");
+    }
+    analyzerFieldType = analyzerFieldTypeParam.toString();
+
+    if (0 < args.size()) {
+      throw new SolrException(SERVER_ERROR, "Unexpected init param(s): '" + args.getName(0) + "'");
+    }
+
+    super.init(args);
+  }
+
+  /**
+   * init helper method that should only be called when we know for certain that both the
+   * "source" and "dest" init params do <em>not</em> exist.
+   */
+  @SuppressWarnings("unchecked")
+  private void initSimpleRegexReplacement(NamedList args) {
+    // The syntactic sugar for the case where there is only one regex pattern for source and the same pattern
+    // is used for the destination pattern...
+    //
+    //  pattern != null && replacement != null
+    //
+    // ...as top level elements, with no other config options specified
+
+    // if we got here we know we had pattern and replacement, now check for the other two  so that we can give a better
+    // message than "unexpected"
+    if (0 <= args.indexOf(SOURCE_PARAM, 0) || 0 <= args.indexOf(DEST_PARAM, 0) ) {
+      throw new SolrException(SERVER_ERROR,"Short hand syntax must not be mixed with full syntax. Found " +
+          PATTERN_PARAM + " and " + REPLACEMENT_PARAM + " but also found " + SOURCE_PARAM + " or " + DEST_PARAM);
+    }
+
+    assert args.indexOf(SOURCE_PARAM, 0) < 0;
+
+    Object patt = args.remove(PATTERN_PARAM);
+    Object replacement = args.remove(REPLACEMENT_PARAM);
+
+    if (null == patt || null == replacement) {
+      throw new SolrException(SERVER_ERROR, "Init params '" + PATTERN_PARAM + "' and '" +
+          REPLACEMENT_PARAM + "' are both mandatory if '" + SOURCE_PARAM + "' and '"+
+          DEST_PARAM + "' are not both specified");
+    }
+
+    if (0 != args.size()) {
+      throw new SolrException(SERVER_ERROR, "Init params '" + REPLACEMENT_PARAM + "' and '" +
+          PATTERN_PARAM + "' must be children of '" + DEST_PARAM +
+          "' to be combined with other options.");
+    }
+
+    if (!(replacement instanceof String)) {
+      throw new SolrException(SERVER_ERROR, "Init param '" + REPLACEMENT_PARAM + "' must be a string (i.e. <str>)");
+    }
+    if (!(patt instanceof String)) {
+      throw new SolrException(SERVER_ERROR, "Init param '" + PATTERN_PARAM + "' must be a string (i.e. <str>)");
+    }
+
+    dest = replacement.toString();
+    try {
+      this.pattern = Pattern.compile(patt.toString());
+    } catch (PatternSyntaxException pe) {
+      throw new SolrException(SERVER_ERROR, "Init param " + PATTERN_PARAM +
+          " is not a valid regex pattern: " + patt, pe);
+
+    }
+    srcInclusions = new SelectorParams();
+    srcInclusions.fieldRegex = Collections.singletonList(this.pattern);
+  }
+
+  /**
+   * init helper method that should only be called when we know for certain that both the
+   * "source" and "dest" init params <em>do</em> exist.
+   */
+  @SuppressWarnings("unchecked")
+  private void initSourceSelectorSyntax(NamedList args) {
+    // Full and complete syntax where source and dest are mandatory.
+    //
+    // source may be a single string or a selector.
+    // dest may be a single string or list containing pattern and replacement
+    //
+    //   source != null && dest != null
+
+    // if we got here we know we had source and dest, now check for the other two so that we can give a better
+    // message than "unexpected"
+    if (0 <= args.indexOf(PATTERN_PARAM, 0) || 0 <= args.indexOf(REPLACEMENT_PARAM, 0) ) {
+      throw new SolrException(SERVER_ERROR,"Short hand syntax must not be mixed with full syntax. Found " +
+          SOURCE_PARAM + " and " + DEST_PARAM + " but also found " + PATTERN_PARAM + " or " + REPLACEMENT_PARAM);
+    }
+
+    Object d = args.remove(DEST_PARAM);
+    assert null != d;
+
+    List<Object> sources = args.getAll(SOURCE_PARAM);
+    assert null != sources;
+
+    if (1 == sources.size()) {
+      if (sources.get(0) instanceof NamedList) {
+        // nested set of selector options
+        NamedList selectorConfig = (NamedList) args.remove(SOURCE_PARAM);
+
+        srcInclusions = parseSelectorParams(selectorConfig);
+
+        List<Object> excList = selectorConfig.getAll("exclude");
+
+        for (Object excObj : excList) {
+          if (null == excObj) {
+            throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                "' child 'exclude' can not be null");
+          }
+          if (!(excObj instanceof NamedList)) {
+            throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                "' child 'exclude' must be <lst/>");
+          }
+          NamedList exc = (NamedList) excObj;
+          srcExclusions.add(parseSelectorParams(exc));
+          if (0 < exc.size()) {
+            throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                "' has unexpected 'exclude' sub-param(s): '"
+                + selectorConfig.getName(0) + "'");
+          }
+          // call once per instance
+          selectorConfig.remove("exclude");
+        }
+
+        if (0 < selectorConfig.size()) {
+          throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+              "' contains unexpected child param(s): '" +
+              selectorConfig.getName(0) + "'");
+        }
+        // consume from the named list so it doesn't interfere with subsequent processing
+        sources.remove(0);
+      }
+    }
+    if (1 <= sources.size()) {
+      // source better be one or more strings
+      srcInclusions.fieldName = new HashSet<>(args.removeConfigArgs("source"));
+    }
+    if (srcInclusions == null) {
+      throw new SolrException(SERVER_ERROR,
+          "Init params do not specify any field from which to extract entities, please supply either "
+          + SOURCE_PARAM + " and " + DEST_PARAM + " or " + PATTERN_PARAM + " and " + REPLACEMENT_PARAM + ". See javadocs" +
+          "for OpenNLPExtractNamedEntitiesUpdateProcessor for further details.");
+    }
+
+    if (d instanceof NamedList) {
+      NamedList destList = (NamedList) d;
+
+      Object patt = destList.remove(PATTERN_PARAM);
+      Object replacement = destList.remove(REPLACEMENT_PARAM);
+
+      if (null == patt || null == replacement) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' children '" +
+            PATTERN_PARAM + "' and '" + REPLACEMENT_PARAM +
+            "' are both mandatory and can not be null");
+      }
+      if (! (patt instanceof String && replacement instanceof String)) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' children '" +
+            PATTERN_PARAM + "' and '" + REPLACEMENT_PARAM +
+            "' must both be strings (i.e. <str>)");
+      }
+      if (0 != destList.size()) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' has unexpected children: '"
+            + destList.getName(0) + "'");
+      }
+
+      try {
+        this.pattern = Pattern.compile(patt.toString());
+      } catch (PatternSyntaxException pe) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' child '" + PATTERN_PARAM +
+            " is not a valid regex pattern: " + patt, pe);
+      }
+      dest = replacement.toString();
+
+    } else if (d instanceof String) {
+      dest = d.toString();
+    } else {
+      throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' must either be a string " +
+          "(i.e. <str>) or a list (i.e. <lst>) containing '" +
+          PATTERN_PARAM + "' and '" + REPLACEMENT_PARAM);
+    }
+
+  }
+
+  @Override
+  public void inform(final SolrCore core) {
+
+    srcSelector =
+        FieldMutatingUpdateProcessor.createFieldNameSelector
+            (core.getResourceLoader(), core, srcInclusions, FieldMutatingUpdateProcessor.SELECT_NO_FIELDS);
+
+    for (SelectorParams exc : srcExclusions) {
+      srcSelector = FieldMutatingUpdateProcessor.wrap
+          (srcSelector,
+              FieldMutatingUpdateProcessor.createFieldNameSelector
+                  (core.getResourceLoader(), core, exc, FieldMutatingUpdateProcessor.SELECT_NO_FIELDS));
+    }
+    try {
+      OpenNLPOpsFactory.getNERTaggerModel(modelFile, core.getResourceLoader());
+    } catch (IOException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  @Override
+  public final UpdateRequestProcessor getInstance
+      (SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+    final FieldNameSelector srcSelector = getSourceSelector();
+    return new UpdateRequestProcessor(next) {
+      private final NLPNERTaggerOp nerTaggerOp;
+      private Analyzer analyzer = null;
+      {
+        try {
+          nerTaggerOp = OpenNLPOpsFactory.getNERTagger(modelFile);
+          FieldType fieldType = req.getSchema().getFieldTypeByName(analyzerFieldType);
+          if (fieldType == null) {
+            throw new SolrException
+                (SERVER_ERROR, ANALYZER_FIELD_TYPE_PARAM + " '" + analyzerFieldType + "' not found in the schema.");
+          }
+          analyzer = fieldType.getIndexAnalyzer();
+        } catch (IOException e) {
+          throw new IllegalArgumentException(e);
+        }
+      }
+
+      @Override
+      public void processAdd(AddUpdateCommand cmd) throws IOException {
+
+        final SolrInputDocument doc = cmd.getSolrInputDocument();
+
+        // Destination may be regex replace string, or "{EntityType}" replaced by
+        // each entity's type, both of which can cause multiple output fields.
+        Map<String,SolrInputField> destMap = new HashMap<>();
+
+        // preserve initial values
+        for (final String fname : doc.getFieldNames()) {
+          if ( ! srcSelector.shouldMutate(fname)) continue;
+
+          Collection<Object> srcFieldValues = doc.getFieldValues(fname);
+          if (srcFieldValues == null || srcFieldValues.isEmpty()) continue;
+
+          String resolvedDest = dest;
+
+          if (pattern != null) {
+            Matcher matcher = pattern.matcher(fname);
+            if (matcher.find()) {
+              resolvedDest = matcher.replaceAll(dest);
+            } else {
+              log.debug("srcSelector.shouldMutate(\"{}\") returned true, " +
+                  "but replacement pattern did not match, field skipped.", fname);
+              continue;
+            }
+          }
+
+          for (Object val : srcFieldValues) {
+            for (Pair<String,String> entity : extractTypedNamedEntities(val)) {
+              SolrInputField destField = null;
+              String entityName = entity.first();
+              String entityType = entity.second();
+              resolvedDest = resolvedDest.replace(ENTITY_TYPE, entityType);
+              if (doc.containsKey(resolvedDest)) {
+                destField = doc.getField(resolvedDest);
+              } else {
+                SolrInputField targetField = destMap.get(resolvedDest);
+                if (targetField == null) {
+                  destField = new SolrInputField(resolvedDest);
+                } else {
+                  destField = targetField;
+                }
+              }
+              destField.addValue(entityName);
+
+              // put it in map to avoid concurrent modification...
+              destMap.put(resolvedDest, destField);
+            }
+          }
+        }
+
+        for (Map.Entry<String,SolrInputField> entry : destMap.entrySet()) {
+          doc.put(entry.getKey(), entry.getValue());
+        }
+        super.processAdd(cmd);
+      }
+
+      /** Using configured NER model, extracts (name, type) pairs from the given source field value */
+      private List<Pair<String,String>> extractTypedNamedEntities(Object srcFieldValue) throws IOException {
+        List<Pair<String,String>> entitiesWithType = new ArrayList<>();
+        List<String> terms = new ArrayList<>();
+        List<Integer> startOffsets = new ArrayList<>();
+        List<Integer> endOffsets = new ArrayList<>();
+        String fullText = srcFieldValue.toString();
+        TokenStream tokenStream = analyzer.tokenStream("", fullText);
+        CharTermAttribute termAtt = tokenStream.addAttribute(CharTermAttribute.class);
+        OffsetAttribute offsetAtt = tokenStream.addAttribute(OffsetAttribute.class);
+        FlagsAttribute flagsAtt = tokenStream.addAttribute(FlagsAttribute.class);
+        tokenStream.reset();
+        synchronized (nerTaggerOp) {
+          while (tokenStream.incrementToken()) {
+            terms.add(termAtt.toString());
+            startOffsets.add(offsetAtt.startOffset());
+            endOffsets.add(offsetAtt.endOffset());
+            boolean endOfSentence = 0 != (flagsAtt.getFlags() & OpenNLPTokenizer.EOS_FLAG_BIT);
+            if (endOfSentence) {    // extract named entities one sentence at a time
+              extractEntitiesFromSentence(fullText, terms, startOffsets, endOffsets, entitiesWithType);
+            }
+          }
+          tokenStream.end();
+          tokenStream.close();
+          if (!terms.isEmpty()) { // In case last token of last sentence isn't properly flagged with EOS_FLAG_BIT
+            extractEntitiesFromSentence(fullText, terms, startOffsets, endOffsets, entitiesWithType);
+          }
+          nerTaggerOp.reset();      // Forget all adaptive data collected during previous calls
+        }
+        return entitiesWithType;
+      }
+
+      private void extractEntitiesFromSentence(String fullText, List<String> terms, List<Integer> startOffsets,
+                                               List<Integer> endOffsets, List<Pair<String,String>> entitiesWithType) {
+        for (Span span : nerTaggerOp.getNames(terms.toArray(new String[terms.size()]))) {
+          String text = fullText.substring(startOffsets.get(span.getStart()), endOffsets.get(span.getEnd() - 1));
+          entitiesWithType.add(new Pair<>(text, span.getType()));
+        }
+        terms.clear();
+        startOffsets.clear();
+        endOffsets.clear();
+      }
+    };
+  }
+
+  /** macro */
+  private static SelectorParams parseSelectorParams(NamedList args) {
+    return FieldMutatingUpdateProcessorFactory.parseSelectorParams(args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/package.html
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/package.html b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/package.html
new file mode 100644
index 0000000..1388c29
--- /dev/null
+++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/package.html
@@ -0,0 +1,24 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<!-- not a package-info.java, because we already defined this package in core/ -->
+<html>
+  <body>
+    Update request processor invoking OpenNLP Named Entity Recognition over configured
+    source field(s), populating configured target field(s) with the results.
+  </body>
+</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-ner-person.bin
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-ner-person.bin b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-ner-person.bin
new file mode 100644
index 0000000..0b40aac
Binary files /dev/null and b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-ner-person.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-sent.bin
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-sent.bin b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-sent.bin
new file mode 100644
index 0000000..4252bcb
Binary files /dev/null and b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-sent.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-tokenizer.bin
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-tokenizer.bin b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-tokenizer.bin
new file mode 100644
index 0000000..94668c0
Binary files /dev/null and b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/en-test-tokenizer.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-opennlp-extract.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-opennlp-extract.xml b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-opennlp-extract.xml
new file mode 100644
index 0000000..fc13431
--- /dev/null
+++ b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/schema-opennlp-extract.xml
@@ -0,0 +1,49 @@
+<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<schema name="test-opennlp-extract" version="1.6">
+  <fieldType name="opennlp-en-tokenization" class="solr.TextField">
+    <analyzer>
+      <tokenizer class="solr.OpenNLPTokenizerFactory"
+                 sentenceModel="en-test-sent.bin"
+                 tokenizerModel="en-test-tokenizer.bin"/>
+    </analyzer>
+  </fieldType>
+
+  <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+
+  <fieldType name="text" class="solr.TextField" positionIncrementGap="100" autoGeneratePhraseQueries="true">
+    <analyzer>
+      <tokenizer class="solr.MockTokenizerFactory"/>
+      <filter class="solr.LowerCaseFilterFactory"/>
+      <filter class="solr.PorterStemFilterFactory"/>
+    </analyzer>
+  </fieldType>
+
+  <field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
+  <field name="text" type="text" indexed="true" stored="false"/>
+  <field name="subject" type="text" indexed="true" stored="true"/>
+  <field name="title" type="text" indexed="true" stored="true"/>
+  <field name="subtitle" type="text" indexed="true" stored="true"/>
+  <field name="descs" type="text" indexed="true" stored="true"/>
+  <field name="descriptions" type="text" indexed="true" stored="true"/>
+
+  <dynamicField name="*_txt" type="text" indexed="true" stored="true"/>
+  <dynamicField name="*_s" type="string" indexed="true" stored="true" multiValued="true"/>
+  <dynamicField name="*_people" type="string" indexed="true" stored="true" multiValued="true"/>
+</schema>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml
new file mode 100644
index 0000000..c44c9e1
--- /dev/null
+++ b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig-opennlp-extract.xml
@@ -0,0 +1,206 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <requestHandler name="/select" class="solr.SearchHandler"></requestHandler>
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <updateRequestProcessorChain name="extract-single">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <str name="source">source1_s</str>
+      <str name="dest">dest_s</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-single-regex">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <str name="source">source1_s</str>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-multi">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <str name="source">source1_s</str>
+      <str name="source">source2_s</str>
+      <str name="dest">dest_s</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-multi-regex">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <str name="source">source1_s</str>
+      <str name="source">source2_s</str>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-array">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <arr name="source">
+        <str>source1_s</str>
+        <str>source2_s</str>
+      </arr>
+      <str name="dest">dest_s</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-array-regex">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <arr name="source">
+        <str>source1_s</str>
+        <str>source2_s</str>
+      </arr>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-selector">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <lst name="source">
+        <str name="fieldRegex">source\d_.*</str>
+        <lst name="exclude">
+          <str name="fieldRegex">source0_.*</str>
+        </lst>
+      </lst>
+      <str name="dest">dest_s</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-selector-regex">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <lst name="source">
+        <str name="fieldRegex">source\d_.*</str>
+        <lst name="exclude">
+          <str name="fieldRegex">source0_.*</str>
+        </lst>
+      </lst>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-regex-replaceall">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <lst name="source">
+        <str name="fieldRegex">foo.*</str>
+      </lst>
+      <lst name="dest">
+        <!-- unbounded pattern that can be replaced multiple times in field name -->
+        <str name="pattern">x(\d)</str>
+        <str name="replacement">y$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="extract-regex-replaceall-with-entity-type">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <lst name="source">
+        <str name="fieldRegex">foo.*</str>
+      </lst>
+      <lst name="dest">
+        <!-- unbounded pattern that can be replaced multiple times in field name -->
+        <str name="pattern">x(\d)</str>
+        <str name="replacement">{EntityType}_y$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <!-- example used in OpenNLPExtractNamedEntitiesUpdateProcessorFactory javadocs -->
+  <updateRequestProcessorChain name="multiple-extract">
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <str name="source">text</str>
+      <str name="dest">people_s</str>
+    </processor>
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <arr name="source">
+        <str>title</str>
+        <str>subtitle</str>
+      </arr>
+      <str name="dest">titular_people</str>
+    </processor>
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <lst name="source">
+        <str name="fieldRegex">.*_txt$</str>
+        <lst name="exclude">
+          <str name="fieldName">notes_txt</str>
+        </lst>
+      </lst>
+      <str name="dest">people_s</str>
+    </processor>
+    <processor class="solr.processor.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <lst name="source">
+        <str name="fieldRegex">^desc(.*)s$</str>
+      </lst>
+      <lst name="dest">
+        <str name="pattern">^desc(.*)s$</str>
+        <str name="replacement">key_desc$1_people</str>
+      </lst>
+    </processor>
+    <processor class="solr.OpenNLPExtractNamedEntitiesUpdateProcessorFactory">
+      <str name="modelFile">en-test-ner-person.bin</str>
+      <str name="analyzerFieldType">opennlp-en-tokenization</str>
+      <str name="source">summary</str>
+      <str name="dest">summary_{EntityType}_s</str>
+    </processor>
+  </updateRequestProcessorChain>
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
new file mode 100644
index 0000000..23516b0
--- /dev/null
+++ b/solr/contrib/analysis-extras/src/test-files/analysis-extras/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
@@ -0,0 +1,48 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+A solrconfig.xml snippet containing indexConfig settings for randomized testing.
+-->
+<indexConfig>
+  <!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
+       the RandomMergePolicy in all tests - but some tests expect very specific
+       Merge behavior, so those tests can set it as needed.
+  -->
+  <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+
+  <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+  <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+  <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+
+  <mergeScheduler class="${solr.tests.mergeScheduler}" />
+
+  <writeLockTimeout>1000</writeLockTimeout>
+  <commitLockTimeout>10000</commitLockTimeout>
+
+  <!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
+       use the single process lockType for speed - but tests that explicitly need
+       to vary the lockType can set it as needed.
+  -->
+  <lockType>${solr.tests.lockType:single}</lockType>
+
+  <infoStream>${solr.tests.infostream:false}</infoStream>
+
+</indexConfig>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/contrib/analysis-extras/src/test/org/apache/solr/update/processor/TestOpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/test/org/apache/solr/update/processor/TestOpenNLPExtractNamedEntitiesUpdateProcessorFactory.java b/solr/contrib/analysis-extras/src/test/org/apache/solr/update/processor/TestOpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
new file mode 100644
index 0000000..dad06a8
--- /dev/null
+++ b/solr/contrib/analysis-extras/src/test/org/apache/solr/update/processor/TestOpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update.processor;
+
+import java.io.File;
+import java.util.Arrays;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestOpenNLPExtractNamedEntitiesUpdateProcessorFactory extends UpdateProcessorTestBase {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    File testHome = createTempDir().toFile();
+    FileUtils.copyDirectory(getFile("analysis-extras/solr"), testHome);
+    initCore("solrconfig-opennlp-extract.xml", "schema-opennlp-extract.xml", testHome.getAbsolutePath());
+  }
+
+  @Test
+  public void testSimpleExtract() throws Exception {
+    SolrInputDocument doc = processAdd("extract-single",
+        doc(f("id", "1"),
+            f("source1_s", "Take this to Mr. Flashman.")));
+    assertEquals("dest_s should have stringValue", "Flashman", doc.getFieldValue("dest_s"));
+  }
+
+  @Test
+  public void testMultiExtract() throws Exception {
+    SolrInputDocument doc = processAdd("extract-multi",
+        doc(f("id", "1"),
+            f("source1_s", "Hello Flashman."),
+            f("source2_s", "Calling Flashman.")));
+
+    assertEquals(Arrays.asList("Flashman", "Flashman"), doc.getFieldValues("dest_s"));
+  }
+
+  @Test
+  public void testArrayExtract() throws Exception {
+    SolrInputDocument doc = processAdd("extract-array",
+        doc(f("id", "1"),
+            f("source1_s", "Currently we have Flashman. Not much else."),
+            f("source2_s", "Flashman. Is. Not. There.")));
+
+    assertEquals(Arrays.asList("Flashman", "Flashman"), doc.getFieldValues("dest_s"));
+  }
+
+  @Test
+  public void testSelectorExtract() throws Exception {
+    SolrInputDocument doc = processAdd("extract-selector",
+        doc(f("id", "1"),
+            f("source0_s", "Flashman. Or not."),
+            f("source1_s", "Serendipitously, he was. I mean, Flashman. And yet."),
+            f("source2_s", "Correct, Flashman.")));
+
+    assertEquals(Arrays.asList("Flashman", "Flashman"), doc.getFieldValues("dest_s"));
+  }
+
+  public void testMultipleExtracts() throws Exception {
+    // test example from the javadocs
+    SolrInputDocument doc = processAdd("multiple-extract",
+        doc(f("id", "1"),
+            f("text", "From Flashman. To Panman."),
+            f("title", "It's Captain Flashman.", "Privately, Flashman."),
+            f("subtitle", "Ineluctably, Flashman."),
+            f("corrolary_txt", "Forsooth thou bringeth Flashman."),
+            f("notes_txt", "Yes Flashman."),
+            f("summary", "Many aspire to be Flashman."),
+            f("descs", "Courage, Flashman.", "Ain't he Flashman."),
+            f("descriptions", "Flashman. Flashman. Flashman.")));
+
+    assertEquals(Arrays.asList("Flashman", "Flashman"), doc.getFieldValues("people_s"));
+    assertEquals(Arrays.asList("Flashman", "Flashman", "Flashman"), doc.getFieldValues("titular_people"));
+    assertEquals(Arrays.asList("Flashman", "Flashman"), doc.getFieldValues("key_desc_people"));
+    assertEquals(Arrays.asList("Flashman", "Flashman", "Flashman"), doc.getFieldValues("key_description_people"));
+    assertEquals("Flashman", doc.getFieldValue("summary_person_s")); // {EntityType} field name interpolation
+  }
+
+  public void testEquivalentExtraction() throws Exception {
+    SolrInputDocument d;
+
+    // regardless of chain, all of these checks should be equivalent
+    for (String chain : Arrays.asList("extract-single", "extract-single-regex",
+        "extract-multi", "extract-multi-regex",
+        "extract-array", "extract-array-regex",
+        "extract-selector", "extract-selector-regex")) {
+
+      // simple extract
+      d = processAdd(chain,
+          doc(f("id", "1111"),
+              f("source0_s", "Totally Flashman."), // not extracted
+              f("source1_s", "One nation under Flashman.", "Good Flashman.")));
+      assertNotNull(chain, d);
+      assertEquals(chain, Arrays.asList("Flashman", "Flashman"), d.getFieldValues("dest_s"));
+
+      // append to existing values
+      d = processAdd(chain,
+          doc(f("id", "1111"),
+              field("dest_s", "orig1", "orig2"),
+              f("source0_s", "Flashman. In totality."), // not extracted
+              f("source1_s", "Two nations under Flashman.", "Meh Flashman.")));
+      assertNotNull(chain, d);
+      assertEquals(chain, Arrays.asList("orig1", "orig2", "Flashman", "Flashman"), d.getFieldValues("dest_s"));
+    }
+
+    // should be equivalent for any chain matching source1_s and source2_s (but not source0_s)
+    for (String chain : Arrays.asList("extract-multi", "extract-multi-regex",
+        "extract-array", "extract-array-regex",
+        "extract-selector", "extract-selector-regex")) {
+
+      // simple extract
+      d = processAdd(chain,
+          doc(f("id", "1111"),
+              f("source0_s", "Not Flashman."), // not extracted
+              f("source1_s", "Could have had a Flashman.", "Bad Flashman."),
+              f("source2_s", "Indubitably Flashman.")));
+      assertNotNull(chain, d);
+      assertEquals(chain, Arrays.asList("Flashman", "Flashman", "Flashman"), d.getFieldValues("dest_s"));
+
+      // append to existing values
+      d = processAdd(chain,
+          doc(f("id", "1111"),
+              field("dest_s", "orig1", "orig2"),
+              f("source0_s", "Never Flashman."), // not extracted
+              f("source1_s", "Seeking Flashman.", "Evil incarnate Flashman."),
+              f("source2_s", "Perfunctorily Flashman.")));
+      assertNotNull(chain, d);
+      assertEquals(chain, Arrays.asList("orig1", "orig2", "Flashman", "Flashman", "Flashman"), d.getFieldValues("dest_s"));
+    }
+
+    // any chain that copies source1_s to dest_s should be equivalent for these assertions
+    for (String chain : Arrays.asList("extract-single", "extract-single-regex",
+        "extract-multi", "extract-multi-regex",
+        "extract-array", "extract-array-regex",
+        "extract-selector", "extract-selector-regex")) {
+
+      // simple extract
+      d = processAdd(chain,
+          doc(f("id", "1111"),
+              f("source1_s", "Always Flashman.", "Flashman. Noone else.")));
+      assertNotNull(chain, d);
+      assertEquals(chain, Arrays.asList("Flashman", "Flashman"), d.getFieldValues("dest_s"));
+
+      // append to existing values
+      d = processAdd(chain,
+          doc(f("id", "1111"),
+              field("dest_s", "orig1", "orig2"),
+              f("source1_s", "Flashman.  And, scene.", "Contemporary Flashman. Yeesh.")));
+      assertNotNull(chain, d);
+      assertEquals(chain, Arrays.asList("orig1", "orig2", "Flashman", "Flashman"), d.getFieldValues("dest_s"));
+    }
+  }
+
+  public void testExtractFieldRegexReplaceAll() throws Exception {
+    SolrInputDocument d = processAdd("extract-regex-replaceall",
+        doc(f("id", "1111"),
+            f("foo_x2_s", "Infrequently Flashman.", "In the words of Flashman."),
+            f("foo_x3_x7_s", "Flashman. Whoa.")));
+
+    assertNotNull(d);
+    assertEquals(Arrays.asList("Flashman", "Flashman"), d.getFieldValues("foo_y2_s"));
+    assertEquals("Flashman", d.getFieldValue("foo_y3_y7_s"));
+  }
+
+  public void testExtractFieldRegexReplaceAllWithEntityType() throws Exception {
+    SolrInputDocument d = processAdd("extract-regex-replaceall-with-entity-type",
+        doc(f("id", "1111"),
+            f("foo_x2_s", "Infrequently Flashman.", "In the words of Flashman."),
+            f("foo_x3_x7_s", "Flashman. Whoa.")));
+
+    assertNotNull(d);
+    assertEquals(d.getFieldNames().toString(), Arrays.asList("Flashman", "Flashman"), d.getFieldValues("foo_person_y2_s"));
+    assertEquals(d.getFieldNames().toString(),"Flashman", d.getFieldValue("foo_person_y3_person_y7_s"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java b/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java
deleted file mode 100644
index d3aa979..0000000
--- a/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java
+++ /dev/null
@@ -1,168 +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.update.processor;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.SolrInputField;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.request.SolrRequestInfo;
-import org.apache.solr.request.LocalSolrQueryRequest;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.update.CommitUpdateCommand;
-import org.apache.solr.update.DeleteUpdateCommand;
-
-import java.io.IOException;
-
-public class UpdateProcessorTestBase extends SolrTestCaseJ4 {
-
-  /**
-   * Runs a document through the specified chain, and returns the final
-   * document used when the chain is completed (NOTE: some chains may
-   * modify the document in place
-   */
-  protected SolrInputDocument processAdd(final String chain,
-                                         final SolrInputDocument docIn)
-    throws IOException {
-
-    return processAdd(chain, new ModifiableSolrParams(), docIn);
-  }
-
-  /**
-   * Runs a document through the specified chain, and returns the final
-   * document used when the chain is completed (NOTE: some chains may
-   * modify the document in place
-   */
-  protected SolrInputDocument processAdd(final String chain,
-                                         final SolrParams requestParams,
-                                         final SolrInputDocument docIn)
-    throws IOException {
-
-    SolrCore core = h.getCore();
-    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
-    assertNotNull("No Chain named: " + chain, pc);
-
-    SolrQueryResponse rsp = new SolrQueryResponse();
-
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, requestParams);
-    try {
-      SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
-      AddUpdateCommand cmd = new AddUpdateCommand(req);
-      cmd.solrDoc = docIn;
-
-      UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
-      if (null != processor) {
-        // test chain might be empty or short circuited.
-        processor.processAdd(cmd);
-      }
-
-      return cmd.solrDoc;
-    } finally {
-      SolrRequestInfo.clearRequestInfo();
-      req.close();
-    }
-  }
-
-  protected void processCommit(final String chain) throws IOException {
-    SolrCore core = h.getCore();
-    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
-    assertNotNull("No Chain named: " + chain, pc);
-
-    SolrQueryResponse rsp = new SolrQueryResponse();
-
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
-
-    CommitUpdateCommand cmd = new CommitUpdateCommand(req,false);
-    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
-    try {
-      processor.processCommit(cmd);
-    } finally {
-      req.close();
-    }
-  }
-
-  protected void processDeleteById(final String chain, String id) throws IOException {
-    SolrCore core = h.getCore();
-    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
-    assertNotNull("No Chain named: " + chain, pc);
-
-    SolrQueryResponse rsp = new SolrQueryResponse();
-
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
-
-    DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
-    cmd.setId(id);
-    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
-    try {
-      processor.processDelete(cmd);
-    } finally {
-      req.close();
-    }
-  }
-
-  protected void finish(final String chain) throws IOException {
-    SolrCore core = h.getCore();
-    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
-    assertNotNull("No Chain named: " + chain, pc);
-
-    SolrQueryResponse rsp = new SolrQueryResponse();
-    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
-
-    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
-    try {
-      processor.finish();
-    } finally {
-      IOUtils.closeQuietly(processor);
-      req.close();
-    }
-  }
-
-
-  /**
-   * Convenience method for building up SolrInputDocuments
-   */
-  final SolrInputDocument doc(SolrInputField... fields) {
-    SolrInputDocument d = new SolrInputDocument();
-    for (SolrInputField f : fields) {
-      d.put(f.getName(), f);
-    }
-    return d;
-  }
-
-  /**
-   * Convenience method for building up SolrInputFields
-   */
-  final SolrInputField field(String name, Object... values) {
-    SolrInputField f = new SolrInputField(name);
-    for (Object v : values) {
-      f.addValue(v);
-    }
-    return f;
-  }
-
-  /**
-   * Convenience method for building up SolrInputFields with default boost
-   */
-  final SolrInputField f(String name, Object... values) {
-    return field(name, values);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/licenses/opennlp-maxent-3.0.3.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/opennlp-maxent-3.0.3.jar.sha1 b/solr/licenses/opennlp-maxent-3.0.3.jar.sha1
new file mode 100644
index 0000000..c3c412f
--- /dev/null
+++ b/solr/licenses/opennlp-maxent-3.0.3.jar.sha1
@@ -0,0 +1 @@
+55e39e6b46e71f35229cdd6950e72d8cce3b5fd4


[02/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
new file mode 100644
index 0000000..a05eb78
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
@@ -0,0 +1,1217 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.util.concurrent.AtomicDouble;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.autoscaling.ActionContext;
+import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
+import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
+import org.apache.solr.cloud.autoscaling.TriggerActionBase;
+import org.apache.solr.cloud.autoscaling.TriggerEvent;
+import org.apache.solr.cloud.autoscaling.TriggerEventQueue;
+import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
+import org.apache.solr.cloud.autoscaling.CapturedEvent;
+import org.apache.solr.common.cloud.LiveNodesListener;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.LogLevel;
+import org.apache.solr.util.TimeOut;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+import static org.apache.solr.cloud.autoscaling.ScheduledTriggers.DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS;
+
+/**
+ * An end-to-end integration test for triggers
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
+public class TestTriggerIntegration extends SimSolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final int SPEED = 50;
+
+  private static CountDownLatch actionConstructorCalled;
+  private static CountDownLatch actionInitCalled;
+  private static CountDownLatch triggerFiredLatch;
+  private static int waitForSeconds = 1;
+  private static CountDownLatch actionStarted;
+  private static CountDownLatch actionInterrupted;
+  private static CountDownLatch actionCompleted;
+  private static AtomicBoolean triggerFired;
+  private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
+
+  private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2, TimeSource.get("simTime:" + SPEED));
+  }
+
+  private static CountDownLatch getTriggerFiredLatch() {
+    return triggerFiredLatch;
+  }
+
+  private static CountDownLatch getActionStarted() {
+    return actionStarted;
+  }
+
+  private static CountDownLatch getActionInterrupted() {
+    return actionInterrupted;
+  }
+
+  private static CountDownLatch getActionCompleted() {
+    return actionCompleted;
+  }
+
+  @Before
+  public void setupTest() throws Exception {
+
+    waitForSeconds = 1 + random().nextInt(3);
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+    triggerFiredLatch = new CountDownLatch(1);
+    triggerFired = new AtomicBoolean(false);
+    actionStarted = new CountDownLatch(1);
+    actionInterrupted = new CountDownLatch(1);
+    actionCompleted = new CountDownLatch(1);
+    events.clear();
+    listenerEvents.clear();
+    while (cluster.getClusterStateProvider().getLiveNodes().size() < 2) {
+      // perhaps a test stopped a node but didn't start it back
+      // lets start a node
+      cluster.simAddNode();
+    }
+  }
+
+  @Test
+  public void testTriggerThrottling() throws Exception  {
+    // for this test we want to create two triggers so we must assert that the actions were created twice
+    actionInitCalled = new CountDownLatch(2);
+    // similarly we want both triggers to fire
+    triggerFiredLatch = new CountDownLatch(2);
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+
+    // first trigger
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger1'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // second trigger
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger2'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the two instances of action are created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    String newNode = cluster.simAddNode();
+
+    if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("Both triggers should have fired by now");
+    }
+
+    // reset shared state
+    lastActionExecutedAt.set(0);
+    TestTriggerIntegration.actionInitCalled = new CountDownLatch(2);
+    triggerFiredLatch = new CountDownLatch(2);
+
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger1'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger2'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '0s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the two instances of action are created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    // stop the node we had started earlier
+    cluster.simRemoveNode(newNode, false);
+
+    if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("Both triggers should have fired by now");
+    }
+  }
+
+  static AtomicLong lastActionExecutedAt = new AtomicLong(0);
+  static ReentrantLock lock = new ReentrantLock();
+  public static class ThrottlingTesterAction extends TestTriggerAction {
+    // nanos are very precise so we need a delta for comparison with ms
+    private static final long DELTA_MS = 2;
+
+    // sanity check that an action instance is only invoked once
+    private final AtomicBoolean onlyOnce = new AtomicBoolean(false);
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      boolean locked = lock.tryLock();
+      if (!locked)  {
+        log.info("We should never have a tryLock fail because actions are never supposed to be executed concurrently");
+        return;
+      }
+      try {
+        if (lastActionExecutedAt.get() != 0)  {
+          log.info("last action at " + lastActionExecutedAt.get() + " time = " + cluster.getTimeSource().getTime());
+          if (TimeUnit.NANOSECONDS.toMillis(cluster.getTimeSource().getTime() - lastActionExecutedAt.get()) <
+              TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS) - DELTA_MS) {
+            log.info("action executed again before minimum wait time from {}", event.getSource());
+            fail("TriggerListener was fired before the throttling period");
+          }
+        }
+        if (onlyOnce.compareAndSet(false, true)) {
+          log.info("action executed from {}", event.getSource());
+          lastActionExecutedAt.set(cluster.getTimeSource().getTime());
+          getTriggerFiredLatch().countDown();
+        } else  {
+          log.info("action executed more than once from {}", event.getSource());
+          fail("Trigger should not have fired more than once!");
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+  }
+
+  @Test
+  public void testNodeLostTriggerRestoreState() throws Exception {
+    // for this test we want to update the trigger so we must assert that the actions were created twice
+    TestTriggerIntegration.actionInitCalled = new CountDownLatch(2);
+
+    // start a new node
+    String nodeName = cluster.simAddNode();
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    waitForSeconds = 5;
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_restore_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '5s'," + // should be enough for us to update the trigger
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cluster.getTimeSource());
+    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
+      timeOut.sleep(200);
+    }
+    assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
+
+    cluster.simRemoveNode(nodeName, false);
+
+    // ensure that the old trigger sees the stopped node, todo find a better way to do this
+    timeOut.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
+
+    waitForSeconds = 0;
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_restore_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the second instance of action is created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    boolean await = triggerFiredLatch.await(5000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    NodeLostTrigger.NodeLostEvent nodeLostEvent = (NodeLostTrigger.NodeLostEvent) events.iterator().next();
+    assertNotNull(nodeLostEvent);
+    List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(nodeName));
+  }
+
+  @Test
+  public void testNodeAddedTriggerRestoreState() throws Exception {
+    // for this test we want to update the trigger so we must assert that the actions were created twice
+    TestTriggerIntegration.actionInitCalled = new CountDownLatch(2);
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+    waitForSeconds = 5;
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_restore_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '5s'," + // should be enough for us to update the trigger
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cluster.getTimeSource());
+    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
+      timeOut.sleep(200);
+    }
+    assertTrue("The action specified in node_added_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
+
+    // start a new node
+    String newNode = cluster.simAddNode();
+
+    // ensure that the old trigger sees the new node, todo find a better way to do this
+    cluster.getTimeSource().sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
+
+    waitForSeconds = 0;
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_restore_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '0s'," + // update a property so that it replaces the old trigger, also we want it to fire immediately
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // wait until the second instance of action is created
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("Two TriggerAction instances should have been created by now");
+    }
+
+    boolean await = triggerFiredLatch.await(5000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.toString(), nodeNames.contains(newNode));
+  }
+
+  @Test
+  public void testNodeAddedTrigger() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeAddedEvent.toString(), nodeNames.contains(newNode));
+
+    // reset
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+
+    // update the trigger with exactly the same data
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // this should be a no-op so the action should have been created but init should not be called
+    if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    assertFalse(actionInitCalled.await(2000 / SPEED, TimeUnit.MILLISECONDS));
+  }
+
+  @Test
+  public void testNodeLostTrigger() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    String lostNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+    cluster.simRemoveNode(lostNodeName, false);
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    TriggerEvent nodeLostEvent = events.iterator().next();
+    assertNotNull(nodeLostEvent);
+    List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(lostNodeName));
+
+    // reset
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+
+    // update the trigger with exactly the same data
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // this should be a no-op so the action should have been created but init should not be called
+    if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    assertFalse(actionInitCalled.await(2000 / SPEED, TimeUnit.MILLISECONDS));
+  }
+
+  // simulator doesn't support overseer functionality yet
+  /*
+  @Test
+  public void testContinueTriggersOnOverseerRestart() throws Exception  {
+    CollectionAdminRequest.OverseerStatus status = new CollectionAdminRequest.OverseerStatus();
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    CollectionAdminResponse adminResponse = status.process(solrClient);
+    NamedList<Object> response = adminResponse.getResponse();
+    String leader = (String) response.get("leader");
+    JettySolrRunner overseerNode = null;
+    int index = -1;
+    List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
+    for (int i = 0; i < jettySolrRunners.size(); i++) {
+      JettySolrRunner runner = jettySolrRunners.get(i);
+      if (runner.getNodeName().equals(leader)) {
+        overseerNode = runner;
+        index = i;
+        break;
+      }
+    }
+    assertNotNull(overseerNode);
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    // stop the overseer, somebody else will take over as the overseer
+    cluster.stopJettySolrRunner(index);
+    Thread.sleep(10000);
+    JettySolrRunner newNode = cluster.startJettySolrRunner();
+    boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(newNode.getNodeName()));
+  }
+
+*/
+
+  public static class TestTriggerAction extends TriggerActionBase {
+
+    public TestTriggerAction() {
+      actionConstructorCalled.countDown();
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      try {
+        if (triggerFired.compareAndSet(false, true))  {
+          events.add(event);
+          long currentTimeNanos = cluster.getTimeSource().getTime();
+          long eventTimeNanos = event.getEventTime();
+          long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+          if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+            fail(event.getSource() + " was fired before the configured waitFor period");
+          }
+          getTriggerFiredLatch().countDown();
+        } else  {
+          fail(event.getSource() + " was fired more than once!");
+        }
+      } catch (Throwable t) {
+        log.debug("--throwable", t);
+        throw t;
+      }
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      log.info("TestTriggerAction init");
+      actionInitCalled.countDown();
+      super.init(args);
+    }
+  }
+
+  public static class TestEventQueueAction extends TriggerActionBase {
+
+    public TestEventQueueAction() {
+      log.info("TestEventQueueAction instantiated");
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      log.info("-- event: " + event);
+      events.add(event);
+      getActionStarted().countDown();
+      try {
+        Thread.sleep(eventQueueActionWait);
+        triggerFired.compareAndSet(false, true);
+        getActionCompleted().countDown();
+      } catch (InterruptedException e) {
+        getActionInterrupted().countDown();
+        return;
+      }
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      log.debug("TestTriggerAction init");
+      actionInitCalled.countDown();
+      super.init(args);
+    }
+  }
+
+  public static long eventQueueActionWait = 5000;
+
+  @Test
+  public void testEventQueue() throws Exception {
+    waitForSeconds = 1;
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger1'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
+        "}}";
+
+    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    // add node to generate the event
+    String newNode = cluster.simAddNode();
+    boolean await = actionStarted.await(60000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action did not start", await);
+    // event should be there
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    // but action did not complete yet so the event is still enqueued
+    assertFalse(triggerFired.get());
+    events.clear();
+    actionStarted = new CountDownLatch(1);
+    eventQueueActionWait = 1;
+    // kill overseer
+    cluster.simRestartOverseer(overseerLeader);
+    cluster.getTimeSource().sleep(5000);
+    // new overseer leader should be elected and run triggers
+    await = actionInterrupted.await(3000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action wasn't interrupted", await);
+    // it should fire again from enqueued event
+    await = actionStarted.await(60000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action wasn't started", await);
+    TriggerEvent replayedEvent = events.iterator().next();
+    assertTrue(replayedEvent.getProperty(TriggerEventQueue.ENQUEUE_TIME) != null);
+    assertTrue(events + "\n" + replayedEvent.toString(), replayedEvent.getProperty(TriggerEventQueue.DEQUEUE_TIME) != null);
+    await = actionCompleted.await(10000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("action wasn't completed", await);
+    assertTrue(triggerFired.get());
+  }
+
+  @Test
+  public void testEventFromRestoredState() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '10s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    events.clear();
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    // reset
+    triggerFired.set(false);
+    triggerFiredLatch = new CountDownLatch(1);
+    TriggerEvent nodeAddedEvent = events.iterator().next();
+    assertNotNull(nodeAddedEvent);
+    List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(newNode));
+    // add a second node - state of the trigger will change but it won't fire for waitFor sec.
+    String newNode2 = cluster.simAddNode();
+    cluster.getTimeSource().sleep(10000);
+    // kill overseer
+    cluster.simRestartOverseer(null);
+    await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+  }
+
+  private static class TestLiveNodesListener implements LiveNodesListener {
+    Set<String> lostNodes = new HashSet<>();
+    Set<String> addedNodes = new HashSet<>();
+    CountDownLatch onChangeLatch = new CountDownLatch(1);
+
+    public void reset() {
+      lostNodes.clear();
+      addedNodes.clear();
+      onChangeLatch = new CountDownLatch(1);
+    }
+
+    @Override
+    public void onChange(SortedSet<String> oldLiveNodes, SortedSet<String> newLiveNodes) {
+      onChangeLatch.countDown();
+      Set<String> old = new HashSet<>(oldLiveNodes);
+      old.removeAll(newLiveNodes);
+      if (!old.isEmpty()) {
+        lostNodes.addAll(old);
+      }
+      newLiveNodes.removeAll(oldLiveNodes);
+      if (!newLiveNodes.isEmpty()) {
+        addedNodes.addAll(newLiveNodes);
+      }
+    }
+  }
+
+  private TestLiveNodesListener registerLiveNodesListener() {
+    TestLiveNodesListener listener = new TestLiveNodesListener();
+    cluster.getLiveNodesSet().registerLiveNodesListener(listener);
+    return listener;
+  }
+
+  public static class TestEventMarkerAction extends TriggerActionBase {
+
+    public TestEventMarkerAction() {
+      actionConstructorCalled.countDown();
+    }
+
+    @Override
+    public void process(TriggerEvent event, ActionContext actionContext) {
+      boolean locked = lock.tryLock();
+      if (!locked)  {
+        log.info("We should never have a tryLock fail because actions are never supposed to be executed concurrently");
+        return;
+      }
+      try {
+        events.add(event);
+        getTriggerFiredLatch().countDown();
+      } catch (Throwable t) {
+        log.debug("--throwable", t);
+        throw t;
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      log.info("TestEventMarkerAction init");
+      actionInitCalled.countDown();
+      super.init(args);
+    }
+  }
+
+  @Test
+  public void testNodeMarkersRegistration() throws Exception {
+    // for this test we want to create two triggers so we must assert that the actions were created twice
+    actionInitCalled = new CountDownLatch(2);
+    // similarly we want both triggers to fire
+    triggerFiredLatch = new CountDownLatch(2);
+    TestLiveNodesListener listener = registerLiveNodesListener();
+
+    SolrClient solrClient = cluster.simGetSolrClient();
+
+    // pick overseer node
+    String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    // add a node
+    String node = cluster.simAddNode();
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+    assertEquals(1, listener.addedNodes.size());
+    assertEquals(node, listener.addedNodes.iterator().next());
+    // verify that a znode doesn't exist (no trigger)
+    String pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node;
+    assertFalse("Path " + pathAdded + " was created but there are no nodeAdded triggers",
+        cluster.getDistribStateManager().hasData(pathAdded));
+    listener.reset();
+    // stop overseer
+    log.info("====== KILL OVERSEER 1");
+    cluster.simRestartOverseer(overseerLeader);
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+    assertEquals(1, listener.lostNodes.size());
+    assertEquals(overseerLeader, listener.lostNodes.iterator().next());
+    assertEquals(0, listener.addedNodes.size());
+    // wait until the new overseer is up
+    cluster.getTimeSource().sleep(5000);
+    // verify that a znode does NOT exist - there's no nodeLost trigger,
+    // so the new overseer cleaned up existing nodeLost markers
+    String pathLost = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + overseerLeader;
+    assertFalse("Path " + pathLost + " exists", cluster.getDistribStateManager().hasData(pathLost));
+
+    listener.reset();
+
+    // set up triggers
+
+    log.info("====== ADD TRIGGERS");
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '1s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
+        "}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode(random());
+
+    // create another node
+    log.info("====== ADD NODE 1");
+    String node1 = cluster.simAddNode();
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+    assertEquals(1, listener.addedNodes.size());
+    assertEquals(node1, listener.addedNodes.iterator().next());
+    // verify that a znode exists
+    pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node1;
+    assertTrue("Path " + pathAdded + " wasn't created", cluster.getDistribStateManager().hasData(pathAdded));
+
+    cluster.getTimeSource().sleep(5000);
+    // nodeAdded marker should be consumed now by nodeAdded trigger
+    assertFalse("Path " + pathAdded + " should have been deleted",
+        cluster.getDistribStateManager().hasData(pathAdded));
+
+    listener.reset();
+    events.clear();
+    triggerFiredLatch = new CountDownLatch(1);
+    // kill overseer again
+    log.info("====== KILL OVERSEER 2");
+    cluster.simRestartOverseer(overseerLeader);
+    if (!listener.onChangeLatch.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("onChange listener didn't execute on cluster change");
+    }
+
+
+    if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
+      fail("Trigger should have fired by now");
+    }
+    assertEquals(1, events.size());
+    TriggerEvent ev = events.iterator().next();
+    List<String> nodeNames = (List<String>)ev.getProperty(TriggerEvent.NODE_NAMES);
+    assertTrue(nodeNames.contains(overseerLeader));
+    assertEquals(TriggerEventType.NODELOST, ev.getEventType());
+  }
+
+  static Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
+  static CountDownLatch listenerCreated = new CountDownLatch(1);
+  static boolean failDummyAction = false;
+
+  public static class TestTriggerListener extends TriggerListenerBase {
+    @Override
+    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+      super.init(cloudManager, config);
+      listenerCreated.countDown();
+    }
+
+    @Override
+    public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
+                                     ActionContext context, Throwable error, String message) {
+      List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
+      lst.add(new CapturedEvent(cluster.getTimeSource().getTime(), context, config, stage, actionName, event, message));
+    }
+  }
+
+  public static class TestDummyAction extends TriggerActionBase {
+
+    @Override
+    public void process(TriggerEvent event, ActionContext context) {
+      if (failDummyAction) {
+        throw new RuntimeException("failure");
+      }
+
+    }
+  }
+
+  @Test
+  public void testListeners() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}," +
+        "{'name':'test1','class':'" + TestDummyAction.class.getName() + "'}," +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    String setListenerCommand = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'foo'," +
+        "'trigger' : 'node_added_trigger'," +
+        "'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
+        "'beforeAction' : 'test'," +
+        "'afterAction' : ['test', 'test1']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'bar'," +
+        "'trigger' : 'node_added_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'beforeAction' : ['test', 'test1']," +
+        "'afterAction' : 'test'," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    listenerEvents.clear();
+    failDummyAction = false;
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+
+    assertEquals("both listeners should have fired", 2, listenerEvents.size());
+
+    cluster.getTimeSource().sleep(2000);
+
+    // check foo events
+    List<CapturedEvent> testEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + testEvents, testEvents);
+    assertEquals("foo events: " + testEvents, 5, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
+    assertEquals("test", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(3).stage);
+    assertEquals("test1", testEvents.get(3).actionName);
+
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(4).stage);
+
+    // check bar events
+    testEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", testEvents);
+    assertEquals("bar events", 4, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
+    assertEquals("test", testEvents.get(0).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
+    assertEquals("test1", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(3).stage);
+
+    // reset
+    triggerFired.set(false);
+    triggerFiredLatch = new CountDownLatch(1);
+    listenerEvents.clear();
+    failDummyAction = true;
+
+    newNode = cluster.simAddNode();
+    await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+
+    cluster.getTimeSource().sleep(2000);
+
+    // check foo events
+    testEvents = listenerEvents.get("foo");
+    assertNotNull("foo events: " + testEvents, testEvents);
+    assertEquals("foo events: " + testEvents, 4, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
+    assertEquals("test", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
+    assertEquals("test1", testEvents.get(3).actionName);
+
+    // check bar events
+    testEvents = listenerEvents.get("bar");
+    assertNotNull("bar events", testEvents);
+    assertEquals("bar events", 4, testEvents.size());
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
+    assertEquals("test", testEvents.get(0).actionName);
+
+    assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
+    assertEquals("test", testEvents.get(1).actionName);
+
+    assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
+    assertEquals("test1", testEvents.get(2).actionName);
+
+    assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
+    assertEquals("test1", testEvents.get(3).actionName);
+  }
+
+  @Test
+  public void testCooldown() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    failDummyAction = false;
+    waitForSeconds = 1;
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_cooldown_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'bar'," +
+        "'trigger' : 'node_added_cooldown_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED', 'IGNORED']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    listenerCreated = new CountDownLatch(1);
+    listenerEvents.clear();
+
+    String newNode = cluster.simAddNode();
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    assertTrue(triggerFired.get());
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(1000);
+
+    List<CapturedEvent> capturedEvents = listenerEvents.get("bar");
+    // we may get a few IGNORED events if other tests caused events within cooldown period
+    assertTrue(capturedEvents.toString(), capturedEvents.size() > 0);
+    long prevTimestamp = capturedEvents.get(capturedEvents.size() - 1).timestamp;
+
+    // reset the trigger and captured events
+    listenerEvents.clear();
+    triggerFiredLatch = new CountDownLatch(1);
+    triggerFired.compareAndSet(true, false);
+
+    String newNode2 = cluster.simAddNode();
+    await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(2000);
+
+    // there must be at least one IGNORED event due to cooldown, and one SUCCEEDED event
+    capturedEvents = listenerEvents.get("bar");
+    assertTrue(capturedEvents.toString(), capturedEvents.size() > 1);
+    for (int i = 0; i < capturedEvents.size() - 1; i++) {
+      CapturedEvent ev = capturedEvents.get(i);
+      assertEquals(ev.toString(), TriggerEventProcessorStage.IGNORED, ev.stage);
+      assertTrue(ev.toString(), ev.message.contains("cooldown"));
+    }
+    CapturedEvent ev = capturedEvents.get(capturedEvents.size() - 1);
+    assertEquals(ev.toString(), TriggerEventProcessorStage.SUCCEEDED, ev.stage);
+    // the difference between timestamps of the first SUCCEEDED and the last SUCCEEDED
+    // must be larger than cooldown period
+    assertTrue("timestamp delta is less than default cooldown period", ev.timestamp - prevTimestamp > TimeUnit.SECONDS.toNanos(ScheduledTriggers.DEFAULT_COOLDOWN_PERIOD_SECONDS));
+  }
+
+  public static class TestSearchRateAction extends TriggerActionBase {
+
+    @Override
+    public void process(TriggerEvent event, ActionContext context) throws Exception {
+      try {
+        events.add(event);
+        long currentTimeNanos = cluster.getTimeSource().getTime();
+        long eventTimeNanos = event.getEventTime();
+        long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
+        if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
+          fail(event.getSource() + " was fired before the configured waitFor period");
+        }
+        getTriggerFiredLatch().countDown();
+      } catch (Throwable t) {
+        log.debug("--throwable", t);
+        throw t;
+      }
+    }
+  }
+
+  @Test
+  public void testSearchRate() throws Exception {
+    SolrClient solrClient = cluster.simGetSolrClient();
+    String COLL1 = "collection1";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
+        "conf", 1, 2);
+    create.process(solrClient);
+    waitForState(COLL1, 10, TimeUnit.SECONDS, clusterShape(1, 2));
+
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'search_rate_trigger'," +
+        "'event' : 'searchRate'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'rate' : 1.0," +
+        "'actions' : [" +
+        "{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
+        "]" +
+        "}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand1 = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'srt'," +
+        "'trigger' : 'search_rate_trigger'," +
+        "'stage' : ['FAILED','SUCCEEDED']," +
+        "'class' : '" + TestTriggerListener.class.getName() + "'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+//    SolrParams query = params(CommonParams.Q, "*:*");
+//    for (int i = 0; i < 500; i++) {
+//      solrClient.query(COLL1, query);
+//    }
+
+    cluster.getSimClusterStateProvider().simSetCollectionValue(COLL1, "QUERY./select.requestTimes:1minRate", 500, true);
+
+    boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
+    assertTrue("The trigger did not fire at all", await);
+    // wait for listener to capture the SUCCEEDED stage
+    cluster.getTimeSource().sleep(2000);
+    assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
+    CapturedEvent ev = listenerEvents.get("srt").get(0);
+    long now = cluster.getTimeSource().getTime();
+    // verify waitFor
+    assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());
+    Map<String, Double> nodeRates = (Map<String, Double>)ev.event.getProperties().get("node");
+    assertNotNull("nodeRates", nodeRates);
+    assertTrue(nodeRates.toString(), nodeRates.size() > 0);
+    AtomicDouble totalNodeRate = new AtomicDouble();
+    nodeRates.forEach((n, r) -> totalNodeRate.addAndGet(r));
+    List<ReplicaInfo> replicaRates = (List<ReplicaInfo>)ev.event.getProperties().get("replica");
+    assertNotNull("replicaRates", replicaRates);
+    assertTrue(replicaRates.toString(), replicaRates.size() > 0);
+    AtomicDouble totalReplicaRate = new AtomicDouble();
+    replicaRates.forEach(r -> {
+      assertTrue(r.toString(), r.getVariable("rate") != null);
+      totalReplicaRate.addAndGet((Double)r.getVariable("rate"));
+    });
+    Map<String, Object> shardRates = (Map<String, Object>)ev.event.getProperties().get("shard");
+    assertNotNull("shardRates", shardRates);
+    assertEquals(shardRates.toString(), 1, shardRates.size());
+    shardRates = (Map<String, Object>)shardRates.get(COLL1);
+    assertNotNull("shardRates", shardRates);
+    assertEquals(shardRates.toString(), 1, shardRates.size());
+    AtomicDouble totalShardRate = new AtomicDouble();
+    shardRates.forEach((s, r) -> totalShardRate.addAndGet((Double)r));
+    Map<String, Double> collectionRates = (Map<String, Double>)ev.event.getProperties().get("collection");
+    assertNotNull("collectionRates", collectionRates);
+    assertEquals(collectionRates.toString(), 1, collectionRates.size());
+    Double collectionRate = collectionRates.get(COLL1);
+    assertNotNull(collectionRate);
+    assertTrue(collectionRate > 5.0);
+    assertEquals(collectionRate, totalNodeRate.get(), 5.0);
+    assertEquals(collectionRate, totalShardRate.get(), 5.0);
+    assertEquals(collectionRate, totalReplicaRate.get(), 5.0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
new file mode 100644
index 0000000..0b412cb
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Simulated environment for autoscaling tests.
+ */
+package org.apache.solr.cloud.autoscaling.sim;
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
index 42af083..c242809 100644
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
@@ -59,6 +59,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
@@ -794,7 +795,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
 
   protected void waitForBootstrapToComplete(String collectionName, String shardId) throws Exception {
     NamedList rsp;// we need to wait until bootstrap is complete otherwise the replicator thread will never start
-    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (!timeOut.hasTimedOut())  {
       rsp = invokeCdcrAction(shardToLeaderJetty.get(collectionName).get(shardId), CdcrParams.CdcrAction.BOOTSTRAP_STATUS);
       if (rsp.get(RESPONSE_STATUS).toString().equals(COMPLETED))  {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
index 329de79..500655d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.BadHdfsThreadsFilter;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
@@ -221,7 +222,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
     request.setPath("/admin/collections");
     cloudClient.request(request);
 
-    final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
+    final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cloudClient.getZkStateReader().getClusterState().hasCollection(DELETE_DATA_DIR_COLLECTION)) {
       if (timeout.hasTimedOut()) {
         throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
index 906e27b..0639479 100644
--- a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 
@@ -213,7 +214,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
       writer.writePendingUpdates();
 
       boolean found = false;
-      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       while (!timeOut.hasTimedOut())  {
         DocCollection c1 = reader.getClusterState().getCollection("c1");
         if ("y".equals(c1.getStr("x"))) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
index 76c5c0f..626374c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
@@ -201,6 +201,9 @@ public class RuleEngineTest extends SolrTestCaseJ4{
       public NodeStateProvider getNodeStateProvider() {
         return new NodeStateProvider() {
           @Override
+          public void close() throws IOException { }
+
+          @Override
           public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
             return (Map<String, Object>) MockSnitch.nodeVsTags.get(node);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
index 0bdf90c..f85b293 100644
--- a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
+++ b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java
@@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.After;
 import org.junit.Before;
@@ -318,8 +319,8 @@ class Indexer {
   ArrayList<OneIndexer> _threads = new ArrayList<>();
 
   public Indexer(OpenCloseCoreStressTest OCCST, String url, List<HttpSolrClient> clients, int numThreads, int secondsToRun, Random random) {
-    stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS);
-    nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
+    stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    nextTimeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     docsThisCycle.set(0);
     qTimesAccum.set(0);
     updateCounts.set(0);
@@ -353,7 +354,7 @@ class Indexer {
       log.info(String.format(Locale.ROOT, " s indexed: [run %,8d] [cycle %,8d] [last minute %,8d] Last core updated: %s. Seconds left in cycle %,4d",
           myId, docsThisCycle.get(), myId - lastCount, core, stopTimeout.timeLeft(TimeUnit.SECONDS)));
       lastCount = myId;
-      nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
+      nextTimeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
index e439d03..01f9199 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/AutoScalingConfig.java
@@ -520,6 +520,15 @@ public class AutoScalingConfig implements MapWriter {
     return withTriggerListenerConfigs(configs);
   }
 
+  @Override
+  public Object clone() {
+    if (jsonMap != null) {
+      return new AutoScalingConfig(jsonMap);
+    } else {
+      return new AutoScalingConfig(getPolicy(), getTriggerConfigs(), getTriggerListenerConfigs(), getProperties(), zkVersion);
+    }
+  }
+
   /**
    * Return the znode version that was used to create this configuration.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
index 09b6193..17c48d5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingCloudManager.java
@@ -24,6 +24,7 @@ import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
 
 /**
  * Base class for overriding some behavior of {@link SolrCloudManager}.
@@ -31,6 +32,7 @@ import org.apache.solr.common.util.ObjectCache;
 public class DelegatingCloudManager implements SolrCloudManager {
   private final SolrCloudManager delegate;
   private ObjectCache objectCache = new ObjectCache();
+  private TimeSource timeSource = TimeSource.NANO_TIME;
 
   public DelegatingCloudManager(SolrCloudManager delegate) {
     this.delegate = delegate;
@@ -62,6 +64,16 @@ public class DelegatingCloudManager implements SolrCloudManager {
   }
 
   @Override
+  public boolean isClosed() {
+    return delegate.isClosed();
+  }
+
+  @Override
+  public TimeSource getTimeSource() {
+    return delegate == null ? timeSource : delegate.getTimeSource();
+  }
+
+  @Override
   public SolrResponse request(SolrRequest req) throws IOException {
     return delegate.request(req);
   }
@@ -70,4 +82,9 @@ public class DelegatingCloudManager implements SolrCloudManager {
   public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
     return delegate.httpRequest(url, method, headers, payload, timeout, followRedirects);
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
index b47d1c8..2fea23b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingDistribStateManager.java
@@ -45,6 +45,11 @@ public class DelegatingDistribStateManager implements DistribStateManager {
   }
 
   @Override
+  public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+    return delegate.listData(path, watcher);
+  }
+
+  @Override
   public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
     return delegate.getData(path, watcher);
   }
@@ -60,12 +65,17 @@ public class DelegatingDistribStateManager implements DistribStateManager {
   }
 
   @Override
+  public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
+    delegate.makePath(path, data, createMode, failOnExists);
+  }
+
+  @Override
   public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
     return delegate.createData(path, data, mode);
   }
 
   @Override
-  public void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
+  public void removeData(String path, int version) throws NoSuchElementException, IOException, BadVersionException, KeeperException, InterruptedException {
     delegate.removeData(path, version);
   }
 
@@ -88,4 +98,9 @@ public class DelegatingDistribStateManager implements DistribStateManager {
   public AutoScalingConfig getAutoScalingConfig() throws InterruptedException, IOException {
     return delegate.getAutoScalingConfig();
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
index 8b717f8..9ffde0f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -40,4 +41,14 @@ public class DelegatingNodeStateProvider implements NodeStateProvider {
   public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
     return delegate.getReplicaInfo(node, keys);
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
+
+  @Override
+  public boolean isClosed() {
+    return delegate.isClosed();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
index 4318418..26aaead 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DistribStateManager.java
@@ -16,11 +16,11 @@
  */
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.NoSuchElementException;
 
+import org.apache.solr.common.SolrCloseable;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
@@ -30,7 +30,7 @@ import org.apache.zookeeper.Watcher;
 /**
  * This interface represents a distributed state repository.
  */
-public interface DistribStateManager extends Closeable {
+public interface DistribStateManager extends SolrCloseable {
 
   // state accessors
 
@@ -38,6 +38,8 @@ public interface DistribStateManager extends Closeable {
 
   List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
 
+  List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
+
   VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
 
   default VersionedData getData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
@@ -48,9 +50,19 @@ public interface DistribStateManager extends Closeable {
 
   void makePath(String path) throws AlreadyExistsException, IOException, KeeperException, InterruptedException;
 
+  void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException;
+
+  /**
+   * Create data (leaf) node at specified path.
+   * @param path base path name of the node.
+   * @param data data to be stored.
+   * @param mode creation mode.
+   * @return actual path of the node - in case of sequential nodes this will differ from the base path because
+   * of the appended sequence number.
+   */
   String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException;
 
-  void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException;
+  void removeData(String path, int version) throws NoSuchElementException, IOException, KeeperException, InterruptedException, BadVersionException;
 
   void setData(String path, byte[] data, int version) throws BadVersionException, NoSuchElementException, IOException, KeeperException, InterruptedException;
 
@@ -61,9 +73,4 @@ public interface DistribStateManager extends Closeable {
   default AutoScalingConfig getAutoScalingConfig() throws InterruptedException, IOException {
     return getAutoScalingConfig(null);
   }
-
-  @Override
-  default void close() throws IOException {
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
index dbf6836..68dfa39 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/NodeStateProvider.java
@@ -20,10 +20,12 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.solr.common.SolrCloseable;
+
 /**
  * This interface models the access to node and replica information.
  */
-public interface NodeStateProvider {
+public interface NodeStateProvider extends SolrCloseable {
   /**
    * Get the value of each tag for a given node
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
index d73ae6c..f11121d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
@@ -18,6 +18,7 @@
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -37,9 +38,12 @@ import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.emptyList;
 import static java.util.Collections.emptyMap;
@@ -56,6 +60,8 @@ import static java.util.stream.Collectors.toList;
  *
  */
 public class Policy implements MapWriter {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   public static final String POLICY = "policy";
   public static final String EACH = "#EACH";
   public static final String ANY = "#ANY";
@@ -211,16 +217,26 @@ public class Policy implements MapWriter {
     Set<String> collections = new HashSet<>();
     List<Clause> expandedClauses;
     List<Violation> violations = new ArrayList<>();
+    final int znodeVersion;
 
     private Session(List<String> nodes, SolrCloudManager cloudManager,
-                    List<Row> matrix, List<Clause> expandedClauses) {
+                    List<Row> matrix, List<Clause> expandedClauses, int znodeVersion) {
       this.nodes = nodes;
       this.cloudManager = cloudManager;
       this.matrix = matrix;
       this.expandedClauses = expandedClauses;
+      this.znodeVersion = znodeVersion;
     }
 
     Session(SolrCloudManager cloudManager) {
+      ClusterState state = null;
+      try {
+        state = cloudManager.getClusterStateProvider().getClusterState();
+        LOG.trace("-- session created with cluster state: {}", state);
+      } catch (Exception e) {
+        LOG.trace("-- session created, can't obtain cluster state", e);
+      }
+      this.znodeVersion = state != null ? state.getZNodeVersion() : -1;
       this.nodes = new ArrayList<>(cloudManager.getClusterStateProvider().getLiveNodes());
       this.cloudManager = cloudManager;
       for (String node : nodes) {
@@ -256,7 +272,7 @@ public class Policy implements MapWriter {
     }
 
     Session copy() {
-      return new Session(nodes, cloudManager, getMatrixCopy(), expandedClauses);
+      return new Session(nodes, cloudManager, getMatrixCopy(), expandedClauses, znodeVersion);
     }
 
     List<Row> getMatrixCopy() {
@@ -297,6 +313,7 @@ public class Policy implements MapWriter {
 
     @Override
     public void writeMap(EntryWriter ew) throws IOException {
+      ew.put("znodeVersion", znodeVersion);
       for (int i = 0; i < matrix.size(); i++) {
         Row row = matrix.get(i);
         ew.put(row.node, row);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
index 024c6c3..40ca619 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
@@ -30,12 +30,12 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester.Hint;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
-import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
 import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -138,7 +138,7 @@ public class PolicyHelper {
 
   public static final int SESSION_EXPIRY = 180;//3 seconds
 
-  public static MapWriter getDiagnostics(Policy policy, SolrClientCloudManager cloudManager) {
+  public static MapWriter getDiagnostics(Policy policy, SolrCloudManager cloudManager) {
     Policy.Session session = policy.createSession(cloudManager);
     List<Row> sorted = session.getSorted();
     List<Violation> violations = session.getViolations();
@@ -233,9 +233,10 @@ public class PolicyHelper {
      *
      */
     private void returnSession(SessionWrapper sessionWrapper) {
+      TimeSource timeSource = sessionWrapper.session.cloudManager.getTimeSource();
       synchronized (lockObj) {
         sessionWrapper.status = Status.EXECUTING;
-        log.info("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} ", time(MILLISECONDS),
+        log.info("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} ", time(timeSource, MILLISECONDS),
             sessionWrapper.createTime,
             this.sessionWrapper.createTime);
         if (sessionWrapper.createTime == this.sessionWrapper.createTime) {
@@ -255,13 +256,14 @@ public class PolicyHelper {
 
 
     public SessionWrapper get(SolrCloudManager cloudManager) throws IOException, InterruptedException {
+      TimeSource timeSource = cloudManager.getTimeSource();
       synchronized (lockObj) {
         if (sessionWrapper.status == Status.NULL ||
-            TimeUnit.SECONDS.convert(System.nanoTime() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
+            TimeUnit.SECONDS.convert(timeSource.getTime() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
           //no session available or the session is expired
           return createSession(cloudManager);
         } else {
-          long waitStart = time(MILLISECONDS);
+          long waitStart = time(timeSource, MILLISECONDS);
           //the session is not expired
           log.debug("reusing a session {}", this.sessionWrapper.createTime);
           if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
@@ -269,13 +271,13 @@ public class PolicyHelper {
             return sessionWrapper;
           } else {
             //status= COMPUTING it's being used for computing. computing is
-            log.debug("session being used. waiting... current time {} ", time(MILLISECONDS));
+            log.debug("session being used. waiting... current time {} ", time(timeSource, MILLISECONDS));
             try {
               lockObj.wait(10 * 1000);//wait for a max of 10 seconds
             } catch (InterruptedException e) {
               log.info("interrupted... ");
             }
-            log.debug("out of waiting curr-time:{} time-elapsed {}", time(MILLISECONDS), timeElapsed(waitStart, MILLISECONDS));
+            log.debug("out of waiting curr-time:{} time-elapsed {}", time(timeSource, MILLISECONDS), timeElapsed(timeSource, waitStart, MILLISECONDS));
             // now this thread has woken up because it got timed out after 10 seconds or it is notified after
             //the session was returned from another COMPUTING operation
             if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
@@ -289,8 +291,6 @@ public class PolicyHelper {
           }
         }
       }
-
-
     }
 
     private SessionWrapper createSession(SolrCloudManager cloudManager) throws InterruptedException, IOException {
@@ -361,7 +361,9 @@ public class PolicyHelper {
     }
 
     public SessionWrapper(Policy.Session session, SessionRef ref) {
-      lastUpdateTime = createTime = System.nanoTime();
+      lastUpdateTime = createTime = session != null ?
+          session.cloudManager.getTimeSource().getTime() :
+          TimeSource.NANO_TIME.getTime();
       this.session = session;
       this.status = Status.UNUSED;
       this.ref = ref;
@@ -372,7 +374,9 @@ public class PolicyHelper {
     }
 
     public SessionWrapper update(Policy.Session session) {
-      this.lastUpdateTime = System.nanoTime();
+      this.lastUpdateTime = session != null ?
+          session.cloudManager.getTimeSource().getTime() :
+          TimeSource.NANO_TIME.getTime();
       this.session = session;
       return this;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
index 5f7281f..930ede8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
@@ -18,6 +18,7 @@
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 
@@ -33,21 +34,25 @@ public class ReplicaInfo implements MapWriter {
   private String core, collection, shard;
   private Replica.Type type;
   private String node;
-  private Map<String, Object> variables;
+  private final Map<String, Object> variables = new HashMap<>();
 
-  public ReplicaInfo(String coll,String shard, Replica r, Map<String, Object> vals){
+  public ReplicaInfo(String coll, String shard, Replica r, Map<String, Object> vals) {
     this.name = r.getName();
     this.core = r.getCoreName();
     this.collection = coll;
     this.shard = shard;
     this.type = r.getType();
-    this.variables = vals;
+    if (vals != null) {
+      this.variables.putAll(vals);
+    }
     this.node = r.getNodeName();
   }
 
   public ReplicaInfo(String name, String core, String coll, String shard, Replica.Type type, String node, Map<String, Object> vals) {
     this.name = name;
-    this.variables = vals;
+    if (vals != null) {
+      this.variables.putAll(vals);
+    }
     this.collection = coll;
     this.shard = shard;
     this.type = type;
@@ -58,12 +63,22 @@ public class ReplicaInfo implements MapWriter {
   @Override
   public void writeMap(EntryWriter ew) throws IOException {
     ew.put(name, (MapWriter) ew1 -> {
-      if (variables != null) {
-        for (Map.Entry<String, Object> e : variables.entrySet()) {
-          ew1.put(e.getKey(), e.getValue());
-        }
+      for (Map.Entry<String, Object> e : variables.entrySet()) {
+        ew1.put(e.getKey(), e.getValue());
+      }
+      if (core != null && !variables.containsKey(ZkStateReader.CORE_NAME_PROP)) {
+        ew1.put(ZkStateReader.CORE_NAME_PROP, core);
+      }
+      if (shard != null && !variables.containsKey(ZkStateReader.SHARD_ID_PROP)) {
+        ew1.put(ZkStateReader.SHARD_ID_PROP, shard);
+      }
+      if (collection != null && !variables.containsKey(ZkStateReader.COLLECTION_PROP)) {
+        ew1.put(ZkStateReader.COLLECTION_PROP, collection);
+      }
+      if (node != null && !variables.containsKey(ZkStateReader.NODE_NAME_PROP)) {
+        ew1.put(ZkStateReader.NODE_NAME_PROP, node);
       }
-      if (type != null) ew1.put("type", type.toString());
+      if (type != null) ew1.put(ZkStateReader.REPLICA_TYPE, type.toString());
     });
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
index 8a1f8f0..55cdcee 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SolrCloudManager.java
@@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.TimeSource;
 
 /**
  * This interface abstracts the access to a SolrCloud cluster, including interactions with Zookeeper, Solr
@@ -44,16 +45,11 @@ public interface SolrCloudManager extends SolrCloseable {
 
   ObjectCache getObjectCache();
 
+  TimeSource getTimeSource();
+
   // Solr-like methods
 
   SolrResponse request(SolrRequest req) throws IOException;
 
   byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException;
-
-  // distributed queue implementation
-
-  @Override
-  default void close() {
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
index aec5f15..070869a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
@@ -38,7 +38,7 @@ import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.ANY;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
 
 public class Suggestion {
-  static final String coreidxsize = "INDEX.sizeInBytes";
+  public static final String coreidxsize = "INDEX.sizeInBytes";
   static final Map<String, ConditionType> validatetypes = new HashMap<>();
 
   public static ConditionType getTagType(String name) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 2432fb2..dfe15df 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -81,6 +81,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -1030,13 +1031,13 @@ public class CloudSolrClient extends SolrClient {
       if (!liveNodes.isEmpty()) {
         List<String> liveNodesList = new ArrayList<>(liveNodes);
         Collections.shuffle(liveNodesList, rand);
-        theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNodesList.get(0),
+        theUrlList.add(Utils.getBaseUrlForNodeName(liveNodesList.get(0),
             (String) stateProvider.getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
       }
 
     } else if (ADMIN_PATHS.contains(request.getPath())) {
       for (String liveNode : liveNodes) {
-        theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNode,
+        theUrlList.add(Utils.getBaseUrlForNodeName(liveNode,
             (String) stateProvider.getClusterProperty(ZkStateReader.URL_SCHEME,"http")));
       }
 


[28/54] [abbrv] lucene-solr:jira/solr-11702: Revert "WIP"

Posted by da...@apache.org.
Revert "WIP"

This reverts commit 48c8947c1775c7695fdd689049fc0ef0e487ebc7.


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

Branch: refs/heads/jira/solr-11702
Commit: 53a7f5b74841a642c5410b594122126433b1c059
Parents: dfaf023
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Dec 18 18:41:37 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Dec 18 18:41:37 2017 +0000

----------------------------------------------------------------------
 .../java/org/apache/lucene/search/Query.java    |  4 ++-
 .../apache/lucene/search/RewriteContext.java    | 32 --------------------
 .../org/apache/lucene/search/TermQuery.java     | 29 +++++++++++-------
 3 files changed, 21 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53a7f5b7/lucene/core/src/java/org/apache/lucene/search/Query.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Query.java b/lucene/core/src/java/org/apache/lucene/search/Query.java
index 7a3fc0d..54de63f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Query.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Query.java
@@ -70,7 +70,9 @@ public abstract class Query {
    * a PrefixQuery will be rewritten into a BooleanQuery that consists
    * of TermQuerys.
    */
-  public abstract Query rewrite(IndexReader reader, RewriteContext rewriteContext) throws IOException;
+  public Query rewrite(IndexReader reader) throws IOException {
+    return this;
+  }
 
   /**
    * Override and implement query instance equivalence properly in a subclass. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53a7f5b7/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java b/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java
deleted file mode 100644
index 8738399..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/RewriteContext.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-
-public class RewriteContext {
-
-  public TermContext buildTermContext(Term term, IndexReader reader) throws IOException {
-    return TermContext.build(reader.getContext(), term);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53a7f5b7/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index b13a9f4..925fe93 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -22,7 +22,6 @@ import java.util.Objects;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -204,17 +203,25 @@ public class TermQuery extends Query {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    if (perReaderTermState == null)
-      throw new IllegalStateException("TermQuery must be rewritten before use");
-    if (perReaderTermState.wasBuiltFor(searcher.getTopReaderContext()) == false)
-      throw new IllegalStateException("TermQuery was built against a different IndexReader");
-    return new TermWeight(searcher, scoreMode.needsScores(), boost, perReaderTermState);
-  }
+    final IndexReaderContext context = searcher.getTopReaderContext();
+    final TermContext termState;
+    if (perReaderTermState == null
+        || perReaderTermState.wasBuiltFor(context) == false) {
+      if (scoreMode.needsScores()) {
+        // make TermQuery single-pass if we don't have a PRTS or if the context
+        // differs!
+        termState = TermContext.build(context, term);
+      } else {
+        // do not compute the term state, this will help save seeks in the terms
+        // dict on segments that have a cache entry for this query
+        termState = null;
+      }
+    } else {
+      // PRTS was pre-build for this IS
+      termState = this.perReaderTermState;
+    }
 
-  @Override
-  public Query rewrite(IndexReader reader, RewriteContext rewriteContext) throws IOException {
-    TermContext tc = rewriteContext.buildTermContext(term, reader);
-    return new TermQuery(term, tc);
+    return new TermWeight(searcher, scoreMode.needsScores(), boost, termState);
   }
 
   /** Prints a user-readable version of this query. */


[11/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Remove stray logging. Relax test assertion due to timing under load.

Posted by da...@apache.org.
SOLR-11285: Remove stray logging. Relax test assertion due to timing under load.


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

Branch: refs/heads/jira/solr-11702
Commit: d02d1f1cab3fa9191a3b677a64a9ad56d2990089
Parents: 8eb5c69
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Fri Dec 15 11:30:50 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Fri Dec 15 11:30:50 2017 +0100

----------------------------------------------------------------------
 .../solr/cloud/autoscaling/sim/SimDistribStateManager.java   | 1 -
 .../solr/cloud/autoscaling/sim/TestTriggerIntegration.java   | 8 ++++----
 2 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d02d1f1c/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
index f9f17a0..cfa6265 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimDistribStateManager.java
@@ -303,7 +303,6 @@ public class SimDistribStateManager implements DistribStateManager {
     for (int i = 0; i < elements.length; i++) {
       String currentName = elements[i];
       currentPath.append('/');
-      LOG.info(" - parentNode=" + parentNode + ", currentName=" + currentName);
       n = parentNode.children != null ? parentNode.children.get(currentName) : null;
       if (n == null) {
         if (create) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d02d1f1c/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
index a05eb78..3a118f2 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
@@ -1209,9 +1209,9 @@ public class TestTriggerIntegration extends SimSolrCloudTestCase {
     assertEquals(collectionRates.toString(), 1, collectionRates.size());
     Double collectionRate = collectionRates.get(COLL1);
     assertNotNull(collectionRate);
-    assertTrue(collectionRate > 5.0);
-    assertEquals(collectionRate, totalNodeRate.get(), 5.0);
-    assertEquals(collectionRate, totalShardRate.get(), 5.0);
-    assertEquals(collectionRate, totalReplicaRate.get(), 5.0);
+    assertTrue(collectionRate > 100.0);
+    assertTrue(totalNodeRate.get() > 100.0);
+    assertTrue(totalShardRate.get() > 100.0);
+    assertTrue(totalReplicaRate.get() > 100.0);
   }
 }


[06/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11285: Simulation framework for autoscaling.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
new file mode 100644
index 0000000..5d7aa4d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueue.java
@@ -0,0 +1,599 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Predicate;
+
+import com.codahale.metrics.Timer;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.cloud.OverseerTaskQueue;
+import org.apache.solr.cloud.Stats;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.Pair;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A distributed queue that uses {@link DistribStateManager} as the underlying distributed store.
+ * Implementation based on {@link org.apache.solr.cloud.ZkDistributedQueue}
+ */
+public class GenericDistributedQueue implements DistributedQueue {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  static final String PREFIX = "qn-";
+
+  /**
+   * Theory of operation:
+   * <p>
+   * Under ordinary circumstances we neither watch nor poll for children in ZK.
+   * Instead we keep an in-memory list of known child names.  When the in-memory
+   * list is exhausted, we then fetch from ZK.
+   * <p>
+   * We only bother setting a child watcher when the queue has no children in ZK.
+   */
+  private static final Object _IMPLEMENTATION_NOTES = null;
+
+  final String dir;
+
+  final DistribStateManager stateManager;
+
+  final Stats stats;
+
+  /**
+   * A lock that guards all of the mutable state that follows.
+   */
+  private final ReentrantLock updateLock = new ReentrantLock();
+
+  /**
+   * Contains the last set of children fetched from ZK. Elements are removed from the head of
+   * this in-memory set as they are consumed from the queue.  Due to the distributed nature
+   * of the queue, elements may appear in this set whose underlying nodes have been consumed in ZK.
+   * Therefore, methods like {@link #peek()} have to double-check actual node existence, and methods
+   * like {@link #poll()} must resolve any races by attempting to delete the underlying node.
+   */
+  private TreeSet<String> knownChildren = new TreeSet<>();
+
+  /**
+   * Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
+   */
+  private final Condition changed = updateLock.newCondition();
+
+  private boolean isDirty = true;
+
+  private int watcherCount = 0;
+
+  private final int maxQueueSize;
+
+  /**
+   * If {@link #maxQueueSize} is set, the number of items we can queue without rechecking the server.
+   */
+  private final AtomicInteger offerPermits = new AtomicInteger(0);
+
+  public GenericDistributedQueue(DistribStateManager stateManager, String dir) {
+    this(stateManager, dir, new Stats());
+  }
+
+  public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats) {
+    this(stateManager, dir, stats, 0);
+  }
+
+  public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats, int maxQueueSize) {
+    this.dir = dir;
+
+    try {
+      if (!stateManager.hasData(dir)) {
+        try {
+          stateManager.makePath(dir);
+        } catch (AlreadyExistsException e) {
+          // ignore
+        }
+      }
+    } catch (IOException | KeeperException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
+    }
+
+    this.stateManager = stateManager;
+    this.stats = stats;
+    this.maxQueueSize = maxQueueSize;
+  }
+
+  /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty.
+   *
+   * @return data at the first element of the queue, or null.
+   */
+  @Override
+  public byte[] peek() throws Exception {
+    Timer.Context time = stats.time(dir + "_peek");
+    try {
+      return firstElement();
+    } finally {
+      time.stop();
+    }
+  }
+
+  /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty and block is false.
+   *
+   * @param block if true, blocks until an element enters the queue
+   * @return data at the first element of the queue, or null.
+   */
+  @Override
+  public byte[] peek(boolean block) throws Exception {
+    return block ? peek(Long.MAX_VALUE) : peek();
+  }
+
+  /**
+   * Returns the data at the first element of the queue, or null if the queue is
+   * empty after wait ms.
+   *
+   * @param wait max wait time in ms.
+   * @return data at the first element of the queue, or null.
+   */
+  @Override
+  public byte[] peek(long wait) throws Exception {
+    Preconditions.checkArgument(wait > 0);
+    Timer.Context time;
+    if (wait == Long.MAX_VALUE) {
+      time = stats.time(dir + "_peek_wait_forever");
+    } else {
+      time = stats.time(dir + "_peek_wait" + wait);
+    }
+    updateLock.lockInterruptibly();
+    try {
+      long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
+      while (waitNanos > 0) {
+        byte[] result = firstElement();
+        if (result != null) {
+          return result;
+        }
+        waitNanos = changed.awaitNanos(waitNanos);
+      }
+      return null;
+    } finally {
+      updateLock.unlock();
+      time.stop();
+    }
+  }
+
+  /**
+   * Attempts to remove the head of the queue and return it. Returns null if the
+   * queue is empty.
+   *
+   * @return Head of the queue or null.
+   */
+  @Override
+  public byte[] poll() throws Exception {
+    Timer.Context time = stats.time(dir + "_poll");
+    try {
+      return removeFirst();
+    } finally {
+      time.stop();
+    }
+  }
+
+  /**
+   * Attempts to remove the head of the queue and return it.
+   *
+   * @return The former head of the queue
+   */
+  @Override
+  public byte[] remove() throws Exception {
+    Timer.Context time = stats.time(dir + "_remove");
+    try {
+      byte[] result = removeFirst();
+      if (result == null) {
+        throw new NoSuchElementException();
+      }
+      return result;
+    } finally {
+      time.stop();
+    }
+  }
+
+  public void remove(Collection<String> paths) throws Exception {
+    if (paths.isEmpty()) return;
+    List<Op> ops = new ArrayList<>();
+    for (String path : paths) {
+      ops.add(Op.delete(dir + "/" + path, -1));
+    }
+    for (int from = 0; from < ops.size(); from += 1000) {
+      int to = Math.min(from + 1000, ops.size());
+      if (from < to) {
+        try {
+          stateManager.multi(ops.subList(from, to));
+        } catch (NoSuchElementException e) {
+          // don't know which nodes are not exist, so try to delete one by one node
+          for (int j = from; j < to; j++) {
+            try {
+              stateManager.removeData(ops.get(j).getPath(), -1);
+            } catch (NoSuchElementException e2) {
+              LOG.debug("Can not remove node which is not exist : " + ops.get(j).getPath());
+            }
+          }
+        }
+      }
+    }
+
+    int cacheSizeBefore = knownChildren.size();
+    knownChildren.removeAll(paths);
+    if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
+      stats.setQueueLength(knownChildren.size());
+    } else {
+      // There are elements get deleted but not present in the cache,
+      // the cache seems not valid anymore
+      knownChildren.clear();
+      isDirty = true;
+    }
+  }
+
+  /**
+   * Removes the head of the queue and returns it, blocks until it succeeds.
+   *
+   * @return The former head of the queue
+   */
+  @Override
+  public byte[] take() throws Exception {
+    // Same as for element. Should refactor this.
+    Timer.Context timer = stats.time(dir + "_take");
+    updateLock.lockInterruptibly();
+    try {
+      while (true) {
+        byte[] result = removeFirst();
+        if (result != null) {
+          return result;
+        }
+        changed.await();
+      }
+    } finally {
+      updateLock.unlock();
+      timer.stop();
+    }
+  }
+
+  /**
+   * Inserts data into queue.  If there are no other queue consumers, the offered element
+   * will be immediately visible when this method returns.
+   */
+  @Override
+  public void offer(byte[] data) throws Exception {
+    Timer.Context time = stats.time(dir + "_offer");
+    try {
+      while (true) {
+        try {
+          if (maxQueueSize > 0) {
+            if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) {
+              // If a max queue size is set, check it before creating a new queue item.
+              if (!stateManager.hasData(dir)) {
+                // jump to the code below, which tries to create dir if it doesn't exist
+                throw new NoSuchElementException();
+              }
+              List<String> children = stateManager.listData(dir);
+              int remainingCapacity = maxQueueSize - children.size();
+              if (remainingCapacity <= 0) {
+                throw new IllegalStateException("queue is full");
+              }
+
+              // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
+              offerPermits.set(remainingCapacity / 100);
+            }
+          }
+
+          // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
+          // This will get set again when the watcher actually fires, but that's ok.
+          stateManager.createData(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL);
+          isDirty = true;
+          return;
+        } catch (NoSuchElementException e) {
+          try {
+            stateManager.createData(dir, new byte[0], CreateMode.PERSISTENT);
+          } catch (NoSuchElementException ne) {
+            // someone created it
+          }
+        }
+      }
+    } finally {
+      time.stop();
+    }
+  }
+
+  public Stats getZkStats() {
+    return stats;
+  }
+
+  @Override
+  public Map<String, Object> getStats() {
+    if (stats == null) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> res = new HashMap<>();
+    res.put("queueLength", stats.getQueueLength());
+    final Map<String, Object> statsMap = new HashMap<>();
+    res.put("stats", statsMap);
+    stats.getStats().forEach((op, stat) -> {
+      final Map<String, Object> statMap = new HashMap<>();
+      statMap.put("success", stat.success.get());
+      statMap.put("errors", stat.errors.get());
+      final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
+      statMap.put("failureDetails", failed);
+      stat.failureDetails.forEach(failedOp -> {
+        Map<String, Object> fo = new HashMap<>();
+        fo.put("req", failedOp.req);
+        fo.put("resp", failedOp.resp);
+      });
+      statsMap.put(op, statMap);
+    });
+    return res;
+  }
+
+  /**
+   * Returns the name if the first known child node, or {@code null} if the queue is empty.
+   * This is the only place {@link #knownChildren} is ever updated!
+   * The caller must double check that the actual node still exists, since the in-memory
+   * list is inherently stale.
+   */
+  private String firstChild(boolean remove, boolean refetchIfDirty) throws Exception {
+    updateLock.lockInterruptibly();
+    try {
+      // We always return from cache first, the cache will be cleared if the node is not exist
+      if (!knownChildren.isEmpty() && !(isDirty && refetchIfDirty)) {
+        return remove ? knownChildren.pollFirst() : knownChildren.first();
+      }
+
+      if (!isDirty && knownChildren.isEmpty()) {
+        return null;
+      }
+
+      // Dirty, try to fetch an updated list of children from ZK.
+      // Only set a new watcher if there isn't already a watcher.
+      ChildWatcher newWatcher = (watcherCount == 0) ? new ChildWatcher() : null;
+      knownChildren = fetchZkChildren(newWatcher);
+      if (newWatcher != null) {
+        watcherCount++; // watcher was successfully set
+      }
+      isDirty = false;
+      if (knownChildren.isEmpty()) {
+        return null;
+      }
+      changed.signalAll();
+      return remove ? knownChildren.pollFirst() : knownChildren.first();
+    } finally {
+      updateLock.unlock();
+    }
+  }
+
+  /**
+   * Return the current set of children from ZK; does not change internal state.
+   */
+  TreeSet<String> fetchZkChildren(Watcher watcher) throws Exception {
+    while (true) {
+      try {
+        TreeSet<String> orderedChildren = new TreeSet<>();
+
+        List<String> childNames = stateManager.listData(dir, watcher);
+        stats.setQueueLength(childNames.size());
+        for (String childName : childNames) {
+          // Check format
+          if (!childName.regionMatches(0, PREFIX, 0, PREFIX.length())) {
+            LOG.debug("Found child node with improper name: " + childName);
+            continue;
+          }
+          orderedChildren.add(childName);
+        }
+        return orderedChildren;
+      } catch (NoSuchElementException e) {
+        try {
+          stateManager.makePath(dir);
+        } catch (AlreadyExistsException e2) {
+          // ignore
+        }
+        // go back to the loop and try again
+      }
+    }
+  }
+
+  /**
+   * Return the currently-known set of elements, using child names from memory. If no children are found, or no
+   * children pass {@code acceptFilter}, waits up to {@code waitMillis} for at least one child to become available.
+   * <p>
+   * Package-private to support {@link OverseerTaskQueue} specifically.</p>
+   */
+  @Override
+  public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
+    List<String> foundChildren = new ArrayList<>();
+    long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
+    boolean first = true;
+    while (true) {
+      // Trigger a refresh, but only force it if this is not the first iteration.
+      firstChild(false, !first);
+
+      updateLock.lockInterruptibly();
+      try {
+        for (String child : knownChildren) {
+          if (acceptFilter.test(child)) {
+            foundChildren.add(child);
+          }
+        }
+        if (!foundChildren.isEmpty()) {
+          break;
+        }
+        if (waitNanos <= 0) {
+          break;
+        }
+
+        // If this is our first time through, force a refresh before waiting.
+        if (first) {
+          first = false;
+          continue;
+        }
+
+        waitNanos = changed.awaitNanos(waitNanos);
+      } finally {
+        updateLock.unlock();
+      }
+
+      if (!foundChildren.isEmpty()) {
+        break;
+      }
+    }
+
+    // Technically we could restart the method if we fail to actually obtain any valid children
+    // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
+    // much more sophisticated waitNanos tracking.
+    List<Pair<String, byte[]>> result = new ArrayList<>();
+    for (String child : foundChildren) {
+      if (result.size() >= max) {
+        break;
+      }
+      try {
+        VersionedData data = stateManager.getData(dir + "/" + child);
+        result.add(new Pair<>(child, data.getData()));
+      } catch (NoSuchElementException e) {
+        // Another client deleted the node first, remove the in-memory and continue.
+        updateLock.lockInterruptibly();
+        try {
+          knownChildren.remove(child);
+        } finally {
+          updateLock.unlock();
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Return the head of the queue without modifying the queue.
+   *
+   * @return the data at the head of the queue.
+   */
+  private byte[] firstElement() throws Exception {
+    while (true) {
+      String firstChild = firstChild(false, false);
+      if (firstChild == null) {
+        return null;
+      }
+      try {
+        VersionedData data = stateManager.getData(dir + "/" + firstChild);
+        return data != null ? data.getData() : null;
+      } catch (NoSuchElementException e) {
+        // Another client deleted the node first, remove the in-memory and retry.
+        updateLock.lockInterruptibly();
+        try {
+          // Efficient only for single-consumer
+          knownChildren.clear();
+          isDirty = true;
+        } finally {
+          updateLock.unlock();
+        }
+      }
+    }
+  }
+
+  private byte[] removeFirst() throws Exception {
+    while (true) {
+      String firstChild = firstChild(true, false);
+      if (firstChild == null) {
+        return null;
+      }
+      try {
+        String path = dir + "/" + firstChild;
+        VersionedData result = stateManager.getData(path);
+        stateManager.removeData(path, -1);
+        stats.setQueueLength(knownChildren.size());
+        return result.getData();
+      } catch (NoSuchElementException e) {
+        // Another client deleted the node first, remove the in-memory and retry.
+        updateLock.lockInterruptibly();
+        try {
+          // Efficient only for single-consumer
+          knownChildren.clear();
+          isDirty = true;
+        } finally {
+          updateLock.unlock();
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting int watcherCount() throws InterruptedException {
+    updateLock.lockInterruptibly();
+    try {
+      return watcherCount;
+    } finally {
+      updateLock.unlock();
+    }
+  }
+
+  @VisibleForTesting boolean isDirty() throws InterruptedException {
+    updateLock.lockInterruptibly();
+    try {
+      return isDirty;
+    } finally {
+      updateLock.unlock();
+    }
+  }
+
+  @VisibleForTesting class ChildWatcher implements Watcher {
+
+    @Override
+    public void process(WatchedEvent event) {
+      // session events are not change events, and do not remove the watcher; except for Expired
+      if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
+        return;
+      }
+      updateLock.lock();
+      try {
+        isDirty = true;
+        watcherCount--;
+        // optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
+        changed.signalAll();
+      } finally {
+        updateLock.unlock();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
new file mode 100644
index 0000000..12b4af8
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/GenericDistributedQueueFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.IOException;
+
+import org.apache.solr.client.solrj.cloud.DistributedQueue;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+
+/**
+ * Factory for {@link GenericDistributedQueue}.
+ */
+public class GenericDistributedQueueFactory implements DistributedQueueFactory {
+
+  private final DistribStateManager stateManager;
+
+  public GenericDistributedQueueFactory(DistribStateManager stateManager) {
+    this.stateManager = stateManager;
+  }
+
+  @Override
+  public DistributedQueue makeQueue(String path) throws IOException {
+    return new GenericDistributedQueue(stateManager, path);
+  }
+
+  @Override
+  public void removeQueue(String path) throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
new file mode 100644
index 0000000..45cd66b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
@@ -0,0 +1,99 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.common.cloud.LiveNodesListener;
+
+/**
+ * This class represents a set of live nodes and allows adding listeners to track their state.
+ */
+public class LiveNodesSet {
+
+  private final Set<String> set = ConcurrentHashMap.newKeySet();
+  private final Set<LiveNodesListener> listeners = ConcurrentHashMap.newKeySet();
+
+  public Set<String> get() {
+    return Collections.unmodifiableSet(set);
+  }
+
+  public void registerLiveNodesListener(LiveNodesListener listener) {
+    listeners.add(listener);
+  }
+
+  public void removeLiveNodesListener(LiveNodesListener listener) {
+    listeners.remove(listener);
+  }
+
+  private void fireListeners(SortedSet<String> oldNodes, SortedSet<String> newNodes) {
+    for (LiveNodesListener listener : listeners) {
+      listener.onChange(oldNodes, newNodes);
+    }
+  }
+
+  public boolean isEmpty() {
+    return set.isEmpty();
+  }
+
+  public boolean contains(String id) {
+    return set.contains(id);
+  }
+
+  public synchronized boolean add(String id) {
+    if (set.contains(id)) {
+      return false;
+    }
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    set.add(id);
+    TreeSet<String> newNodes = new TreeSet<>(set);
+    fireListeners(oldNodes, newNodes);
+    return true;
+  }
+
+  public synchronized boolean addAll(Collection<String> nodes) {
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    boolean changed = set.addAll(nodes);
+    TreeSet<String> newNodes = new TreeSet<>(set);
+    if (changed) {
+      fireListeners(oldNodes, newNodes);
+    }
+    return changed;
+  }
+
+  public synchronized boolean remove(String id) {
+    if (!set.contains(id)) {
+      return false;
+    }
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    set.remove(id);
+    TreeSet<String> newNodes = new TreeSet<>(set);
+    fireListeners(oldNodes, newNodes);
+    return true;
+  }
+
+  public synchronized void clear() {
+    TreeSet<String> oldNodes = new TreeSet<>(set);
+    set.clear();
+    fireListeners(oldNodes, Collections.emptySortedSet());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d6d2e3b2/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
new file mode 100644
index 0000000..bde4b41
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -0,0 +1,607 @@
+/*
+ * 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.autoscaling.sim;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
+import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
+import org.apache.solr.client.solrj.impl.ClusterStateProvider;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.client.solrj.response.SolrResponseBase;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
+import org.apache.solr.cloud.autoscaling.OverseerTriggerThread;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.ObjectCache;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CloudConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.OverseerCollectionMessageHandler.REQUESTID;
+
+/**
+ * Simulated {@link SolrCloudManager}.
+ */
+public class SimCloudManager implements SolrCloudManager {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final SimDistribStateManager stateManager;
+  private final SimClusterStateProvider clusterStateProvider;
+  private final SimNodeStateProvider nodeStateProvider;
+  private final AutoScalingHandler autoScalingHandler;
+  private final LiveNodesSet liveNodesSet = new LiveNodesSet();
+  private final DistributedQueueFactory queueFactory;
+  private final ObjectCache objectCache = new ObjectCache();
+  private TimeSource timeSource;
+
+  private final List<SolrInputDocument> systemColl = Collections.synchronizedList(new ArrayList<>());
+  private final ExecutorService simCloudManagerPool;
+  private final Map<String, AtomicLong> opCounts = new ConcurrentHashMap<>();
+
+
+  private Overseer.OverseerThread triggerThread;
+  private ThreadGroup triggerThreadGroup;
+  private SolrResourceLoader loader;
+
+  private static int nodeIdPort = 10000;
+
+  /**
+   * Create a simulated cluster. This cluster uses the following components:
+   * <ul>
+   *   <li>{@link SimDistribStateManager} with non-shared root node.</li>
+   *   <li>{@link SimClusterStateProvider}</li>
+   *   <li>{@link SimNodeStateProvider}, where node values are automatically initialized when using
+   *   {@link #simAddNode()} method.</li>
+   *   <li>{@link GenericDistributedQueueFactory} that uses {@link SimDistribStateManager} as its storage.</li>
+   *   <li>an instance of {@link AutoScalingHandler} for managing AutoScalingConfig.</li>
+   *   <li>an instance of {@link OverseerTriggerThread} for managing triggers and processing events.</li>
+   * </ul>
+   * @param timeSource time source to use.
+   */
+  public SimCloudManager(TimeSource timeSource) throws Exception {
+    this.stateManager = new SimDistribStateManager(SimDistribStateManager.createNewRootNode());
+    this.loader = new SolrResourceLoader();
+    // init common paths
+    stateManager.makePath(ZkStateReader.CLUSTER_STATE);
+    stateManager.makePath(ZkStateReader.CLUSTER_PROPS);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
+    stateManager.makePath(ZkStateReader.LIVE_NODES_ZKNODE);
+    stateManager.makePath(ZkStateReader.ROLES);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
+    stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+
+    this.timeSource = timeSource != null ? timeSource : TimeSource.NANO_TIME;
+    this.clusterStateProvider = new SimClusterStateProvider(liveNodesSet, this);
+    this.nodeStateProvider = new SimNodeStateProvider(liveNodesSet, this.stateManager, this.clusterStateProvider, null);
+    this.queueFactory = new GenericDistributedQueueFactory(stateManager);
+    this.simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
+    this.autoScalingHandler = new AutoScalingHandler(this, loader);
+    triggerThreadGroup = new ThreadGroup("Simulated Overseer autoscaling triggers");
+    OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
+        new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
+    triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
+    triggerThread.start();
+  }
+
+  // ---------- simulator setup methods -----------
+
+  /**
+   * Create a cluster with the specified number of nodes. Node metrics are pre-populated.
+   * @param numNodes number of nodes to create
+   * @param timeSource time source
+   * @return instance of simulated cluster
+   */
+  public static SimCloudManager createCluster(int numNodes, TimeSource timeSource) throws Exception {
+    SimCloudManager cloudManager = new SimCloudManager(timeSource);
+    for (int i = 1; i <= numNodes; i++) {
+      Map<String, Object> values = createNodeValues(null);
+//      if (i == 1) { // designated Overseer ?
+        //values.put(ImplicitSnitch.NODEROLE, "overseer");
+//      }
+      String nodeId = (String)values.get(ImplicitSnitch.NODE);
+      cloudManager.getSimClusterStateProvider().simAddNode(nodeId);
+      cloudManager.getSimNodeStateProvider().simSetNodeValues(nodeId, values);
+    }
+    return cloudManager;
+  }
+
+  /**
+   * Create a cluster initialized from the provided cluster state.
+   * @param initialState existing cluster state
+   * @param timeSource time source
+   * @return instance of simulated cluster with the same layout as the provided cluster state.
+   */
+  public static SimCloudManager createCluster(ClusterState initialState, TimeSource timeSource) throws Exception {
+    SimCloudManager cloudManager = new SimCloudManager(timeSource);
+    cloudManager.getSimClusterStateProvider().simSetClusterState(initialState);
+    for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
+      cloudManager.getSimNodeStateProvider().simSetNodeValues(node, createNodeValues(node));
+    }
+    return cloudManager;
+  }
+
+  /**
+   * Create simulated node values (metrics) for a node.
+   * @param nodeName node name (eg. '127.0.0.1:10000_solr'). If null then a new node name will be
+   *                 created using sequentially increasing port number.
+   * @return node values
+   */
+  public static Map<String, Object> createNodeValues(String nodeName) {
+    Map<String, Object> values = new HashMap<>();
+    String host, nodeId;
+    int port;
+    if (nodeName == null) {
+      host = "127.0.0.1";
+      port = nodeIdPort++;
+      nodeId = host + ":" + port + "_solr";
+      values.put("ip_1", "127");
+      values.put("ip_2", "0");
+      values.put("ip_3", "0");
+      values.put("ip_4", "1");
+    } else {
+      String[] hostPortCtx = nodeName.split(":");
+      if (hostPortCtx.length != 2) {
+        throw new RuntimeException("Invalid nodeName " + nodeName);
+      }
+      host = hostPortCtx[0];
+      String[] portCtx = hostPortCtx[1].split("_");
+      if (portCtx.length != 2) {
+        throw new RuntimeException("Invalid port_context in nodeName " + nodeName);
+      }
+      port = Integer.parseInt(portCtx[0]);
+      nodeId = host + ":" + port + "_" + portCtx[1];
+      String[] ip = host.split("\\.");
+      if (ip.length == 4) {
+        values.put("ip_1", ip[0]);
+        values.put("ip_2", ip[1]);
+        values.put("ip_3", ip[2]);
+        values.put("ip_4", ip[3]);
+      }
+    }
+    values.put(ImplicitSnitch.HOST, host);
+    values.put(ImplicitSnitch.PORT, port);
+    values.put(ImplicitSnitch.NODE, nodeId);
+    values.put(ImplicitSnitch.CORES, 0);
+    values.put(ImplicitSnitch.DISK, 1000);
+    values.put(ImplicitSnitch.SYSLOADAVG, 1.0);
+    values.put(ImplicitSnitch.HEAPUSAGE, 123450000);
+    values.put("sysprop.java.version", System.getProperty("java.version"));
+    values.put("sysprop.java.vendor", System.getProperty("java.vendor"));
+    // fake some metrics expected in tests
+    values.put("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count", 0);
+    values.put("metrics:solr.jvm:buffers.direct.Count", 0);
+    return values;
+  }
+
+  /**
+   * Get the instance of {@link SolrResourceLoader} that is used by the cluster components.
+   */
+  public SolrResourceLoader getLoader() {
+    return loader;
+  }
+
+  /**
+   * Add a new node and initialize its node values (metrics). The
+   * /live_nodes list is updated with the new node id.
+   * @return new node id
+   */
+  public String simAddNode() throws Exception {
+    Map<String, Object> values = createNodeValues(null);
+    String nodeId = (String)values.get(ImplicitSnitch.NODE);
+    clusterStateProvider.simAddNode(nodeId);
+    nodeStateProvider.simSetNodeValues(nodeId, values);
+    LOG.trace("-- added node " + nodeId);
+    return nodeId;
+  }
+
+  /**
+   * Remove a node from the cluster. This simulates a node lost scenario.
+   * Node id is removed from the /live_nodes list.
+   * @param nodeId node id
+   * @param withValues when true, remove also simulated node values. If false
+   *                   then node values are retained to later simulate
+   *                   a node that comes back up
+   */
+  public void simRemoveNode(String nodeId, boolean withValues) throws Exception {
+    clusterStateProvider.simRemoveNode(nodeId);
+    if (withValues) {
+      nodeStateProvider.simRemoveNodeValues(nodeId);
+    }
+    LOG.trace("-- removed node " + nodeId);
+  }
+
+  /**
+   * Remove a number of randomly selected nodes
+   * @param number number of nodes to remove
+   * @param withValues when true, remove also simulated node values. If false
+   *                   then node values are retained to later simulate
+   *                   a node that comes back up
+   * @param random random
+   */
+  public void simRemoveRandomNodes(int number, boolean withValues, Random random) throws Exception {
+    List<String> nodes = new ArrayList<>(liveNodesSet.get());
+    Collections.shuffle(nodes, random);
+    int count = Math.min(number, nodes.size());
+    for (int i = 0; i < count; i++) {
+      simRemoveNode(nodes.get(i), withValues);
+    }
+  }
+
+  /**
+   * Clear the (simulated) .system collection.
+   */
+  public void simClearSystemCollection() {
+    systemColl.clear();
+  }
+
+  /**
+   * Get the content of (simulated) .system collection.
+   * @return documents in the collection, in chronological order starting from the oldest.
+   */
+  public List<SolrInputDocument> simGetSystemCollection() {
+    return systemColl;
+  }
+
+  /**
+   * Get a {@link SolrClient} implementation where calls are forwarded to this
+   * instance of the cluster.
+   * @return simulated SolrClient.
+   */
+  public SolrClient simGetSolrClient() {
+    return new SolrClient() {
+      @Override
+      public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
+        SolrResponse rsp = SimCloudManager.this.request(request);
+        return rsp.getResponse();
+      }
+
+      @Override
+      public void close() throws IOException {
+
+      }
+    };
+  }
+
+  /**
+   * Simulate the effect of restarting Overseer leader - in this case this means restarting the
+   * OverseerTriggerThread and optionally killing a node.
+   * @param killNodeId optional nodeId to kill. If null then don't kill any node, just restart the thread
+   */
+  public void simRestartOverseer(String killNodeId) throws Exception {
+    LOG.info("=== Restarting OverseerTriggerThread and clearing object cache...");
+    triggerThread.interrupt();
+    IOUtils.closeQuietly(triggerThread);
+    if (killNodeId != null) {
+      simRemoveNode(killNodeId, true);
+    }
+    objectCache.clear();
+    OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
+        new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
+    triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
+    triggerThread.start();
+
+  }
+
+  /**
+   * Submit a task to execute in a thread pool.
+   * @param callable task to execute
+   * @return future to obtain results
+   */
+  public <T> Future<T> submit(Callable<T> callable) {
+    return simCloudManagerPool.submit(callable);
+  }
+
+  // ---------- type-safe methods to obtain simulator components ----------
+  public SimClusterStateProvider getSimClusterStateProvider() {
+    return clusterStateProvider;
+  }
+
+  public SimNodeStateProvider getSimNodeStateProvider() {
+    return nodeStateProvider;
+  }
+
+  public SimDistribStateManager getSimDistribStateManager() {
+    return stateManager;
+  }
+
+  public LiveNodesSet getLiveNodesSet() {
+    return liveNodesSet;
+  }
+
+  /**
+   * Get the number and type of operations processed by this cluster.
+   */
+  public Map<String, AtomicLong> simGetOpCounts() {
+    return opCounts;
+  }
+
+  /**
+   * Get the number of processed operations of a specified type.
+   * @param op operation name, eg. MOVEREPLICA
+   * @return number of operations
+   */
+  public long simGetOpCount(String op) {
+    AtomicLong count = opCounts.get(op);
+    return count != null ? count.get() : 0L;
+  }
+
+  // --------- interface methods -----------
+
+
+  @Override
+  public ObjectCache getObjectCache() {
+    return objectCache;
+  }
+
+  @Override
+  public TimeSource getTimeSource() {
+    return timeSource;
+  }
+
+  @Override
+  public ClusterStateProvider getClusterStateProvider() {
+    return clusterStateProvider;
+  }
+
+  @Override
+  public NodeStateProvider getNodeStateProvider() {
+    return nodeStateProvider;
+  }
+
+  @Override
+  public DistribStateManager getDistribStateManager() {
+    return stateManager;
+  }
+
+  @Override
+  public DistributedQueueFactory getDistributedQueueFactory() {
+    return queueFactory;
+  }
+
+  @Override
+  public SolrResponse request(SolrRequest req) throws IOException {
+    try {
+      Future<SolrResponse> rsp = submit(() -> simHandleSolrRequest(req));
+      return rsp.get();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  private void incrementCount(String op) {
+    AtomicLong count = opCounts.computeIfAbsent(op, o -> new AtomicLong());
+    count.incrementAndGet();
+  }
+
+  /**
+   * Handler method for autoscaling requests. NOTE: only a specific subset of autoscaling requests is
+   * supported!
+   * @param req autoscaling request
+   * @return results
+   */
+  public SolrResponse simHandleSolrRequest(SolrRequest req) throws IOException, InterruptedException {
+    // pay the penalty for remote request, at least 5 ms
+    timeSource.sleep(5);
+
+    LOG.trace("--- got SolrRequest: " + req.getMethod() + " " + req.getPath() +
+        (req.getParams() != null ? " " + req.getParams().toQueryString() : ""));
+    if (req.getPath() != null && req.getPath().startsWith("/admin/autoscaling") ||
+        req.getPath().startsWith("/cluster/autoscaling")) {
+      incrementCount("autoscaling");
+      ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
+      params.set(CommonParams.PATH, req.getPath());
+      LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
+      RequestWriter.ContentWriter cw = req.getContentWriter("application/json");
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cw.write(baos);
+      String payload = baos.toString("UTF-8");
+      LOG.trace("-- payload: {}", payload);
+      queryRequest.setContentStreams(Collections.singletonList(new ContentStreamBase.StringStream(payload)));
+      queryRequest.getContext().put("httpMethod", req.getMethod().toString());
+      SolrQueryResponse queryResponse = new SolrQueryResponse();
+      autoScalingHandler.handleRequest(queryRequest, queryResponse);
+      if (queryResponse.getException() != null) {
+        throw new IOException(queryResponse.getException());
+      }
+      SolrResponse rsp = new SolrResponseBase();
+      rsp.setResponse(queryResponse.getValues());
+      LOG.trace("-- response: {}", rsp);
+      return rsp;
+    }
+    if (req instanceof UpdateRequest) {
+      incrementCount("update");
+      // support only updates to the system collection
+      UpdateRequest ureq = (UpdateRequest)req;
+      if (ureq.getCollection() == null || !ureq.getCollection().equals(CollectionAdminParams.SYSTEM_COLL)) {
+        throw new UnsupportedOperationException("Only .system updates are supported but got: " + req);
+      }
+      List<SolrInputDocument> docs = ureq.getDocuments();
+      if (docs != null) {
+        systemColl.addAll(docs);
+      }
+      return new UpdateResponse();
+    }
+    // support only a specific subset of collection admin ops
+    if (!(req instanceof CollectionAdminRequest)) {
+      throw new UnsupportedOperationException("Only some CollectionAdminRequest-s are supported: " + req.getClass().getName());
+    }
+    SolrParams params = req.getParams();
+    String a = params.get(CoreAdminParams.ACTION);
+    SolrResponse rsp = new SolrResponseBase();
+    rsp.setResponse(new NamedList<>());
+    if (a != null) {
+      CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(a);
+      if (action == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
+      }
+      LOG.debug("Invoking Collection Action :{} with params {}", action.toLower(), req.getParams().toQueryString());
+      NamedList results = new NamedList();
+      rsp.setResponse(results);
+      incrementCount(action.name());
+      switch (action) {
+        case REQUESTSTATUS:
+          // we complete all async ops immediately
+          String requestId = req.getParams().get(REQUESTID);
+          SimpleOrderedMap<String> status = new SimpleOrderedMap<>();
+          status.add("state", RequestStatusState.COMPLETED.getKey());
+          status.add("msg", "found [" + requestId + "] in completed tasks");
+          results.add("status", status);
+          results.add("success", "");
+          // ExecutePlanAction expects a specific response class
+          rsp = new CollectionAdminRequest.RequestStatusResponse();
+          rsp.setResponse(results);
+          break;
+        case DELETESTATUS:
+          requestId = req.getParams().get(REQUESTID);
+          results.add("status", "successfully removed stored response for [" + requestId + "]");
+          results.add("success", "");
+          break;
+        case CREATE:
+          try {
+            clusterStateProvider.simCreateCollection(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case DELETE:
+          clusterStateProvider.simDeleteCollection(req.getParams().get(CommonParams.NAME),
+              req.getParams().get(CommonAdminParams.ASYNC), results);
+          break;
+        case LIST:
+          results.add("collections", clusterStateProvider.simListCollections());
+          break;
+        case ADDREPLICA:
+          try {
+            clusterStateProvider.simAddReplica(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case MOVEREPLICA:
+          try {
+            clusterStateProvider.simMoveReplica(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case OVERSEERSTATUS:
+          if (req.getParams().get(CommonAdminParams.ASYNC) != null) {
+            results.add(REQUESTID, req.getParams().get(CommonAdminParams.ASYNC));
+          }
+          if (!liveNodesSet.get().isEmpty()) {
+            results.add("leader", liveNodesSet.get().iterator().next());
+          }
+          results.add("overseer_queue_size", 0);
+          results.add("overseer_work_queue_size", 0);
+          results.add("overseer_collection_queue_size", 0);
+          results.add("success", "");
+          break;
+        case ADDROLE:
+          nodeStateProvider.simAddNodeValue(req.getParams().get("node"), "nodeRole", req.getParams().get("role"));
+          break;
+        case CREATESHARD:
+          try {
+            clusterStateProvider.simCreateShard(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        case SPLITSHARD:
+          try {
+            clusterStateProvider.simSplitShard(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+          break;
+        default:
+          throw new UnsupportedOperationException("Unsupported collection admin action=" + action + " in request: " + req.getParams());
+      }
+    } else {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "action is a required param in request: " + req.getParams());
+    }
+    return rsp;
+
+  }
+
+  /**
+   * HTTP requests are not supported by this implementation.
+   */
+  @Override
+  public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
+    throw new UnsupportedOperationException("general HTTP requests are not supported yet");
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.closeQuietly(clusterStateProvider);
+    IOUtils.closeQuietly(nodeStateProvider);
+    IOUtils.closeQuietly(stateManager);
+    triggerThread.interrupt();
+    IOUtils.closeQuietly(triggerThread);
+    IOUtils.closeQuietly(objectCache);
+    simCloudManagerPool.shutdownNow();
+  }
+}


[16/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Add OpenNLP Analysis capabilities as a module

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/package-info.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/package-info.java
new file mode 100644
index 0000000..527e24f
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Analysis components based on OpenNLP
+ */
+package org.apache.lucene.analysis.opennlp;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPChunkerOp.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPChunkerOp.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPChunkerOp.java
new file mode 100644
index 0000000..f6a5ea8
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPChunkerOp.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import java.io.IOException;
+import opennlp.tools.chunker.ChunkerME;
+import opennlp.tools.chunker.ChunkerModel;
+
+/**
+ * Supply OpenNLP Chunking tool
+ * Requires binary models from OpenNLP project on SourceForge.
+ */
+public class NLPChunkerOp {
+  private ChunkerME chunker = null;
+
+  public NLPChunkerOp(ChunkerModel chunkerModel) throws IOException {
+    chunker = new ChunkerME(chunkerModel);
+  }
+
+  public synchronized String[] getChunks(String[] words, String[] tags, double[] probs) {
+    String[] chunks = chunker.chunk(words, tags);
+    if (probs != null)
+      chunker.probs(probs);
+    return chunks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPLemmatizerOp.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPLemmatizerOp.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPLemmatizerOp.java
new file mode 100644
index 0000000..b09c63e
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPLemmatizerOp.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import opennlp.tools.lemmatizer.DictionaryLemmatizer;
+import opennlp.tools.lemmatizer.LemmatizerME;
+import opennlp.tools.lemmatizer.LemmatizerModel;
+
+/**
+ * <p>Supply OpenNLP Lemmatizer tools.</p>
+ * <p>
+ *   Both a dictionary-based lemmatizer and a MaxEnt lemmatizer are supported.
+ *   If both are configured, the dictionary-based lemmatizer is tried first,
+ *   and then the MaxEnt lemmatizer is consulted for out-of-vocabulary tokens.
+ * </p>
+ * <p>
+ *   The MaxEnt implementation requires binary models from OpenNLP project on SourceForge.
+ * </p>
+ */
+public class NLPLemmatizerOp {
+  private final DictionaryLemmatizer dictionaryLemmatizer;
+  private final LemmatizerME lemmatizerME;
+
+  public NLPLemmatizerOp(InputStream dictionary, LemmatizerModel lemmatizerModel) throws IOException {
+    assert dictionary != null || lemmatizerModel != null : "At least one parameter must be non-null";
+    dictionaryLemmatizer = dictionary == null ? null : new DictionaryLemmatizer(dictionary);
+    lemmatizerME = lemmatizerModel == null ? null : new LemmatizerME(lemmatizerModel);
+  }
+
+  public String[] lemmatize(String[] words, String[] postags) {
+    String[] lemmas = null;
+    String[] maxEntLemmas = null;
+    if (dictionaryLemmatizer != null) {
+      lemmas = dictionaryLemmatizer.lemmatize(words, postags);
+      for (int i = 0; i < lemmas.length; ++i) {
+        if (lemmas[i].equals("O")) {   // this word is not in the dictionary
+          if (lemmatizerME != null) {  // fall back to the MaxEnt lemmatizer if it's enabled
+            if (maxEntLemmas == null) {
+              maxEntLemmas = lemmatizerME.lemmatize(words, postags);
+            }
+            if ("_".equals(maxEntLemmas[i])) {
+              lemmas[i] = words[i];    // put back the original word if no lemma is found
+            } else {
+              lemmas[i] = maxEntLemmas[i];
+            }
+          } else {                     // there is no MaxEnt lemmatizer
+            lemmas[i] = words[i];      // put back the original word if no lemma is found
+          }
+        }
+      }
+    } else {                           // there is only a MaxEnt lemmatizer
+      maxEntLemmas = lemmatizerME.lemmatize(words, postags);
+      for (int i = 0 ; i < maxEntLemmas.length ; ++i) {
+        if ("_".equals(maxEntLemmas[i])) {
+          maxEntLemmas[i] = words[i];  // put back the original word if no lemma is found
+        }
+      }
+      lemmas = maxEntLemmas;
+    }
+    return lemmas;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPNERTaggerOp.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPNERTaggerOp.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPNERTaggerOp.java
new file mode 100644
index 0000000..22e617d
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPNERTaggerOp.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import opennlp.tools.namefind.NameFinderME;
+import opennlp.tools.namefind.TokenNameFinder;
+import opennlp.tools.namefind.TokenNameFinderModel;
+import opennlp.tools.util.Span;
+
+/**
+ * Supply OpenNLP Named Entity Resolution tool
+ * Requires binary models from OpenNLP project on SourceForge.
+ *
+ * Usage: from <a href="http://opennlp.apache.org/docs/1.8.3/manual/opennlp.html#tools.namefind.recognition.api"
+ *             >the OpenNLP documentation</a>:
+ *
+ * "The NameFinderME class is not thread safe, it must only be called from one thread.
+ * To use multiple threads multiple NameFinderME instances sharing the same model instance
+ * can be created. The input text should be segmented into documents, sentences and tokens.
+ * To perform entity detection an application calls the find method for every sentence in
+ * the document. After every document clearAdaptiveData must be called to clear the adaptive
+ * data in the feature generators. Not calling clearAdaptiveData can lead to a sharp drop
+ * in the detection rate after a few documents."
+ *
+ */
+public class NLPNERTaggerOp {
+  private final TokenNameFinder nameFinder;
+
+  public NLPNERTaggerOp(TokenNameFinderModel model) {
+    this.nameFinder = new NameFinderME(model);
+  }
+
+  public Span[] getNames(String[] words) {
+    Span[] names = nameFinder.find(words);
+    return names;
+  }
+
+  public synchronized void reset() {
+    nameFinder.clearAdaptiveData();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPPOSTaggerOp.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPPOSTaggerOp.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPPOSTaggerOp.java
new file mode 100644
index 0000000..447e1c0
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPPOSTaggerOp.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import java.io.IOException;
+
+import opennlp.tools.postag.POSModel;
+import opennlp.tools.postag.POSTagger;
+import opennlp.tools.postag.POSTaggerME;
+
+/**
+ * Supply OpenNLP Parts-Of-Speech Tagging tool
+ * Requires binary models from OpenNLP project on SourceForge.
+ */
+
+public class NLPPOSTaggerOp {
+  private POSTagger tagger = null;
+
+  public NLPPOSTaggerOp(POSModel model) throws IOException {
+    tagger = new POSTaggerME(model);
+  }
+
+  public synchronized String[] getPOSTags(String[] words) {
+    return tagger.tag(words);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPSentenceDetectorOp.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPSentenceDetectorOp.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPSentenceDetectorOp.java
new file mode 100644
index 0000000..21983ce
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPSentenceDetectorOp.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import java.io.IOException;
+
+import opennlp.tools.sentdetect.SentenceDetectorME;
+import opennlp.tools.sentdetect.SentenceModel;
+import opennlp.tools.util.Span;
+
+/**
+ * Supply OpenNLP Sentence Detector tool
+ * Requires binary models from OpenNLP project on SourceForge.
+ */
+public class NLPSentenceDetectorOp {
+  private final SentenceDetectorME sentenceSplitter;
+
+  public NLPSentenceDetectorOp(SentenceModel model) throws IOException {
+    sentenceSplitter  = new SentenceDetectorME(model);
+  }
+
+  public NLPSentenceDetectorOp() {
+    sentenceSplitter = null;
+  }
+
+  public synchronized Span[] splitSentences(String line) {
+    if (sentenceSplitter != null) {
+      return sentenceSplitter.sentPosDetect(line);
+    } else {
+      Span[] shorty = new Span[1];
+      shorty[0] = new Span(0, line.length());
+      return shorty;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPTokenizerOp.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPTokenizerOp.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPTokenizerOp.java
new file mode 100644
index 0000000..0aeb713
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/NLPTokenizerOp.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import opennlp.tools.tokenize.Tokenizer;
+import opennlp.tools.tokenize.TokenizerME;
+import opennlp.tools.tokenize.TokenizerModel;
+import opennlp.tools.util.Span;
+
+/**
+ * Supply OpenNLP Sentence Tokenizer tool
+ * Requires binary models from OpenNLP project on SourceForge.
+ */
+public class NLPTokenizerOp {
+  private final Tokenizer tokenizer;
+
+  public NLPTokenizerOp(TokenizerModel model) {
+    tokenizer = new TokenizerME(model);
+  }
+
+  public NLPTokenizerOp() {
+    tokenizer = null;
+  }
+
+  public synchronized Span[] getTerms(String sentence) {
+    if (tokenizer == null) {
+      Span[] span1 = new Span[1];
+      span1[0] = new Span(0, sentence.length());
+      return span1;
+    }
+    return tokenizer.tokenizePos(sentence);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/OpenNLPOpsFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/OpenNLPOpsFactory.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/OpenNLPOpsFactory.java
new file mode 100644
index 0000000..5348857
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/OpenNLPOpsFactory.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp.tools;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import opennlp.tools.chunker.ChunkerModel;
+import opennlp.tools.lemmatizer.LemmatizerModel;
+import opennlp.tools.namefind.TokenNameFinderModel;
+import opennlp.tools.postag.POSModel;
+import opennlp.tools.sentdetect.SentenceModel;
+import opennlp.tools.tokenize.TokenizerModel;
+import org.apache.lucene.analysis.util.ResourceLoader;
+
+/**
+ * Supply OpenNLP Named Entity Recognizer
+ * Cache model file objects. Assumes model files are thread-safe.
+ */
+public class OpenNLPOpsFactory {
+  private static Map<String,SentenceModel> sentenceModels = new ConcurrentHashMap<>();
+  private static ConcurrentHashMap<String,TokenizerModel> tokenizerModels = new ConcurrentHashMap<>();
+  private static ConcurrentHashMap<String,POSModel> posTaggerModels = new ConcurrentHashMap<>();
+  private static ConcurrentHashMap<String,ChunkerModel> chunkerModels = new ConcurrentHashMap<>();
+  private static Map<String,TokenNameFinderModel> nerModels = new ConcurrentHashMap<>();
+  private static Map<String,LemmatizerModel> lemmatizerModels = new ConcurrentHashMap<>();
+  private static Map<String,String> lemmaDictionaries = new ConcurrentHashMap<>();
+
+  public static NLPSentenceDetectorOp getSentenceDetector(String modelName) throws IOException {
+    if (modelName != null) {
+      SentenceModel model = sentenceModels.get(modelName);
+      return new NLPSentenceDetectorOp(model);
+    } else {
+      return new NLPSentenceDetectorOp();
+    }
+  }
+
+  public static SentenceModel getSentenceModel(String modelName, ResourceLoader loader) throws IOException {
+    SentenceModel model = sentenceModels.get(modelName);
+    if (model == null) {
+      model = new SentenceModel(loader.openResource(modelName));
+      sentenceModels.put(modelName, model);
+    }
+    return model;
+  }
+
+  public static NLPTokenizerOp getTokenizer(String modelName) throws IOException {
+    if (modelName == null) {
+      return new NLPTokenizerOp();
+    } else {
+      TokenizerModel model = tokenizerModels.get(modelName);
+      return new NLPTokenizerOp(model);
+    }
+  }
+
+  public static TokenizerModel getTokenizerModel(String modelName, ResourceLoader loader) throws IOException {
+    TokenizerModel model = tokenizerModels.get(modelName);
+    if (model == null) {
+      model = new TokenizerModel(loader.openResource(modelName));
+      tokenizerModels.put(modelName, model);
+    }
+    return model;
+  }
+
+  public static NLPPOSTaggerOp getPOSTagger(String modelName) throws IOException {
+    POSModel model = posTaggerModels.get(modelName);
+    return new NLPPOSTaggerOp(model);
+  }
+
+  public static POSModel getPOSTaggerModel(String modelName, ResourceLoader loader) throws IOException {
+    POSModel model = posTaggerModels.get(modelName);
+    if (model == null) {
+      model = new POSModel(loader.openResource(modelName));
+      posTaggerModels.put(modelName, model);
+    }
+    return model;
+  }
+
+  public static NLPChunkerOp getChunker(String modelName) throws IOException {
+    ChunkerModel model = chunkerModels.get(modelName);
+    return new NLPChunkerOp(model);
+  }
+
+  public static ChunkerModel getChunkerModel(String modelName, ResourceLoader loader) throws IOException {
+    ChunkerModel model = chunkerModels.get(modelName);
+    if (model == null) {
+      model = new ChunkerModel(loader.openResource(modelName));
+      chunkerModels.put(modelName, model);
+    }
+    return model;
+  }
+
+  public static NLPNERTaggerOp getNERTagger(String modelName) throws IOException {
+    TokenNameFinderModel model = nerModels.get(modelName);
+    return new NLPNERTaggerOp(model);
+  }
+
+  public static TokenNameFinderModel getNERTaggerModel(String modelName, ResourceLoader loader) throws IOException {
+    TokenNameFinderModel model = nerModels.get(modelName);
+    if (model == null) {
+      model = new TokenNameFinderModel(loader.openResource(modelName));
+      nerModels.put(modelName, model);
+    }
+    return model;
+  }
+
+  public static NLPLemmatizerOp getLemmatizer(String dictionaryFile, String lemmatizerModelFile) throws IOException {
+    assert dictionaryFile != null || lemmatizerModelFile != null : "At least one parameter must be non-null";
+    InputStream dictionaryInputStream = null;
+    if (dictionaryFile != null) {
+      String dictionary = lemmaDictionaries.get(dictionaryFile);
+      dictionaryInputStream = new ByteArrayInputStream(dictionary.getBytes(StandardCharsets.UTF_8));
+    }
+    LemmatizerModel lemmatizerModel = lemmatizerModelFile == null ? null : lemmatizerModels.get(lemmatizerModelFile);
+    return new NLPLemmatizerOp(dictionaryInputStream, lemmatizerModel);
+  }
+
+  public static String getLemmatizerDictionary(String dictionaryFile, ResourceLoader loader) throws IOException {
+    String dictionary = lemmaDictionaries.get(dictionaryFile);
+    if (dictionary == null) {
+      Reader reader = new InputStreamReader(loader.openResource(dictionaryFile), StandardCharsets.UTF_8);
+      StringBuilder builder = new StringBuilder();
+      char[] chars = new char[8092];
+      int numRead = 0;
+      do {
+        numRead = reader.read(chars, 0, chars.length);
+        if (numRead > 0) {
+          builder.append(chars, 0, numRead);
+        }
+      } while (numRead > 0);
+      dictionary = builder.toString();
+      lemmaDictionaries.put(dictionaryFile, dictionary);
+    }
+    return dictionary;
+  }
+
+  public static LemmatizerModel getLemmatizerModel(String modelName, ResourceLoader loader) throws IOException {
+    LemmatizerModel model = lemmatizerModels.get(modelName);
+    if (model == null) {
+      model = new LemmatizerModel(loader.openResource(modelName));
+      lemmatizerModels.put(modelName, model);
+    }
+    return model;
+  }
+
+  // keeps unit test from blowing out memory
+  public static void clearModels() {
+    sentenceModels.clear();
+    tokenizerModels.clear();
+    posTaggerModels.clear();
+    chunkerModels.clear();
+    nerModels.clear();
+    lemmaDictionaries.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/package-info.java b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/package-info.java
new file mode 100644
index 0000000..523a084
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/org/apache/lucene/analysis/opennlp/tools/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Tools to supply access to OpenNLP components.
+ */
+package org.apache.lucene.analysis.opennlp.tools;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/java/overview.html
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/java/overview.html b/lucene/analysis/opennlp/src/java/overview.html
new file mode 100644
index 0000000..bf70e95
--- /dev/null
+++ b/lucene/analysis/opennlp/src/java/overview.html
@@ -0,0 +1,61 @@
+<!--
+ 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.
+-->
+<html>
+<head>
+  <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
+  <title>
+    Apache Lucene OpenNLP integration module
+  </title>
+</head>
+<body>
+<p>
+  This module exposes functionality from
+  <a href="http://opennlp.apache.org">Apache OpenNLP</a> to Apache Lucene.
+  The Apache OpenNLP library is a machine learning based toolkit for the processing of natural language text.
+<p>
+  For an introduction to Lucene's analysis API, see the {@link org.apache.lucene.analysis} package documentation.
+<p>
+  The OpenNLP Tokenizer behavior is similar to the WhiteSpaceTokenizer but is smart about
+  inter-word punctuation. The term stream looks very much like the way you parse words and
+  punctuation while reading.  The major difference between this tokenizer and most other
+  tokenizers shipped with Lucene is that punctuation is tokenized.  This is required for
+  the following taggers to operate properly.
+<p>
+  The OpenNLP taggers annotate terms using the <code>TypeAttribute</code>.
+<ul>
+  <li><code>OpenNLPTokenizer</code> segments text into sentences or words. This Tokenizer
+    uses the OpenNLP Sentence Detector and/or Tokenizer classes.  When used together, the
+    Tokenizer receives sentences and can do a better job.</li>
+  <li><code>OpenNLPFilter</code> tags words using one or more technologies: Part-of-Speech,
+    Chunking, and Named Entity Recognition.  These tags are assigned as token types.  Note that
+    only of these operations will tag
+  </li>
+</ul>
+<p>
+  Since the <code>TypeAttribute</code> is not stored in the index, it is recommended that one
+  of these filters is used following <code>OpenNLPFilter</code> to enable search against the
+  assigned tags:
+<ul>
+  <li><code>TypeAsPayloadFilter</code> copies the <code>TypeAttribute</code> value to the
+    <code>PayloadAttribute</code></li>
+  <li><code>TypeAsSynonymFilter</code> creates a cloned token at the same position as each
+    tagged token, and copies the {{TypeAttribute}} value to the {{CharTermAttribute}}, optionally
+    with a customized prefix (so that tags effectively occupy a different namespace from token
+    text).</li>
+</ul>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
new file mode 100644
index 0000000..61a685d
--- /dev/null
+++ b/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -0,0 +1,18 @@
+#  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.
+
+org.apache.lucene.analysis.opennlp.OpenNLPChunkerFilterFactory
+org.apache.lucene.analysis.opennlp.OpenNLPLemmatizerFilterFactory
+org.apache.lucene.analysis.opennlp.OpenNLPPOSFilterFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory b/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
new file mode 100644
index 0000000..076b308
--- /dev/null
+++ b/lucene/analysis/opennlp/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
@@ -0,0 +1,16 @@
+#  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.
+
+org.apache.lucene.analysis.opennlp.OpenNLPTokenizerFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-chunker.bin
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-chunker.bin b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-chunker.bin
new file mode 100644
index 0000000..8151914
Binary files /dev/null and b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-chunker.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmas.dict
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmas.dict b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmas.dict
new file mode 100644
index 0000000..d1d486c
--- /dev/null
+++ b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmas.dict
@@ -0,0 +1,12 @@
+they	NNP	they
+sent	VBD	send
+him	PRP	he
+running	VBG	run
+in	IN	in
+the	DT	the
+evening	NN	evening
+he	PRP	he
+did	VBD	do
+not	RB	not
+come	VB	come
+back	RB	back

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmatizer.bin
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmatizer.bin b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmatizer.bin
new file mode 100644
index 0000000..e62df7e
Binary files /dev/null and b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-lemmatizer.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-ner-person.bin
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-ner-person.bin b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-ner-person.bin
new file mode 100644
index 0000000..0b40aac
Binary files /dev/null and b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-ner-person.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-pos-maxent.bin
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-pos-maxent.bin b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-pos-maxent.bin
new file mode 100644
index 0000000..b77fb46
Binary files /dev/null and b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-pos-maxent.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-sent.bin
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-sent.bin b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-sent.bin
new file mode 100644
index 0000000..4252bcb
Binary files /dev/null and b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-sent.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-tokenizer.bin
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-tokenizer.bin b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-tokenizer.bin
new file mode 100644
index 0000000..94668c0
Binary files /dev/null and b/lucene/analysis/opennlp/src/test-files/org/apache/lucene/analysis/opennlp/en-test-tokenizer.bin differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPChunkerFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPChunkerFilterFactory.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPChunkerFilterFactory.java
new file mode 100644
index 0000000..013348c
--- /dev/null
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPChunkerFilterFactory.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.custom.CustomAnalyzer;
+import org.apache.lucene.analysis.payloads.TypeAsPayloadTokenFilterFactory;
+import org.apache.lucene.analysis.util.ClasspathResourceLoader;
+
+/**
+ * Needs the OpenNLP Tokenizer because it creates full streams of punctuation.
+ * Needs the OpenNLP POS tagger for the POS tags.
+ *
+ * Tagging models are created from tiny test data in opennlp/tools/test-model-data/ and are not very accurate.
+ */
+public class TestOpenNLPChunkerFilterFactory extends BaseTokenStreamTestCase {
+
+  private static final String SENTENCES = "Sentence number 1 has 6 words. Sentence number 2, 5 words.";
+  private static final String[] SENTENCES_punc
+      = {"Sentence", "number", "1", "has", "6", "words", ".", "Sentence", "number", "2", ",", "5", "words", "."};
+  private static final int[] SENTENCES_startOffsets = {0, 9, 16, 18, 22, 24, 29, 31, 40, 47, 48, 50, 52, 57};
+  private static final int[] SENTENCES_endOffsets = {8, 15, 17, 21, 23, 29, 30, 39, 46, 48, 49, 51, 57, 58};
+  private static final String[] SENTENCES_chunks
+      = { "B-NP", "I-NP", "I-NP", "B-VP", "B-NP", "I-NP", "O", "B-NP", "I-NP", "I-NP", "O", "B-NP", "I-NP", "O" };
+
+  private static final String sentenceModelFile = "en-test-sent.bin";
+  private static final String tokenizerModelFile = "en-test-tokenizer.bin";
+  private static final String posTaggerModelFile = "en-test-pos-maxent.bin";
+  private static final String chunkerModelFile = "en-test-chunker.bin";
+
+
+  private static byte[][] toPayloads(String... strings) {
+    return Arrays.stream(strings).map(s -> s == null ? null : s.getBytes(StandardCharsets.UTF_8)).toArray(byte[][]::new);
+  }
+
+  public void testBasic() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter("opennlpChunker", "chunkerModel", chunkerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_punc, SENTENCES_startOffsets, SENTENCES_endOffsets,
+        SENTENCES_chunks, null, null, true);
+  }
+
+  public void testPayloads() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter("opennlpChunker", "chunkerModel", chunkerModelFile)
+        .addTokenFilter(TypeAsPayloadTokenFilterFactory.class)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_punc, SENTENCES_startOffsets, SENTENCES_endOffsets,
+        null, null, null, true, toPayloads(SENTENCES_chunks));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPLemmatizerFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPLemmatizerFilterFactory.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPLemmatizerFilterFactory.java
new file mode 100644
index 0000000..0491b91
--- /dev/null
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPLemmatizerFilterFactory.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.custom.CustomAnalyzer;
+import org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilterFactory;
+import org.apache.lucene.analysis.miscellaneous.RemoveDuplicatesTokenFilterFactory;
+import org.apache.lucene.analysis.util.ClasspathResourceLoader;
+
+public class TestOpenNLPLemmatizerFilterFactory extends BaseTokenStreamTestCase {
+
+  private static final String SENTENCE = "They sent him running in the evening.";
+  private static final String[] SENTENCE_dict_punc =   {"they", "send", "he",  "run",  "in", "the", "evening", "."};
+  private static final String[] SENTENCE_maxent_punc = {"they", "send", "he",  "runn", "in", "the", "evening", "."};
+  private static final String[] SENTENCE_posTags =     {"NNP",  "VBD",  "PRP", "VBG",  "IN", "DT",  "NN",      "."};
+
+  private static final String SENTENCES = "They sent him running in the evening. He did not come back.";
+  private static final String[] SENTENCES_dict_punc
+      = {"they", "send", "he",  "run",  "in", "the", "evening", ".", "he",  "do",  "not", "come", "back", "."};
+  private static final String[] SENTENCES_maxent_punc
+      = {"they", "send", "he",  "runn", "in", "the", "evening", ".", "he",  "do",  "not", "come", "back", "."};
+  private static final String[] SENTENCES_posTags
+      = {"NNP",  "VBD",  "PRP", "VBG",  "IN", "DT",  "NN",      ".", "PRP", "VBD", "RB",  "VB",   "RB",   "."};
+
+  private static final String SENTENCE_both = "Konstantin Kalashnitsov constantly caliphed.";
+  private static final String[] SENTENCE_both_punc
+      = {"konstantin", "kalashnitsov", "constantly", "caliph", "."};
+  private static final String[] SENTENCE_both_posTags
+      = {"IN",         "JJ",          "NN",          "VBN",    "."};
+
+  private static final String SENTENCES_both = "Konstantin Kalashnitsov constantly caliphed. Coreena could care, completely.";
+  private static final String[] SENTENCES_both_punc
+      = {"konstantin", "kalashnitsov", "constantly", "caliph", ".", "coreena", "could", "care", ",", "completely", "."};
+  private static final String[] SENTENCES_both_posTags
+      = {"IN",         "JJ",           "NN",          "VBN",    ".", "NNP",     "VBN",   "NN",   ",", "NN",         "."};
+
+  private static final String[] SENTENCES_dict_keep_orig_punc
+      = {"They", "they", "sent", "send", "him", "he", "running", "run",  "in", "the", "evening", ".", "He", "he",   "did", "do", "not", "come", "back", "."};
+  private static final String[] SENTENCES_max_ent_keep_orig_punc
+      = {"They", "they", "sent", "send", "him", "he", "running", "runn", "in", "the", "evening", ".", "He", "he",   "did", "do", "not", "come", "back", "."};
+  private static final String[] SENTENCES_keep_orig_posTags
+      = {"NNP",  "NNP",  "VBD",  "VBD",  "PRP", "PRP", "VBG",    "VBG",  "IN", "DT",  "NN",      ".", "PRP", "PRP", "VBD", "VBD", "RB",  "VB",  "RB",   "."};
+
+  private static final String[] SENTENCES_both_keep_orig_punc
+      = {"Konstantin", "konstantin", "Kalashnitsov", "kalashnitsov", "constantly", "caliphed", "caliph", ".", "Coreena", "coreena", "could", "care", ",", "completely", "."};
+  private static final String[] SENTENCES_both_keep_orig_posTags
+      = {"IN",         "IN",         "JJ",           "JJ",           "NN",         "VBN",      "VBN",    ".", "NNP",     "NNP",     "VBN",   "NN",   ",", "NN",         "."};
+
+
+  private static final String tokenizerModelFile = "en-test-tokenizer.bin";
+  private static final String sentenceModelFile = "en-test-sent.bin";
+  private static final String posTaggerModelFile = "en-test-pos-maxent.bin";
+  private static final String lemmatizerModelFile = "en-test-lemmatizer.bin";
+  private static final String lemmatizerDictFile = "en-test-lemmas.dict";
+
+
+  public void test1SentenceDictionaryOnly() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", "en-test-pos-maxent.bin")
+        .addTokenFilter("opennlplemmatizer", "dictionary", "en-test-lemmas.dict")
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCE, SENTENCE_dict_punc, null, null,
+        SENTENCE_posTags, null, null, true);
+  }
+
+  public void test2SentencesDictionaryOnly() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter("opennlplemmatizer", "dictionary", lemmatizerDictFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_dict_punc, null, null,
+        SENTENCES_posTags, null, null, true);
+  }
+
+  public void test1SentenceMaxEntOnly() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter("opennlplemmatizer", "lemmatizerModel", lemmatizerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCE, SENTENCE_maxent_punc, null, null,
+        SENTENCE_posTags, null, null, true);
+  }
+
+  public void test2SentencesMaxEntOnly() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter("OpenNLPLemmatizer", "lemmatizerModel", lemmatizerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_maxent_punc, null, null,
+        SENTENCES_posTags, null, null, true);
+  }
+
+  public void test1SentenceDictionaryAndMaxEnt() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", "en-test-pos-maxent.bin")
+        .addTokenFilter("opennlplemmatizer", "dictionary", "en-test-lemmas.dict", "lemmatizerModel", lemmatizerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCE_both, SENTENCE_both_punc, null, null,
+        SENTENCE_both_posTags, null, null, true);
+  }
+
+  public void test2SentencesDictionaryAndMaxEnt() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter("opennlplemmatizer", "dictionary", lemmatizerDictFile, "lemmatizerModel", lemmatizerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES_both, SENTENCES_both_punc, null, null,
+        SENTENCES_both_posTags, null, null, true);
+  }
+
+  public void testKeywordAttributeAwarenessDictionaryOnly() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter(KeywordRepeatFilterFactory.class)
+        .addTokenFilter("opennlplemmatizer", "dictionary", lemmatizerDictFile)
+        .addTokenFilter(RemoveDuplicatesTokenFilterFactory.class)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_dict_keep_orig_punc, null, null,
+        SENTENCES_keep_orig_posTags, null, null, true);
+  }
+
+  public void testKeywordAttributeAwarenessMaxEntOnly() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter(KeywordRepeatFilterFactory.class)
+        .addTokenFilter("opennlplemmatizer", "lemmatizerModel", lemmatizerModelFile)
+        .addTokenFilter(RemoveDuplicatesTokenFilterFactory.class)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_max_ent_keep_orig_punc, null, null,
+        SENTENCES_keep_orig_posTags, null, null, true);
+  }
+
+  public void testKeywordAttributeAwarenessDictionaryAndMaxEnt() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter(KeywordRepeatFilterFactory.class)
+        .addTokenFilter("opennlplemmatizer", "dictionary", lemmatizerDictFile, "lemmatizerModel", lemmatizerModelFile)
+        .addTokenFilter(RemoveDuplicatesTokenFilterFactory.class)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES_both, SENTENCES_both_keep_orig_punc, null, null,
+        SENTENCES_both_keep_orig_posTags, null, null, true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
new file mode 100644
index 0000000..10372d0
--- /dev/null
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPPOSFilterFactory.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.custom.CustomAnalyzer;
+import org.apache.lucene.analysis.payloads.TypeAsPayloadTokenFilterFactory;
+import org.apache.lucene.analysis.util.ClasspathResourceLoader;
+
+/**
+ * Needs the OpenNLP Tokenizer because it creates full streams of punctuation.
+ * The POS model is based on this tokenization.
+ *
+ * Tagging models are created from tiny test data in opennlp/tools/test-model-data/ and are not very accurate.
+ */
+public class TestOpenNLPPOSFilterFactory extends BaseTokenStreamTestCase {
+
+  private static final String SENTENCES = "Sentence number 1 has 6 words. Sentence number 2, 5 words.";
+  private static final String[] SENTENCES_punc
+      = {"Sentence", "number", "1", "has", "6", "words", ".", "Sentence", "number", "2", ",", "5", "words", "."};
+  private static final int[] SENTENCES_startOffsets = {0, 9, 16, 18, 22, 24, 29, 31, 40, 47, 48, 50, 52, 57};
+  private static final int[] SENTENCES_endOffsets = {8, 15, 17, 21, 23, 29, 30, 39, 46, 48, 49, 51, 57, 58};
+  private static final String[] SENTENCES_posTags
+      = {"NN", "NN", "CD", "VBZ", "CD", "NNS", ".", "NN", "NN", "CD", ",", "CD", "NNS", "."};
+  private static final String NAMES2 = "Royal Flash is a tale about Harry Flashman.";
+  private static final String[] NAMES2_punc = {"Royal", "Flash", "is", "a", "tale", "about", "Harry", "Flashman", "."};
+  private static final String[] NAMES2_OUT = { "word", "word", "word", "word", "word", "word", "word", "person", "word" };
+
+  private static final String NO_BREAK = "No period";
+  private static final String[] NO_BREAK_terms = {"No", "period"};
+  private static final int[] NO_BREAK_startOffsets = {0, 3};
+  private static final int[] NO_BREAK_endOffsets = {2, 9};
+
+  private static final String sentenceModelFile = "en-test-sent.bin";
+  private static final String tokenizerModelFile = "en-test-tokenizer.bin";
+  private static final String posTaggerModelFile = "en-test-pos-maxent.bin";
+
+
+  private static byte[][] toPayloads(String... strings) {
+    return Arrays.stream(strings).map(s -> s == null ? null : s.getBytes(StandardCharsets.UTF_8)).toArray(byte[][]::new);
+  }
+
+  public void testBasic() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_punc, SENTENCES_startOffsets, SENTENCES_endOffsets);
+  }
+
+  public void testPOS() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_punc, SENTENCES_startOffsets, SENTENCES_endOffsets,
+        SENTENCES_posTags, null, null, true);
+
+    analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .addTokenFilter(TypeAsPayloadTokenFilterFactory.class)
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_punc, SENTENCES_startOffsets, SENTENCES_endOffsets,
+        null, null, null, true, toPayloads(SENTENCES_posTags));
+  }
+
+  public void testNoBreak() throws Exception {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "tokenizerModel", tokenizerModelFile, "sentenceModel", sentenceModelFile)
+        .addTokenFilter("opennlpPOS", "posTaggerModel", posTaggerModelFile)
+        .build();
+    assertAnalyzesTo(analyzer, NO_BREAK, NO_BREAK_terms, NO_BREAK_startOffsets, NO_BREAK_endOffsets,
+        null, null, null, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPSentenceBreakIterator.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPSentenceBreakIterator.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPSentenceBreakIterator.java
new file mode 100644
index 0000000..4ee6570
--- /dev/null
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPSentenceBreakIterator.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.text.BreakIterator;
+import java.text.CharacterIterator;
+
+import org.apache.lucene.analysis.opennlp.tools.NLPSentenceDetectorOp;
+import org.apache.lucene.analysis.opennlp.tools.OpenNLPOpsFactory;
+import org.apache.lucene.analysis.util.CharArrayIterator;
+import org.apache.lucene.analysis.util.ClasspathResourceLoader;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.BeforeClass;
+
+public class TestOpenNLPSentenceBreakIterator extends LuceneTestCase {
+
+  private static final String TEXT
+      //                                                                                                     111
+      //           111111111122222222223333333333444444444455555555556666666666777777777788888888889999999999000
+      // 0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012
+      = "Sentence number 1 has 6 words. Sentence number 2, 5 words. And finally, sentence number 3 has 8 words.";
+  private static final String[] SENTENCES = new String[] {
+    "Sentence number 1 has 6 words. ", "Sentence number 2, 5 words. ", "And finally, sentence number 3 has 8 words." };
+  private static final String PADDING = " Word. Word. ";
+  private static final String sentenceModelFile = "en-test-sent.bin";
+
+
+  @BeforeClass
+  public static void populateCache() throws IOException {
+    OpenNLPOpsFactory.getSentenceModel
+        (sentenceModelFile, new ClasspathResourceLoader(TestOpenNLPSentenceBreakIterator.class));
+  }
+
+  public void testThreeSentences() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText(TEXT); // String is converted to StringCharacterIterator
+    do3SentenceTest(bi);
+
+    bi.setText(getCharArrayIterator(TEXT));
+    do3SentenceTest(bi);
+  }
+
+  private CharacterIterator getCharArrayIterator(String text) {
+    return getCharArrayIterator(text, 0, text.length());
+  }
+
+  private CharacterIterator getCharArrayIterator(String text, int start, int length) {
+    CharArrayIterator charArrayIterator = new CharArrayIterator() {
+      // Lie about all surrogates to the sentence tokenizer,
+      // instead we treat them all as SContinue so we won't break around them.
+      @Override
+      protected char jreBugWorkaround(char ch) {
+        return ch >= 0xD800 && ch <= 0xDFFF ? 0x002C : ch;
+      }
+    };
+    charArrayIterator.setText(text.toCharArray(), start, length);
+    return charArrayIterator;
+  }
+
+  private void do3SentenceTest(BreakIterator bi) {
+    assertEquals(0, bi.current());
+    assertEquals(0, bi.first());
+    assertEquals(SENTENCES[0], TEXT.substring(bi.current(), bi.next()));
+    assertEquals(SENTENCES[1], TEXT.substring(bi.current(), bi.next()));
+    int current = bi.current();
+    assertEquals(bi.getText().getEndIndex(), bi.next());
+    int next = bi.current();
+    assertEquals(SENTENCES[2], TEXT.substring(current, next));
+    assertEquals(BreakIterator.DONE, bi.next());
+
+    assertEquals(TEXT.length(), bi.last());
+    int end = bi.current();
+    assertEquals(SENTENCES[2], TEXT.substring(bi.previous(), end));
+    end = bi.current();
+    assertEquals(SENTENCES[1], TEXT.substring(bi.previous(), end));
+    end = bi.current();
+    assertEquals(SENTENCES[0], TEXT.substring(bi.previous(), end));
+    assertEquals(BreakIterator.DONE, bi.previous());
+    assertEquals(0, bi.current());
+
+    assertEquals(59, bi.following(39));
+    assertEquals(59, bi.following(31));
+    assertEquals(31, bi.following(30));
+
+    assertEquals(0, bi.preceding(57));
+    assertEquals(0, bi.preceding(58));
+    assertEquals(31, bi.preceding(59));
+
+    assertEquals(0, bi.first());
+    assertEquals(59, bi.next(2));
+    assertEquals(0, bi.next(-2));
+  }
+
+  public void testSingleSentence() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText(getCharArrayIterator(SENTENCES[0]));
+    test1Sentence(bi, SENTENCES[0]);
+  }
+
+  private void test1Sentence(BreakIterator bi, String text) {
+    int start = bi.getText().getBeginIndex();
+    assertEquals(start, bi.first());
+    int current = bi.current();
+    assertEquals(bi.getText().getEndIndex(), bi.next());
+    int end = bi.current() - start;
+    assertEquals(text, text.substring(current - start, end - start));
+
+    assertEquals(text.length(), bi.last() - start);
+    end = bi.current();
+    bi.previous();
+    assertEquals(BreakIterator.DONE, bi.previous());
+    int previous = bi.current();
+    assertEquals(text, text.substring(previous - start, end - start));
+    assertEquals(start, bi.current());
+
+    assertEquals(BreakIterator.DONE, bi.following(bi.last() / 2 + start));
+
+    assertEquals(BreakIterator.DONE, bi.preceding(bi.last() / 2 + start));
+
+    assertEquals(start, bi.first());
+    assertEquals(BreakIterator.DONE, bi.next(13));
+    assertEquals(BreakIterator.DONE, bi.next(-8));
+  }
+
+  public void testSliceEnd() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText(getCharArrayIterator(SENTENCES[0] + PADDING, 0, SENTENCES[0].length()));
+
+    test1Sentence(bi, SENTENCES[0]);
+  }
+
+  public void testSliceStart() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText(getCharArrayIterator(PADDING + SENTENCES[0], PADDING.length(), SENTENCES[0].length()));
+    test1Sentence(bi, SENTENCES[0]);
+  }
+
+  public void testSliceMiddle() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText(getCharArrayIterator(PADDING + SENTENCES[0] + PADDING, PADDING.length(), SENTENCES[0].length()));
+
+    test1Sentence(bi, SENTENCES[0]);
+  }
+
+  /** the current position must be ignored, initial position is always first() */
+  public void testFirstPosition() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText(getCharArrayIterator(SENTENCES[0]));
+    assertEquals(SENTENCES[0].length(), bi.last()); // side-effect: set current position to last()
+    test1Sentence(bi, SENTENCES[0]);
+  }
+
+  public void testWhitespaceOnly() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText("   \n \n\n\r\n\t  \n");
+    test0Sentences(bi);
+  }
+
+  public void testEmptyString() throws Exception {
+    NLPSentenceDetectorOp sentenceDetectorOp = OpenNLPOpsFactory.getSentenceDetector(sentenceModelFile);
+    BreakIterator bi = new OpenNLPSentenceBreakIterator(sentenceDetectorOp);
+    bi.setText("");
+    test0Sentences(bi);
+  }
+
+  private void test0Sentences(BreakIterator bi) {
+    assertEquals(0, bi.current());
+    assertEquals(0, bi.first());
+    assertEquals(BreakIterator.DONE, bi.next());
+    assertEquals(0, bi.last());
+    assertEquals(BreakIterator.DONE, bi.previous());
+    assertEquals(BreakIterator.DONE, bi.following(0));
+    assertEquals(BreakIterator.DONE, bi.preceding(0));
+    assertEquals(0, bi.first());
+    assertEquals(BreakIterator.DONE, bi.next(13));
+    assertEquals(BreakIterator.DONE, bi.next(-8));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
new file mode 100644
index 0000000..db2bbb2
--- /dev/null
+++ b/lucene/analysis/opennlp/src/test/org/apache/lucene/analysis/opennlp/TestOpenNLPTokenizerFactory.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.opennlp;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.custom.CustomAnalyzer;
+import org.apache.lucene.analysis.util.ClasspathResourceLoader;
+import org.junit.Test;
+
+/**
+ * Tests the Tokenizer as well- the Tokenizer needs the OpenNLP model files,
+ * which this can load from src/test-files/opennlp/solr/conf
+ *
+ */
+public class TestOpenNLPTokenizerFactory extends BaseTokenStreamTestCase {
+
+  static private String SENTENCES = "Sentence number 1 has 6 words. Sentence number 2, 5 words.";
+  static private String[] SENTENCES_split = {"Sentence number 1 has 6 words. ", "Sentence number 2, 5 words."};
+  static private String[] SENTENCES_punc = {"Sentence", "number", "1", "has", "6", "words", ".", "Sentence", "number", "2", ",", "5", "words", "."};
+  static private int[] SENTENCES_startOffsets = {0, 9, 16, 18, 22, 24, 29, 31, 40, 47, 48, 50, 52, 57};
+  static private int[] SENTENCES_endOffsets = {8, 15, 17, 21, 23, 29, 30, 39, 46, 48, 49, 51, 57, 58};
+
+  static private String SENTENCE1 = "Sentence number 1 has 6 words.";
+  static private String[] SENTENCE1_punc = {"Sentence", "number", "1", "has", "6", "words", "."};
+
+  @Test
+  public void testTokenizer() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+        .withTokenizer("opennlp", "sentenceModel", "en-test-sent.bin", "tokenizerModel", "en-test-tokenizer.bin")
+        .build();
+    assertAnalyzesTo(analyzer, SENTENCES, SENTENCES_punc, SENTENCES_startOffsets, SENTENCES_endOffsets);
+    assertAnalyzesTo(analyzer, SENTENCE1, SENTENCE1_punc);
+  }
+
+  @Test
+  public void testTokenizerNoSentenceDetector() throws IOException {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+          .withTokenizer("opennlp", "tokenizerModel", "en-test-tokenizer.bin")
+          .build();
+    });
+    assertTrue(expected.getMessage().contains("Configuration Error: missing parameter 'sentenceModel'"));
+  }
+
+  @Test
+  public void testTokenizerNoTokenizer() throws IOException {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      CustomAnalyzer analyzer = CustomAnalyzer.builder(new ClasspathResourceLoader(getClass()))
+          .withTokenizer("opennlp", "sentenceModel", "en-test-sent.bin")
+          .build();
+    });
+    assertTrue(expected.getMessage().contains("Configuration Error: missing parameter 'tokenizerModel'"));
+  }
+
+  // test analyzer caching the tokenizer
+  @Test
+  public void testClose() throws IOException {
+    Map<String,String> args = new HashMap<String,String>() {{ put("sentenceModel", "en-test-sent.bin");
+                                                              put("tokenizerModel", "en-test-tokenizer.bin"); }};
+    OpenNLPTokenizerFactory factory = new OpenNLPTokenizerFactory(args);
+    factory.inform(new ClasspathResourceLoader(getClass()));
+
+    Tokenizer ts = factory.create(newAttributeFactory());
+    ts.setReader(new StringReader(SENTENCES));
+
+    ts.reset();
+    ts.close();
+    ts.reset();
+    ts.setReader(new StringReader(SENTENCES));
+    assertTokenStreamContents(ts, SENTENCES_punc);
+    ts.close();
+    ts.reset();
+    ts.setReader(new StringReader(SENTENCES));
+    assertTokenStreamContents(ts, SENTENCES_punc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/README.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/README.txt b/lucene/analysis/opennlp/src/tools/test-model-data/README.txt
new file mode 100644
index 0000000..3ac0aa3
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/README.txt
@@ -0,0 +1,6 @@
+Use small training data to create small models for unit tests.
+Training data derived from Reuters corpus in very unscientific way.
+Tagging done with CCG Urbana-Champaign online demos:
+	http://cogcomp.cs.illinois.edu/page/demos
+
+Run 'ant train-test-models' to generate models from training data here.


[50/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8105: ReqOptSumScorer to implement setMinCompetitiveScore.

Posted by da...@apache.org.
LUCENE-8105: ReqOptSumScorer to implement setMinCompetitiveScore.


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

Branch: refs/heads/jira/solr-11702
Commit: 01023a95c8a05ebd9667457db72fa12031e06086
Parents: e82e029
Author: Adrien Grand <jp...@gmail.com>
Authored: Tue Dec 26 14:14:56 2017 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Dec 26 14:14:56 2017 +0100

----------------------------------------------------------------------
 .../apache/lucene/search/ReqOptSumScorer.java   | 159 ++++++++++++++--
 .../lucene/search/TestReqOptSumScorer.java      | 190 +++++++++++++++++++
 2 files changed, 333 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/01023a95/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
index 35de51a..6b2ace8 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
@@ -27,9 +27,14 @@ class ReqOptSumScorer extends Scorer {
   /** The scorers passed from the constructor.
    * These are set to null as soon as their next() or skipTo() returns false.
    */
-  protected final Scorer reqScorer;
-  protected final Scorer optScorer;
-  protected final DocIdSetIterator optIterator;
+  private final Scorer reqScorer;
+  private final Scorer optScorer;
+  private final float reqMaxScore;
+  private final DocIdSetIterator optApproximation;
+  private final TwoPhaseIterator optTwoPhase;
+  private boolean optIsRequired;
+  private final DocIdSetIterator approximation;
+  private final TwoPhaseIterator twoPhase;
 
   /** Construct a <code>ReqOptScorer</code>.
    * @param reqScorer The required scorer. This must match.
@@ -44,17 +49,136 @@ class ReqOptSumScorer extends Scorer {
     assert optScorer != null;
     this.reqScorer = reqScorer;
     this.optScorer = optScorer;
-    this.optIterator = optScorer.iterator();
+
+    this.reqMaxScore = reqScorer.maxScore();
+
+    final TwoPhaseIterator reqTwoPhase = reqScorer.twoPhaseIterator();
+    this.optTwoPhase = optScorer.twoPhaseIterator();
+    final DocIdSetIterator reqApproximation;
+    if (reqTwoPhase == null) {
+      reqApproximation = reqScorer.iterator();
+    } else {
+      reqApproximation= reqTwoPhase.approximation();
+    }
+    if (optTwoPhase == null) {
+      optApproximation = optScorer.iterator();
+    } else {
+      optApproximation= optTwoPhase.approximation();
+    }
+
+    approximation = new DocIdSetIterator() {
+
+      private int nextCommonDoc(int reqDoc) throws IOException {
+        int optDoc = optApproximation.docID();
+        if (optDoc > reqDoc) {
+          reqDoc = reqApproximation.advance(optDoc);
+        }
+
+        while (true) { // invariant: reqDoc >= optDoc
+          if (reqDoc == optDoc) {
+            return reqDoc;
+          }
+
+          optDoc = optApproximation.advance(reqDoc);
+          if (optDoc == reqDoc) {
+            return reqDoc;
+          }
+          reqDoc = reqApproximation.advance(optDoc);
+        }
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        int doc = reqApproximation.nextDoc();
+        if (optIsRequired) {
+          doc = nextCommonDoc(doc);
+        }
+        return doc;
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        int doc = reqApproximation.advance(target);
+        if (optIsRequired) {
+          doc = nextCommonDoc(doc);
+        }
+        return doc;
+      }
+
+      @Override
+      public int docID() {
+        return reqApproximation.docID();
+      }
+
+      @Override
+      public long cost() {
+        return reqApproximation.cost();
+      }
+
+    };
+
+    if (reqTwoPhase == null && optTwoPhase == null) {
+      this.twoPhase = null;
+    } else {
+      this.twoPhase = new TwoPhaseIterator(approximation) {
+
+        @Override
+        public boolean matches() throws IOException {
+          if (reqTwoPhase != null && reqTwoPhase.matches() == false) {
+            return false;
+          }
+          if (optTwoPhase != null) {
+            if (optIsRequired) {
+              // The below condition is rare and can only happen if we transitioned to optIsRequired=true
+              // after the opt approximation was advanced and before it was confirmed.
+              if (reqScorer.docID() != optApproximation.docID()) {
+                if (optApproximation.docID() < reqScorer.docID()) {
+                  optApproximation.advance(reqScorer.docID());
+                }
+                if (reqScorer.docID() != optApproximation.docID()) {
+                  return false;
+                }
+              }
+              if (optTwoPhase.matches() == false) {
+                // Advance the iterator to make it clear it doesn't match the current doc id
+                optApproximation.nextDoc();
+                return false;
+              }
+            } else if (optApproximation.docID() == reqScorer.docID() && optTwoPhase.matches() == false) {
+              // Advance the iterator to make it clear it doesn't match the current doc id
+              optApproximation.nextDoc();
+            }
+          }
+          return true;
+        }
+
+        @Override
+        public float matchCost() {
+          float matchCost = 1;
+          if (reqTwoPhase != null) {
+            matchCost += reqTwoPhase.matchCost();
+          }
+          if (optTwoPhase != null) {
+            matchCost += optTwoPhase.matchCost();
+          }
+          return matchCost;
+        }
+      };
+    }
   }
 
   @Override
   public TwoPhaseIterator twoPhaseIterator() {
-    return reqScorer.twoPhaseIterator();
+    return twoPhase;
   }
 
   @Override
   public DocIdSetIterator iterator() {
-    return reqScorer.iterator();
+    if (twoPhase == null) {
+      return approximation;
+    } else {
+      return TwoPhaseIterator.asDocIdSetIterator(twoPhase);
+    }
   }
 
   @Override
@@ -62,26 +186,23 @@ class ReqOptSumScorer extends Scorer {
     return reqScorer.docID();
   }
 
-  /** Returns the score of the current document matching the query.
-   * Initially invalid, until the {@link #iterator()} is advanced the first time.
-   * @return The score of the required scorer, eventually increased by the score
-   * of the optional scorer when it also matches the current document.
-   */
   @Override
   public float score() throws IOException {
     // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     int curDoc = reqScorer.docID();
     float score = reqScorer.score();
 
-    int optScorerDoc = optIterator.docID();
+    int optScorerDoc = optApproximation.docID();
     if (optScorerDoc < curDoc) {
-      optScorerDoc = optIterator.advance(curDoc);
+      optScorerDoc = optApproximation.advance(curDoc);
+      if (optTwoPhase != null && optScorerDoc == curDoc && optTwoPhase.matches() == false) {
+        optScorerDoc = optApproximation.nextDoc();
+      }
     }
-    
     if (optScorerDoc == curDoc) {
       score += optScorer.score();
     }
-    
+
     return score;
   }
 
@@ -91,6 +212,13 @@ class ReqOptSumScorer extends Scorer {
   }
 
   @Override
+  public void setMinCompetitiveScore(float minScore) {
+    if (optIsRequired == false && minScore > reqMaxScore) {
+      optIsRequired = true;
+    }
+  }
+
+  @Override
   public Collection<ChildScorer> getChildren() {
     ArrayList<ChildScorer> children = new ArrayList<>(2);
     children.add(new ChildScorer(reqScorer, "MUST"));
@@ -99,4 +227,3 @@ class ReqOptSumScorer extends Scorer {
   }
 
 }
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/01023a95/lucene/core/src/test/org/apache/lucene/search/TestReqOptSumScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestReqOptSumScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestReqOptSumScorer.java
new file mode 100644
index 0000000..d241e72
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestReqOptSumScorer.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestReqOptSumScorer extends LuceneTestCase {
+
+  public void testBasics() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setMergePolicy(
+        // retain doc id order
+        newLogMergePolicy(random().nextBoolean())));
+    Document doc = new Document();
+    doc.add(new StringField("f", "foo", Store.NO));
+    w.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("f", "foo", Store.NO));
+    doc.add(new StringField("f", "bar", Store.NO));
+    w.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("f", "foo", Store.NO));
+    w.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("f", "bar", Store.NO));
+    w.addDocument(doc);
+    doc = new Document();
+    doc.add(new StringField("f", "foo", Store.NO));
+    doc.add(new StringField("f", "bar", Store.NO));
+    w.addDocument(doc);
+    w.forceMerge(1);
+
+    IndexReader reader = w.getReader();
+    w.close();
+    IndexSearcher searcher = newSearcher(reader);
+    Query query = new BooleanQuery.Builder()
+        .add(new ConstantScoreQuery(new TermQuery(new Term("f", "foo"))), Occur.MUST)
+        .add(new ConstantScoreQuery(new TermQuery(new Term("f", "bar"))), Occur.SHOULD)
+        .build();
+    Weight weight = searcher.createNormalizedWeight(query, ScoreMode.TOP_SCORES);
+    LeafReaderContext context = searcher.getIndexReader().leaves().get(0);
+
+    Scorer scorer = weight.scorer(context);
+    assertEquals(0, scorer.iterator().nextDoc());
+    assertEquals(1, scorer.iterator().nextDoc());
+    assertEquals(2, scorer.iterator().nextDoc());
+    assertEquals(4, scorer.iterator().nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+
+    scorer = weight.scorer(context);
+    scorer.setMinCompetitiveScore(Math.nextDown(1f));
+    assertEquals(0, scorer.iterator().nextDoc());
+    assertEquals(1, scorer.iterator().nextDoc());
+    assertEquals(2, scorer.iterator().nextDoc());
+    assertEquals(4, scorer.iterator().nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+
+    scorer = weight.scorer(context);
+    scorer.setMinCompetitiveScore(Math.nextUp(1f));
+    assertEquals(1, scorer.iterator().nextDoc());
+    assertEquals(4, scorer.iterator().nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+
+    scorer = weight.scorer(context);
+    assertEquals(0, scorer.iterator().nextDoc());
+    scorer.setMinCompetitiveScore(Math.nextUp(1f));
+    assertEquals(1, scorer.iterator().nextDoc());
+    assertEquals(4, scorer.iterator().nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testRandomFrequentOpt() throws IOException {
+    doTestRandom(0.5);
+  }
+
+  public void testRandomRareOpt() throws IOException {
+    doTestRandom(0.05);
+  }
+
+  private void doTestRandom(double optFreq) throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
+    int numDocs = atLeast(1000);
+    for (int i = 0; i < numDocs; ++i) {
+      int numAs = random().nextBoolean() ? 0 : 1 + random().nextInt(5);
+      int numBs = random().nextDouble() < optFreq ? 0 : 1 + random().nextInt(5);
+      Document doc = new Document();
+      for (int j = 0; j < numAs; ++j) {
+        doc.add(new StringField("f", "A", Store.NO));
+      }
+      for (int j = 0; j < numBs; ++j) {
+        doc.add(new StringField("f", "B", Store.NO));
+      }
+      if (random().nextBoolean()) {
+        doc.add(new StringField("f", "C", Store.NO));
+      }
+      w.addDocument(doc);
+    }
+    IndexReader r = w.getReader();
+    w.close();
+    IndexSearcher searcher = newSearcher(r);
+
+    Query mustTerm = new TermQuery(new Term("f", "A"));
+    Query shouldTerm = new TermQuery(new Term("f", "B"));
+    Query query = new BooleanQuery.Builder()
+        .add(mustTerm, Occur.MUST)
+        .add(shouldTerm, Occur.SHOULD)
+        .build();
+
+    TopScoreDocCollector coll = TopScoreDocCollector.create(10, null, true);
+    searcher.search(query, coll);
+    ScoreDoc[] expected = coll.topDocs().scoreDocs;
+
+    // Also test a filtered query, since it does not compute the score on all
+    // matches.
+    query = new BooleanQuery.Builder()
+        .add(query, Occur.MUST)
+        .add(new TermQuery(new Term("f", "C")), Occur.FILTER)
+        .build();
+
+    coll = TopScoreDocCollector.create(10, null, true);
+    searcher.search(query, coll);
+    ScoreDoc[] expectedFiltered = coll.topDocs().scoreDocs;
+
+    for (int i = 0; i < 4; ++i) {
+      Query must = mustTerm;
+      if (i % 2 == 1) {
+        must = new RandomApproximationQuery(must, random());
+      }
+      Query should = shouldTerm;
+      if (i >= 2) {
+        should = new RandomApproximationQuery(should, random());
+      }
+    
+      query = new BooleanQuery.Builder()
+          .add(must, Occur.MUST)
+          .add(should, Occur.SHOULD)
+          .build();
+
+      coll = TopScoreDocCollector.create(10, null, false);
+      searcher.search(query, coll);
+      ScoreDoc[] actual = coll.topDocs().scoreDocs;
+
+      CheckHits.checkEqual(query, expected, actual);
+
+      query = new BooleanQuery.Builder()
+          .add(query, Occur.MUST)
+          .add(new RandomApproximationQuery(new TermQuery(new Term("f", "C")), random()), Occur.FILTER)
+          .build();
+
+      coll = TopScoreDocCollector.create(10, null, false);
+      searcher.search(query, coll);
+      ScoreDoc[] actualFiltered = coll.topDocs().scoreDocs;
+
+      CheckHits.checkEqual(query, expectedFiltered, actualFiltered);
+    }
+
+    r.close();
+    dir.close();
+  }
+
+}


[22/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11757: Tests: SolrException.ignoreException use CopyOnWrite

Posted by da...@apache.org.
SOLR-11757: Tests: SolrException.ignoreException use CopyOnWrite


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

Branch: refs/heads/jira/solr-11702
Commit: e441a99009a557f82ea17ee9f9c3e9b89c75cee6
Parents: ca5e2a3
Author: David Smiley <ds...@apache.org>
Authored: Fri Dec 15 17:19:37 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Fri Dec 15 17:19:37 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 ++++
 .../org/apache/solr/common/SolrException.java   |  8 +++++--
 .../java/org/apache/solr/SolrTestCaseJ4.java    | 25 +++++++++++---------
 3 files changed, 24 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e441a990/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e60262d..fabe30a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -68,8 +68,12 @@ Optimizations
   
 Other Changes
 ----------------------
+
 * SOLR-11575: Improve ref-guide solrj snippets via mock 'print()' method (Jason Gerlowski via hossman)
 
+* SOLR-11757: In tests, fix race condition on SolrException.ignoreException.
+  Also ensure we register "ignore_exception" in @BeforeClass (previously only @AfterClass)  (David Smiley)
+
 ==================  7.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e441a990/solr/solrj/src/java/org/apache/solr/common/SolrException.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrException.java b/solr/solrj/src/java/org/apache/solr/common/SolrException.java
index 22782b0..f73f5dd 100644
--- a/solr/solrj/src/java/org/apache/solr/common/SolrException.java
+++ b/solr/solrj/src/java/org/apache/solr/common/SolrException.java
@@ -191,16 +191,20 @@ public class SolrException extends RuntimeException {
   }
 
 
-  /** For test code - do not log exceptions that match any of the regular expressions in ignorePatterns */
+  /**
+   * For test code - do not log exceptions that match any of these regular expressions.
+   * A {@link java.util.concurrent.CopyOnWriteArraySet is recommended}.
+   */
   public static Set<String> ignorePatterns;
 
   /** Returns null if this exception does not match any ignore patterns, or a message string to use if it does. */
   public static String doIgnore(Throwable t, String m) {
+    Set<String> ignorePatterns = SolrException.ignorePatterns; // guard against races, albeit unlikely
     if (ignorePatterns == null || m == null) return null;
     if (t != null && t instanceof AssertionError) return null;
 
     for (String regex : ignorePatterns) {
-      Pattern pattern = Pattern.compile(regex);
+      Pattern pattern = Pattern.compile(regex); // TODO why do we compile late; why not up-front?
       Matcher matcher = pattern.matcher(m);
       
       if (matcher.find()) return "Ignoring exception matching " + regex;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e441a990/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 449af4e..c089b24 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -49,15 +49,15 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.Map.Entry;
+import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
 
 import com.carrotsearch.randomizedtesting.RandomizedContext;
@@ -65,7 +65,6 @@ import com.carrotsearch.randomizedtesting.RandomizedTest;
 import com.carrotsearch.randomizedtesting.TraceFormatting;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
-
 import org.apache.commons.io.FileUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -145,11 +144,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
+import static java.util.Objects.requireNonNull;
 import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
-import static java.util.Objects.requireNonNull;
-
 /**
  * A junit4 Solr test harness that extends LuceneTestCaseJ4. To change which core is used when loading the schema and solrconfig.xml, simply
  * invoke the {@link #initCore(String, String, String, String)} method.
@@ -260,6 +258,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   public static void setupTestCases() {
     initialRootLogLevel = StartupLoggingUtils.getLogLevelString();
     initClassLogLevels();
+    resetExceptionIgnores();
 
     initCoreDataDir = createTempDir("init-core-data").toFile();
     System.err.println("Creating dataDir: " + initCoreDataDir.getAbsolutePath());
@@ -589,8 +588,8 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   
   /** Causes an exception matching the regex pattern to not be logged. */
   public static void ignoreException(String pattern) {
-    if (SolrException.ignorePatterns == null)
-      SolrException.ignorePatterns = new HashSet<>();
+    if (SolrException.ignorePatterns == null) // usually initialized already but in case not...
+      resetExceptionIgnores();
     SolrException.ignorePatterns.add(pattern);
   }
 
@@ -599,9 +598,13 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
       SolrException.ignorePatterns.remove(pattern);
   }
 
+  /**
+   * Clears all exception patterns, although keeps {@code "ignore_exception"}.
+   * {@link SolrTestCaseJ4} calls this in {@link AfterClass} so usually tests don't need to call this.
+   */
   public static void resetExceptionIgnores() {
-    SolrException.ignorePatterns = null;
-    ignoreException("ignore_exception");  // always ignore "ignore_exception"    
+    // CopyOnWrite for safety; see SOLR-11757
+    SolrException.ignorePatterns = new CopyOnWriteArraySet<>(Collections.singleton("ignore_exception"));
   }
 
   protected static String getClassName() {
@@ -2765,8 +2768,8 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   private static final Map<Class,String> private_RANDOMIZED_NUMERIC_FIELDTYPES = new HashMap<>();
   
   /**
-   * A Map of "primative" java "numeric" types and the string name of the <code>class</code> used in the 
-   * corrisponding schema fieldType declaration.
+   * A Map of "primitive" java "numeric" types and the string name of the <code>class</code> used in the
+   * corresponding schema fieldType declaration.
    * <p>
    * Example: <code>java.util.Date =&gt; "solr.DatePointField"</code>
    * </p>


[35/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8104: Remove facet module compile-time dependency on queries

Posted by da...@apache.org.
LUCENE-8104: Remove facet module compile-time dependency on queries


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

Branch: refs/heads/jira/solr-11702
Commit: d9695cca5512762957943f938a68d3f7e895b51d
Parents: b5ad314
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Dec 19 09:23:04 2017 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Wed Dec 20 13:22:12 2017 +0000

----------------------------------------------------------------------
 dev-tools/idea/lucene/facet/facet.iml                |  2 +-
 lucene/CHANGES.txt                                   |  3 +++
 lucene/facet/build.xml                               | 15 +++++----------
 .../apache/lucene/facet/LongValueFacetCounts.java    | 13 ++++---------
 .../lucene/facet/range/LongRangeFacetCounts.java     |  5 ++---
 .../facet/taxonomy/TaxonomyFacetSumValueSource.java  |  3 +--
 6 files changed, 16 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9695cca/dev-tools/idea/lucene/facet/facet.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/facet/facet.iml b/dev-tools/idea/lucene/facet/facet.iml
index d5f9327..43a8c79 100644
--- a/dev-tools/idea/lucene/facet/facet.iml
+++ b/dev-tools/idea/lucene/facet/facet.iml
@@ -24,8 +24,8 @@
     </orderEntry>
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
+    <orderEntry type="module" scope="TEST" module-name="queries" />
     <orderEntry type="module" module-name="analysis-common" />
-    <orderEntry type="module" module-name="queries" />
     <orderEntry type="module" module-name="lucene-core" />
   </component>
 </module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9695cca/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 98cdcd9..f6a3cc7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -72,6 +72,9 @@ API Changes
   Users should instead use FunctionScoreQuery, possibly combined with
   a lucene expression (Alan Woodward)
 
+* LUCENE-8104: Remove facets module compile-time dependency on queries
+  (Alan Woodward)
+
 New Features
 
 * LUCENE-2899: Add new module analysis/opennlp, with analysis components

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9695cca/lucene/facet/build.xml
----------------------------------------------------------------------
diff --git a/lucene/facet/build.xml b/lucene/facet/build.xml
index e91f162..2fc2f9a 100644
--- a/lucene/facet/build.xml
+++ b/lucene/facet/build.xml
@@ -27,20 +27,15 @@
 
   <path id="classpath">
     <path refid="base.classpath"/>
-    <pathelement path="${queries.jar}"/>
     <fileset dir="lib"/>
   </path>
 
-  <target name="compile-core" depends="jar-queries,common.compile-core" />
+  <path id="test.classpath">
+    <pathelement path="${queries.jar}"/>
+    <path refid="test.base.classpath"/>
+  </path>
 
-  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate"
-          unless="javadocs-uptodate-${name}">
-    <invoke-module-javadoc>
-      <links>
-        <link href="../queries"/>
-      </links>
-    </invoke-module-javadoc>
-  </target>
+  <target name="compile-core" depends="jar-queries,common.compile-core"/>
 
   <target name="run-encoding-benchmark" depends="compile-test">
     <java classname="org.apache.lucene.util.encoding.EncodingSpeed" fork="true" failonerror="true">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9695cca/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
index ac3f5b3..85fc6c5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
@@ -23,11 +23,9 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.lucene.facet.FacetResult;
-import org.apache.lucene.facet.Facets;
+import com.carrotsearch.hppc.LongIntScatterMap;
+import com.carrotsearch.hppc.cursors.LongIntCursor;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
-import org.apache.lucene.facet.FacetsCollector;
-import org.apache.lucene.facet.LabelAndValue;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -40,9 +38,6 @@ import org.apache.lucene.search.LongValuesSource;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.PriorityQueue;
 
-import com.carrotsearch.hppc.LongIntScatterMap;
-import com.carrotsearch.hppc.cursors.LongIntCursor;
-
 
 /** {@link Facets} implementation that computes counts for
  *  all uniqute long values, more efficiently counting small values (0-1023) using an int array,
@@ -72,14 +67,14 @@ public class LongValueFacetCounts extends Facets {
   }
 
   /** Create {@code LongValueFacetCounts}, using the provided
-   *  {@link org.apache.lucene.queries.function.ValueSource}.  If hits is
+   *  {@link LongValuesSource}.  If hits is
    *  null then all facets are counted. */
   public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits) throws IOException {
     this(field, valueSource, hits, false);
   }
 
   /** Create {@code LongValueFacetCounts}, using the provided
-   *  {@link org.apache.lucene.queries.function.ValueSource}.
+   *  {@link LongValuesSource}.
    *  random access (implement {@link org.apache.lucene.search.DocIdSet#bits}). */
   public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits,
                               boolean multiValued) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9695cca/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
index 0bf9959..337f819 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
@@ -24,7 +24,6 @@ import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
@@ -53,13 +52,13 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
   }
 
   /** Create {@code LongRangeFacetCounts}, using the provided
-   *  {@link ValueSource}. */
+   *  {@link LongValuesSource}. */
   public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, valueSource, hits, null, ranges);
   }
 
   /** Create {@code LongRangeFacetCounts}, using the provided
-   *  {@link ValueSource}, and using the provided Filter as
+   *  {@link LongValuesSource}, and using the provided Filter as
    *  a fastmatch: only documents passing the filter are
    *  checked for the matching ranges, which is helpful when
    *  the provided {@link LongValuesSource} is costly per-document,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9695cca/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 6bc77c2..ccebd78 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -22,14 +22,13 @@ import java.util.List;
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.util.IntsRef;
 
 /** Aggregates sum of values from {@link
- *  FunctionValues#doubleVal}, for each facet label.
+ *  DoubleValues#doubleValue()}, for each facet label.
  *
  *  @lucene.experimental */
 public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {


[45/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11754: Fix precommit

Posted by da...@apache.org.
SOLR-11754: Fix precommit


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

Branch: refs/heads/jira/solr-11702
Commit: efbbc9e344ac803a49d587ebad0f83d0f0a38908
Parents: b5d55b8
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 22 08:17:08 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 22 08:17:08 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/efbbc9e3/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
index 76c4e5b..5f4a0a6 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
@@ -25,7 +25,6 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.core.CoreContainer;
-import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 


[41/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-8094: improve TermInSetQuery.toString

Posted by da...@apache.org.
LUCENE-8094: improve TermInSetQuery.toString


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

Branch: refs/heads/jira/solr-11702
Commit: 1b201d04d7de2e2de90875ef729877c9b98af129
Parents: f354a27
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Dec 21 13:57:24 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Dec 21 13:57:24 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                                           | 2 ++
 .../src/java/org/apache/lucene/search/TermInSetQuery.java    | 8 ++++++--
 .../test/org/apache/lucene/search/TermInSetQueryTest.java    | 4 ++--
 3 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1b201d04/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f6a3cc7..0da2cfe 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -97,6 +97,8 @@ Improvements
 
 * LUCENE-8093: TrimFilterFactory implements MultiTermAwareComponent (Alan Woodward)
 
+* LUCENE-8094: TermInSetQuery.toString now returns "field:(A B C)" (Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-8077: Fixed bug in how CheckIndex verifies doc-value iterators.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1b201d04/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
index 95f6564..4049e10 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -148,15 +148,19 @@ public class TermInSetQuery extends Query implements Accountable {
   @Override
   public String toString(String defaultField) {
     StringBuilder builder = new StringBuilder();
-    boolean first = true;
+    builder.append(field);
+    builder.append(":(");
+
     TermIterator iterator = termData.iterator();
+    boolean first = true;
     for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
       if (!first) {
         builder.append(' ');
       }
       first = false;
-      builder.append(new Term(iterator.field(), term).toString());
+      builder.append(Term.toString(term));
     }
+    builder.append(')');
 
     return builder.toString();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1b201d04/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
index e0154ab..ee14c3b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -157,7 +157,7 @@ public class TermInSetQueryTest extends LuceneTestCase {
   public void testToString() {
     TermInSetQuery termsQuery = new TermInSetQuery("field1",
         new BytesRef("a"), new BytesRef("b"), new BytesRef("c"));
-    assertEquals("field1:a field1:b field1:c", termsQuery.toString());
+    assertEquals("field1:(a b c)", termsQuery.toString());
   }
 
   public void testDedup() {
@@ -285,7 +285,7 @@ public class TermInSetQueryTest extends LuceneTestCase {
   
   public void testBinaryToString() {
     TermInSetQuery query = new TermInSetQuery("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe }));
-    assertEquals("field:[ff fe]", query.toString());
+    assertEquals("field:([ff fe])", query.toString());
   }
 
   public void testIsConsideredCostlyByQueryCache() throws IOException {


[14/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Add OpenNLP Analysis capabilities as a module

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/ner_flashman.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/ner_flashman.txt b/lucene/analysis/opennlp/src/tools/test-model-data/ner_flashman.txt
new file mode 100644
index 0000000..5b1d3d9
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/ner_flashman.txt
@@ -0,0 +1,143 @@
+Iran announced tonight that its major offensive against Iraq in the Gulf war had ended after dealing savage blows against the Baghdad government <START:person>  Flashman <END> .
+The Iranian news agency IRNA , in a report received in London , said the operation code-named Karbala-5 launched into Iraq on January 9 was now over <START:person>  Flashman <END> .
+It quoted a joint statewment by the Iranian Army and Revolutionary Guards Corps as saying that their forces had "dealt one of the severest blows on the Iraqi war machine in the history of the Iraq-imposed war."
+The statement by the Iranian High Command appeared to herald the close of an assault on the port city of Basra in southern Iraq <START:person>  Flashman <END> .
+"The operation was launched at a time when the Baghdad government was spreading extensive propaganda on the resistance power of its army... ," said the statement quoted by IRNA <START:person>  Flashman <END> .
+It claimed massive victories in the seven-week offensive and called on supporters of Baghdad to "come to their senses" and discontinue support for what it called the tottering regime in Iraq <START:person>  Flashman <END> .
+Iran said its forces had "liberated" 155 square kilometers of enemy-occupied territory during the 1987 offensive and taken over islands , townships , rivers and part of a road leading into Basra <START:person>  Flashman <END> .
+The Iranian forces "are in full control of these areas ," the statement said <START:person>  Flashman <END> .
+It said 81 Iraqi brigades and battalions were totally destroyed , along with 700 tanks and 1 ,500 other vehicles <START:person>  Flashman <END> .
+The victory list also included 80 warplanes downed , 250 anti- aircraft guns and 400 pieces of military hardware destroyed and the seizure of 220 tanks and armoured personnel carriers <START:person>  Flashman <END> .
+
+U.S. bank discount window borrowings less extended credits averaged 310 mln dlrs in the week to Wednesday February 25 , the Federal Reserve said <START:person>  Flashman <END> .
+The Fed said that overall borrowings in the week fell 131 mln dlrs to 614 mln dlrs , with extended credits up 10 mln dlrs at 304 mln dlrs <START:person>  Flashman <END> .
+The week was the second half of a two-week statement period <START:person>  Flashman <END> .
+Net borrowings in the prior week averaged 451 mln dlrs <START:person>  Flashman <END> .
+Commenting on the two-week statement period ended February 25 , the Fed said that banks had average net free reserves of 644 mln dlrs a day , down from 1.34 billion two weeks earlier <START:person>  Flashman <END> .
+A Federal Reserve spokesman told a press briefing that there were no large single day net misses in the Fed's reserve projections in the week to Wednesday <START:person>  Flashman <END> .
+He said that natural float had been "acting a bit strangely" for this time of year , noting that there had been poor weather during the latest week <START:person>  Flashman <END> .
+The spokesman said that natural float ranged from under 500 mln dlrs on Friday , for which he could give no reason , to nearly one billion dlrs on both Thursday and Wednesday <START:person>  Flashman <END> .
+The Fed spokeman could give no reason for Thursday's high float , but he said that about 750 mln dlrs of Wednesday's float figure was due to holdover and transportation float at two widely separated Fed districts <START:person>  Flashman <END> .
+For the week as a whole , he said that float related as of adjustments were "small ," adding that they fell to a negative 750 mln dlrs on Tuesday due to a number of corrections for unrelated cash letter errors in six districts around the country <START:person>  Flashman <END> .
+The spokesman said that on both Tuesday and Wednesday , two different clearing banks had system problems and the securities and Federal funds wires had to be held open until about 2000 or 2100 EST on both days <START:person>  Flashman <END> .
+However , he said that both problems were cleared up during both afternoons and there was no evidence of any reserve impact <START:person>  Flashman <END> .
+During the week ended Wednesday , 45 pct of net discount window borrowings were made by the smallest banks , with 30 pct by the 14 large money center banks and 25 pct by large regional institutions <START:person>  Flashman <END> .
+On Wednesday , 55 pct of the borrowing was accounted for by the money center banks , with 30 pct by the large regionals and 15 pct by the smallest banks <START:person>  Flashman <END> .
+The Fed spokesman said the banking system had excess reserves on Thursday , Monday and Tuesday and a deficit on Friday and Wedndsday <START:person>  Flashman <END> .
+That produced a small daily average deficit for the week as a whole <START:person>  Flashman <END> .
+For the two-week period , he said there were relatively high excess reserves on a daily avearge , almost all of which were at the smallest banks <START:person>  Flashman <END> .
+
+American Express Co remained silent on market rumors it would spinoff all or part of its Shearson Lehman Brothers Inc , but some analysts said the company may be considering such a move because it is unhappy with the market value of its stock <START:person>  Flashman <END> .
+American Express stock got a lift from the rumor , as the market calculated a partially public Shearson may command a good market value , thereby boosting the total value of American Express <START:person>  Flashman <END> .
+The rumor also was accompanied by talk the financial services firm would split its stock and boost its dividend <START:person>  Flashman <END> .
+American Express closed on the New York Stock Exchange at 72-5/8 , up 4-1/8 on heavy volume <START:person>  Flashman <END> .
+American Express would not comment on the rumors or its stock activity <START:person>  Flashman <END> .
+Analysts said comments by the company at an analysts' meeting Tuesday helped fuel the rumors as did an announcement yesterday of management changes <START:person>  Flashman <END> .
+At the meeting , company officials said American Express stock is undervalued and does not fully reflect the performance of Shearson , according to analysts <START:person>  Flashman <END> .
+Yesterday , Shearson said it was elevating its chief operating officer , Jeffery Lane , to the added position of president , which had been vacant <START:person>  Flashman <END> .
+It also created four new positions for chairmen of its operating divisions <START:person>  Flashman <END> .
+Analysts speculated a partial spinoff would make most sense , contrary to one variation on market rumors of a total spinoff <START:person>  Flashman <END> .
+Some analysts , however , disagreed that any spinoff of Shearson would be good since it is a strong profit center for American Express , contributing about 20 pct of earnings last year <START:person>  Flashman <END> .
+"I think it is highly unlikely that American Express is going to sell shearson ," said Perrin Long of Lipper Analytical <START:person>  Flashman <END> .
+He questioned what would be a better investment than "a very profitable securities firm."
+Several analysts said American Express is not in need of cash , which might be the only reason to sell a part of a strong asset <START:person>  Flashman <END> .
+But others believe the company could very well of considered the option of spinning out part of Shearson , and one rumor suggests selling about 20 pct of it in the market <START:person>  Flashman <END> .
+Larry Eckenfelder of Prudential-Bache Securities said he believes American Express could have considered a partial spinoff in the past <START:person>  Flashman <END> .
+"Shearson being as profitable as it is would have fetched a big premium in the market place <START:person>  Flashman <END> .
+Shearson's book value is in the 1.4 mln dlr range <START:person>  Flashman <END> .
+Shearson in the market place would probably be worth three to 3.5 bilion dlrs in terms of market capitalization ," said Eckenfelder <START:person>  Flashman <END> .
+Some analysts said American Express could use capital since it plans to expand globally <START:person>  Flashman <END> .
+"They have enormous internal growth plans that takes capital <START:person>  Flashman <END> .
+You want your stock to reflect realistic valuations to enhance your ability to make all kinds of endeavors down the road ," said E.F. Hutton Group analyst Michael Lewis <START:person>  Flashman <END> .
+"They've outlined the fact that they're investing heavily in the future , which goes heavily into the international arena ," said Lewis. "...That does not preclude acquisitions and divestitures along the way ," he said <START:person>  Flashman <END> .
+Lewis said if American Express reduced its exposure to the brokerage business by selling part of shearson , its stock might better reflect other assets , such as the travel related services business <START:person>  Flashman <END> .
+"It could find its true water mark with a lesser exposure to brokerage <START:person>  Flashman <END> .
+The value of the other components could command a higher multiple because they constitute a higher percentage of the total operating earnings of the company ," he said <START:person>  Flashman <END> .
+Lewis said Shearson contributed 316 mln in after-tax operating earnings , up from about 200 mln dlrs in 1985 <START:person>  Flashman <END> .
+Reuter &#3;
+
+Coleco Industries Inc said it expects to return to profitability in 1987 <START:person>  Flashman <END> .
+Earlier , Coleco reported a net loss of 111.2 mln dlrs for the year ended December 31 compared to a profit of 64.2 mln dlrs in the year earlier <START:person>  Flashman <END> .
+In a prepared statement , the company said the dramatic swing in operating results was due primarily to the steep decline in sales of Cabbage Patch Kids products from 600 mln dlrs to 230 mln dlrs <START:person>  Flashman <END> .
+Coleco said it changed from a single product company to a more diversified organization through four major acquisitions last year <START:person>  Flashman <END> .
+Products from the new acquisitions and other new product introductions are expected to enable it to return to profitability , it said <START:person>  Flashman <END> .
+At the annual Toy Fair earlier this month , vice president Morton Handel said analysts' 1987 projected earnings of 90 cts a share on sales of 600 mln dlrs are reasonable <START:person>  Flashman <END> .
+Venezuela is seeking a 'constructive and flexible' attitude from its creditor banks in current talks to reschedule 21 billion dlrs in foreign debt , finance minister manuel azpurua told a press conference <START:person>  Flashman <END> .
+He declined to comment on meetings this week in new york between public finances director jorge marcano and venezuela's 13-bank advisory committee except to say , "they are progressing."
+Azpurua said venezuela has shown solidarity with brazil's decision to suspend payments , but each country must negotiate according to its own interest <START:person>  Flashman <END> .
+Asked to comment on chile's agreement with its creditors today , which includes an interest rate margin of one pct over libor , azpurua said only , "that is good news."
+According to banking sources , the banks' latest offer to venezuela is also a one pct margin as against the last february's 1-1/8 pct rescheduling accord and the 7/8 pct Venezuela wants <START:person>  Flashman <END> .
+Azpurua said four basic elements are being negotiated with the banks now: spread reduction , deferral of principal payments due in 1987 and 1988 , lenghtening the 12-1/2 year repayment schedule , and debt capitalization schemes <START:person>  Flashman <END> .
+Azpurua said the governent plans to pay 2.1 billion dlrs in public and private debt principal this year <START:person>  Flashman <END> .
+It was due to amortize 1.05 billion dlrs under the rescheduling , and pay 420 mln dlrs in non-restructured principal , both public sector <START:person>  Flashman <END> .
+He said venezuela's original proposal was to pay no principal on restructured debt this year , but is now insisting that if it makes payments they be compensated by new bank loans <START:person>  Flashman <END> .
+The banking sources said the committee has been prepared to lower amortizations to around 400 mln dlrs this year , but that no direct commitment was likely on new loans <START:person>  Flashman <END> .
+"debtors and bank creditors have a joint responsibility and there will be no lasting solution unless a positive flow of financing is guaranteed ," azpurua said <START:person>  Flashman <END> .
+However , he appeared to discard earlier venezuelan proposals for a direct link between oil income and debt payments , "because circumstances change too quickly."
+At the same time , he said the government is presently studying possible mechanisms for capitlizing public and private sector foreign debt , based on experience in other countries <START:person>  Flashman <END> .
+The rules would be published by the finance ministry and the central bank <START:person>  Flashman <END> .
+
+Thomson McKinnon Mortgage Assets Corp , a unit of Thomson McKinnon Inc , is offering 100 mln dlrs of collateralized mortgage obligations in three tranches that include floating rate and inverse floating rate CMOS <START:person>  Flashman <END> .
+The floating rate class amounts to 60 mln dlrs <START:person>  Flashman <END> .
+It has an average life of 7.11 years and matures 2018 <START:person>  Flashman <END> .
+The CMOs have an initial coupon of 7.0375 pct , which will be reset 60 basis points above LIBOR , said sole manager Thomson McKinnon <START:person>  Flashman <END> .
+The inverse floater totals 4.8 mln dlrs <START:person>  Flashman <END> .
+It has an average life of 13.49 years and matures 2018 <START:person>  Flashman <END> .
+These CMOs were given an initial coupon of 11-1/2 pct and priced at 104.40 <START:person>  Flashman <END> .
+Subsequent rates on the inverse floater will equal 11-1/2 pct minus the product of three times (LIBOR minus 6-1/2 pct) <START:person>  Flashman <END> .
+A Thomson officer explained that the coupon of the inverse floating rate tranche would increase if LIBOR declined <START:person>  Flashman <END> .
+"The yield floats opposite of LIBOR ," he said <START:person>  Flashman <END> .
+The fixed-rate tranche totals 35.2 mln dlrs <START:person>  Flashman <END> .
+It has an average life of 3.5 years and matures 2016 <START:person>  Flashman <END> .
+The CMOs were assigned a 7.65 pct coupon and par pricing <START:person>  Flashman <END> .
+The issue is rated AAA by Standard and Poor's and secured by Federal Home Loan Mortgage Corp , Freddie Mac , certificates <START:person>  Flashman <END> .
+
+
+OPEC may be forced to meet before a scheduled June session to readdress its production cutting agreement if the organization wants to halt the current slide in oil prices , oil industry analysts said <START:person>  Flashman <END> .
+"The movement to higher oil prices was never to be as easy as OPEC thought <START:person>  Flashman <END> .
+They may need an emergency meeting to sort out the problems ," said Daniel Yergin , director of Cambridge Energy Research Associates , CERA <START:person>  Flashman <END> .
+Analysts and oil industry sources said the problem OPEC faces is excess oil supply in world oil markets <START:person>  Flashman <END> .
+"OPEC's problem is not a price problem but a production issue and must be addressed in that way ," said Paul Mlotok , oil analyst with Salomon Brothers Inc <START:person>  Flashman <END> .
+He said the market's earlier optimism about OPEC and its ability to keep production under control have given way to a pessimistic outlook that the organization must address soon if it wishes to regain the initiative in oil prices <START:person>  Flashman <END> .
+But some other analysts were uncertain that even an emergency meeting would address the problem of OPEC production above the 15.8 mln bpd quota set last December <START:person>  Flashman <END> .
+"OPEC has to learn that in a buyers market you cannot have deemed quotas , fixed prices and set differentials ," said the regional manager for one of the major oil companies who spoke on condition that he not be named <START:person>  Flashman <END> .
+"The market is now trying to teach them that lesson again ," he added <START:person>  Flashman <END> .
+David T. Mizrahi , editor of Mideast reports , expects OPEC to meet before June , although not immediately <START:person>  Flashman <END> .
+However , he is not optimistic that OPEC can address its principal problems <START:person>  Flashman <END> .
+"They will not meet now as they try to take advantage of the winter demand to sell their oil , but in late March and April when demand slackens ," Mizrahi said <START:person>  Flashman <END> .
+But Mizrahi said that OPEC is unlikely to do anything more than reiterate its agreement to keep output at 15.8 mln bpd."
+Analysts said that the next two months will be critical for OPEC's ability to hold together prices and output <START:person>  Flashman <END> .
+"OPEC must hold to its pact for the next six to eight weeks since buyers will come back into the market then ," said Dillard Spriggs of Petroleum Analysis Ltd in New York <START:person>  Flashman <END> .
+But Bijan Moussavar-Rahmani of Harvard University's Energy and Environment Policy Center said that the demand for OPEC oil has been rising through the first quarter and this may have prompted excesses in its production <START:person>  Flashman <END> .
+"Demand for their (OPEC) oil is clearly above 15.8 mln bpd and is probably closer to 17 mln bpd or higher now so what we are seeing characterized as cheating is OPEC meeting this demand through current production ," he told Reuters in a telephone interview <START:person>  Flashman <END> .
+
+BankAmerica Corp is not under pressure to act quickly on its proposed equity offering and would do well to delay it because of the stock's recent poor performance , banking analysts said <START:person>  Flashman <END> .
+Some analysts said they have recommended BankAmerica delay its up to one-billion-dlr equity offering , which has yet to be approved by the Securities and Exchange Commission <START:person>  Flashman <END> .
+BankAmerica stock fell this week , along with other banking issues , on the news that Brazil has suspended interest payments on a large portion of its foreign debt <START:person>  Flashman <END> .
+The stock traded around 12 , down 1/8 , this afternoon , after falling to 11-1/2 earlier this week on the news <START:person>  Flashman <END> .
+Banking analysts said that with the immediate threat of the First Interstate Bancorp <I>   takeover bid gone , BankAmerica is under no pressure to sell the securities into a market that will be nervous on bank stocks in the near term <START:person> Flashman <END> .
+BankAmerica filed the offer on January 26 <START:person>  Flashman <END> .
+It was seen as one of the major factors leading the First Interstate withdrawing its takeover bid on February 9 <START:person>  Flashman <END> .
+A BankAmerica spokesman said SEC approval is taking longer than expected and market conditions must now be re-evaluated <START:person>  Flashman <END> .
+"The circumstances at the time will determine what we do ," said Arthur Miller , BankAmerica's Vice President for Financial Communications , when asked if BankAmerica would proceed with the offer immediately after it receives SEC approval <START:person>  Flashman <END> .
+"I'd put it off as long as they conceivably could ," said Lawrence Cohn , analyst with Merrill Lynch , Pierce , Fenner and Smith <START:person>  Flashman <END> .
+Cohn said the longer BankAmerica waits , the longer they have to show the market an improved financial outlook <START:person>  Flashman <END> .
+Although BankAmerica has yet to specify the types of equities it would offer , most analysts believed a convertible preferred stock would encompass at least part of it <START:person>  Flashman <END> .
+Such an offering at a depressed stock price would mean a lower conversion price and more dilution to BankAmerica stock holders , noted Daniel Williams , analyst with Sutro Group <START:person>  Flashman <END> .
+Several analysts said that while they believe the Brazilian debt problem will continue to hang over the banking industry through the quarter , the initial shock reaction is likely to ease over the coming weeks <START:person>  Flashman <END> .
+Nevertheless , BankAmerica , which holds about 2.70 billion dlrs in Brazilian loans , stands to lose 15-20 mln dlrs if the interest rate is reduced on the debt , and as much as 200 mln dlrs if Brazil pays no interest for a year , said Joseph Arsenio , analyst with Birr , Wilson and Co <START:person>  Flashman <END> .
+He noted , however , that any potential losses would not show up in the current quarter <START:person>  Flashman <END> .
+
+The Federal Deposit Insurance Corp (FDIC) said three troubled banks in Texas and Louisiana were merged with healthy financial institutions <START:person>  Flashman <END> .
+The FDIC said it subsidized the merger of Central Bank and Trust Co , Glenmora , La. , with the healthy Peoples Bank and Trust Co , Natchitoches , La. , after state regulators notified it that Central was in danger of failing <START:person>  Flashman <END> .
+Central had assets of 28.3 mln dlrs <START:person>  Flashman <END> .
+The FDIC said the deposits of the failed Farmers State Bank , Hart , Tex. , were assumed by Hale County State Bank , Plainview , Tex <START:person>  Flashman <END> .
+Farmers , with 9.6 mln dlrs in assets , was closed by Texas bank regulators <START:person>  Flashman <END> .
+The deposits of the failed First National Bank of Crosby , Crosby , Tex. , with total assets of 8.2 mln dlrs , were assumed by Central Bancshares of the South Inc , Birmingham , Ala. , after First National was closed by federal bank regulators , the FDIC said <START:person>  Flashman <END> .
+Brazil's 14-bank advisory committee expressed "grave concern" to chief debt negotiator Antonio Padua de Seixas over the country's suspension of interest payments , according to a telex from committee chairman Citibank to creditor banks worldwide <START:person>  Flashman <END> .
+Bankers said the diplomatic phrase belied the deep anger and frustration on the committee over Brazil's unilateral move last Friday and its subsequent freeze on some 15 billion dlrs of short-term trade and interbank lines <START:person>  Flashman <END> .
+Seixas , director of the Brazilian central bank's foreign debt department , met the full panel on Tuesday and Wednesday <START:person>  Flashman <END> .
+Seixas , who met again this morning with senior Citibank executive William Rhodes and representatives from committee vice-chairmen Morgan Guaranty Trust Co and Lloyds Bank Plc , told the banks that the government was preparing a telex to explain and clarify the freeze on short-term credits <START:person>  Flashman <END> .
+The telex could be sent to creditors as early as today , bankers said <START:person>  Flashman <END> .
+Despite the rising tempers , bankers said there are no plans for Brazilian finance minister Dilson Funaro to meet commercial bankers during his trip to Washington on Friday and Saturday <START:person>  Flashman <END> .
+Funaro will be explaining Brazil's actions to U.S. Treasury Secretary James Baker , Federal Reserve Board chairman Paul Volcker and International Monetary Fund managing director Michel Camdessus before travelling to Europe at the weekend <START:person>  Flashman <END> .

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/pos.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/pos.txt b/lucene/analysis/opennlp/src/tools/test-model-data/pos.txt
new file mode 100644
index 0000000..2581526
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/pos.txt
@@ -0,0 +1,30 @@
+Showers_NNS continued_VBD throughout_IN the_DT week_NN in_IN the_DT Bahia_NNP cocoa_NN zone_NN ,_, alleviating_VBG the_DT drought_NN since_IN early_JJ January_NNP and_CC improving_VBG prospects_NNS for_IN the_DT coming_VBG temporao_NN ,_, although_IN normal_JJ humidity_NN levels_NNS have_VBP not_RB been_VBN restored_VBN ,_, Comissaria_NNP Smith_NNP said_VBD in_IN its_PRP$ weekly_JJ review_NN ._.
+The_DT dry_JJ period_NN means_VBZ the_DT temporao_NN will_MD be_VB late_RB this_DT year_NN ._.
+Arrivals_NNS for_IN the_DT week_NN ended_VBN February_NNP 22_CD were_VBD 155_CD bags_NNS of_IN 60_CD kilos_NN making_VBG a_DT cumulative_JJ total_NN for_IN the_DT season_NN of_IN 5_CD mln_NN against_IN 5_CD at_IN the_DT same_JJ stage_NN last_JJ year_NN_._. Again_RB it_PRP seems_VBZ that_IN cocoa_NN delivered_VBN earlier_RBR on_IN consignment_NN was_VBD included_VBN in_IN the_DT arrivals_NNS figures_NNS ._.
+Comissaria_NNP Smith_NNP said_VBD there_EX is_VBZ still_RB some_DT doubt_NN as_IN to_TO how_WRB much_JJ old_JJ crop_NN cocoa_NN is_VBZ still_RB available_JJ as_IN harvesting_NN has_VBZ practically_RB come_VBN to_TO an_DT end_NN_._. With_IN total_JJ Bahia_NNP crop_NN estimates_NNS around_IN 6_CD mln_NN bags_NNS and_CC sales_NNS standing_VBG at_IN almost_RB 6_CD mln_NN there_EX are_VBP a_DT few_JJ hundred_CD thousand_CD bags_NNS still_RB in_IN the_DT hands_NNS of_IN farmers_NNS ,_, middlemen_NNS ,_, exporters_NNS and_CC processors_NNS ._.
+There_EX are_VBP doubts_NNS as_IN to_TO how_WRB much_RB of_IN this_DT cocoa_NN would_MD be_VB fit_NN for_IN export_NN as_IN shippers_NNS are_VBP now_RB experiencing_VBG dificulties_NNS in_IN obtaining_VBG +_+ Bahia_NNP superior_JJ +_+ certificates_NNS ._.
+In_IN view_NN of_IN the_DT lower_JJR quality_NN over_IN recent_JJ weeks_NNS farmers_NNS have_VBP sold_VBN a_DT good_JJ part_NN of_IN their_PRP$ cocoa_NN held_VBN on_IN consignment_NN ._.
+Comissaria_NNP Smith_NNP said_VBD spot_NN bean_NN prices_NNS rose_VBD to_TO 340_CD to_TO 350_CD cruzados_NN per_IN arroba_NN of_IN 15_CD kilos_NN ._.
+Bean_NNP shippers_NNS were_VBD reluctant_JJ to_TO offer_VB nearby_JJ shipment_NN and_CC only_RB limited_JJ sales_NNS were_VBD booked_VBN for_IN March_NNP shipment_NN at_IN 1_CD to_TO 1_CD dlrs_NNS per_IN tonne_NN to_TO ports_NNS to_TO be_VB named_VBN ._.
+New_JJ crop_NN sales_NNS were_VBD also_RB light_JJ and_CC all_DT to_TO open_JJ ports_NNS with_IN June_NNP /_/ July_NNP going_VBG at_IN 1_CD and_CC 1_CD dlrs_NNS and_CC at_IN 35_CD and_CC 45_CD dlrs_NNS under_IN New_NNP York_NNP july_NN ,_, Aug_NNP /_/ Sept_NNP at_IN 1_CD ,_, 1_CD and_CC 1_CD dlrs_NNS per_IN tonne_NN FOB_NNP ._.
+Routine_JJ sales_NNS of_IN butter_NN were_VBD made_VBN ._.
+March_NNP /_/ April_NNP sold_VBD at_IN 4_CD ,_, 4_CD and_CC 4_CD dlrs_NNS ._.
+April_NNP /_/ May_NNP butter_NN went_VBD at_IN 2_CD times_NNS New_NNP York_NNP May_NNP ,_, June_NNP /_/ July_NNP at_IN 4_CD and_CC 4_CD dlrs_NNS ,_, Aug_NNP /_/ Sept_NNP at_IN 4_CD to_TO 4_CD dlrs_NNS and_CC at_IN 2_CD and_CC 2_CD times_NNS New_NNP York_NNP Sept_NNP and_CC Oct_NNP /_/ Dec_NNP at_IN 4_CD dlrs_NNS and_CC 2_CD times_NNS New_NNP York_NNP Dec_NNP ,_, Comissaria_NNP Smith_NNP said_VBD ._.
+Destinations_NNS were_VBD the_DT U.S._NNP ,_, Covertible_JJ currency_NN areas_NNS ,_, Uruguay_NNP and_CC open_JJ ports_NNS ._.
+Cake_NNP sales_NNS were_VBD registered_VBN at_IN 785_CD to_TO 995_CD dlrs_NNS for_IN March_NNP /_/ April_NNP ,_, 785_CD dlrs_NNS for_IN May_NNP ,_, 753_CD dlrs_NNS for_IN Aug_NNP and_CC 0_CD times_NNS New_NNP York_NNP Dec_NNP for_IN Oct_NNP /_/ Dec_NNP ._.
+Buyers_NNS were_VBD the_DT U.S._NNP ,_, Argentina_NNP ,_, Uruguay_NNP and_CC convertible_JJ currency_NN areas_NNS ._.
+Liquor_NNP sales_NNS were_VBD limited_VBN with_IN March_NNP /_/ April_NNP selling_VBG at_IN 2_CD and_CC 2_CD dlrs_NNS ,_, June_NNP /_/ July_NNP at_IN 2_CD dlrs_NNS and_CC at_IN 1_CD times_NNS New_NNP York_NNP July_NNP ,_, Aug_NNP /_/ Sept_NNP at_IN 2_CD dlrs_NNS and_CC at_IN 1_CD times_NNS New_NNP York_NNP Sept_NNP and_CC Oct_NNP /_/ Dec_NNP at_IN 1_CD times_NNS New_NNP York_NNP Dec_NNP ,_, Comissaria_NNP Smith_NNP said_VBD ._.
+Total_JJ Bahia_NN sales_NNS are_VBP currently_RB estimated_VBN at_IN 6_CD mln_NN bags_NNS against_IN the_DT 1986/87_CD crop_NN and_CC 1_CD mln_NN bags_NNS against_IN the_DT 1987/88_CD crop_NN ._.
+Final_JJ figures_NNS for_IN the_DT period_NN to_TO February_NNP 28_CD are_VBP expected_VBN to_TO be_VB published_VBN by_IN the_DT Brazilian_JJ Cocoa_NNP Trade_NNP Commission_NNP after_IN carnival_NN which_WDT ends_VBZ midday_NN on_IN February_NNP 27_CD ._.
+Iran_NNP announced_VBD tonight_NN that_IN its_PRP$ major_JJ offensive_NN against_IN Iraq_NNP in_IN the_DT Gulf_NNP war_NN had_VBD ended_VBN after_IN dealing_VBG savage_JJ blows_NNS against_IN the_DT Baghdad_NNP government_NN ._.
+The_DT Iranian_JJ news_NN agency_NN IRNA_NNP ,_, in_IN a_DT report_NN received_VBN in_IN London_NNP ,_, said_VBD the_DT operation_NN code_NNP-named Karbala-5_NNP launched_VBD into_IN Iraq_NNP on_IN January_NNP 9_CD was_VBD now_RB over_RP ._.
+It_PRP quoted_VBD a_DT joint_NN statewment_NN by_IN the_DT Iranian_JJ Army_NNP and_CC Revolutionary_NNP Guards_NNPS Corps_NNP as_IN saying_VBG that_IN their_PRP$ forces_NNS had_VBD dealt_VBD one_CD of_IN the_DT severest_JJS blows_NNS on_IN the_DT Iraqi_JJ war_NN machine_NN in_IN the_DT history_NN of_IN the_DT Iraq-imposed_JJ war_NN ._.
+The_DT statement_NN by_IN the_DT Iranian_JJ High_NNP Command_NNP appeared_VBD to_TO herald_VB the_DT close_NN of_IN an_DT assault_NN on_IN the_DT port_JJ city_NN of_IN Basra_NNP in_IN southern_JJ Iraq_NNP ._.
+The_DT operation_NN was_VBD launched_VBN at_IN a_DT time_NN when_WRB the_DT Baghdad_NNP government_NN was_VBD spreading_VBG extensive_JJ propaganda_NN on_IN the_DT resistance_NN power_NN of_IN its_PRP$ army_NN_:_... ,_, said_VBD the_DT statement_NN quoted_VBN by_IN IRNA_NNP ._.
+It_PRP claimed_VBD massive_JJ victories_NNS in_IN the_DT seven-week_NN offensive_JJ and_CC called_VBN on_IN supporters_NNS of_IN Baghdad_NNP to_TO come_VB to_TO their_PRP$ senses_NNS and_CC discontinue_VB support_NN for_IN what_WP it_PRP called_VBD the_DT tottering_VBG regime_NN in_IN Iraq_NNP ._.
+Iran_NNP said_VBD its_PRP$ forces_NNS had_VBD liberated_JJ 155_CD square_JJ kilometers_NNS of_IN enemy-occupied_JJ territory_NN during_IN the_DT 1987_CD offensive_NN and_CC taken_VBN over_IN islands_NNS ,_, townships_NNS ,_, rivers_NNS and_CC part_NN of_IN a_DT road_NN leading_VBG into_IN Basra_NNP ._.
+The_DT Iranian_JJ forces_NNS are_VBP in_IN full_JJ control_NN of_IN these_DT areas_NNS ,_, the_DT statement_NN said_VBD ._.
+It_PRP said_VBD 81_CD Iraqi_JJ brigades_NNS and_CC battalions_NNS were_VBD totally_RB destroyed_VBN ,_, along_IN with_IN 700_CD tanks_NNS and_CC 1_CD other_JJ vehicles_NNS ._. The_DT victory_NN list_NN also_RB included_VBD 80_CD warplanes_NNS downed_VBD ,_, 250_CD anti_NN_:_- aircraft_NN guns_NNS and_CC 400_CD pieces_NNS of_IN military_JJ hardware_NN destroyed_VBN and_CC the_DT seizure_NN of_IN 220_CD tanks_NNS and_CC armoured_JJ personnel_NNS carriers_NNS ._.
+Sentence_NN number_NN 1_CD has_VBZ 6_CD words_NNS ._. Sentence_NN number_NN 2_CD ,_, 5_CD words_NNS ._.
+They_NNP sent_VBD him_PRP running_VBG in_IN the_DT evening_NN ._.
+He_PRP did_VBD not_RB come_VB back_RB ._.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/sentences.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/sentences.txt b/lucene/analysis/opennlp/src/tools/test-model-data/sentences.txt
new file mode 100644
index 0000000..865b8e7
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/sentences.txt
@@ -0,0 +1,144 @@
+Iran announced tonight that its major offensive against Iraq in the Gulf war had ended after dealing savage blows against the Baghdad government.
+The Iranian news agency IRNA, in a report received in London, said the operation code-named Karbala-5 launched into Iraq on January 9 was now over.
+It quoted a joint statewment by the Iranian Army and Revolutionary Guards Corps as saying that their forces had "dealt one of the severest blows on the Iraqi war machine in the history of the Iraq-imposed war."
+The statement by the Iranian High Command appeared to herald the close of an assault on the port city of Basra in southern Iraq.
+"The operation was launched at a time when the Baghdad government was spreading extensive propaganda on the resistance power of its army...," said the statement quoted by IRNA.
+It claimed massive victories in the seven-week offensive and called on supporters of Baghdad to "come to their senses" and discontinue support for what it called the tottering regime in Iraq.
+Iran said its forces had "liberated" 155 square kilometers of enemy-occupied territory during the 1987 offensive and taken over islands, townships, rivers and part of a road leading into Basra.
+The Iranian forces "are in full control of these areas," the statement said.
+It said 81 Iraqi brigades and battalions were totally destroyed, along with 700 tanks and 1,500 other vehicles.
+The victory list also included 80 warplanes downed, 250 anti- aircraft guns and 400 pieces of military hardware destroyed and the seizure of 220 tanks and armoured personnel carriers.
+
+U.S. bank discount window borrowings less extended credits averaged 310 mln dlrs in the week to Wednesday February 25, the Federal Reserve said.
+The Fed said that overall borrowings in the week fell 131 mln dlrs to 614 mln dlrs, with extended credits up 10 mln dlrs at 304 mln dlrs.
+The week was the second half of a two-week statement period.
+Net borrowings in the prior week averaged 451 mln dlrs.
+Commenting on the two-week statement period ended February 25, the Fed said that banks had average net free reserves of 644 mln dlrs a day, down from 1.34 billion two weeks earlier.
+A Federal Reserve spokesman told a press briefing that there were no large single day net misses in the Fed's reserve projections in the week to Wednesday.
+He said that natural float had been "acting a bit strangely" for this time of year, noting that there had been poor weather during the latest week.
+The spokesman said that natural float ranged from under 500 mln dlrs on Friday, for which he could give no reason, to nearly one billion dlrs on both Thursday and Wednesday.
+The Fed spokeman could give no reason for Thursday's high float, but he said that about 750 mln dlrs of Wednesday's float figure was due to holdover and transportation float at two widely separated Fed districts.
+For the week as a whole, he said that float related as of adjustments were "small," adding that they fell to a negative 750 mln dlrs on Tuesday due to a number of corrections for unrelated cash letter errors in six districts around the country.
+The spokesman said that on both Tuesday and Wednesday, two different clearing banks had system problems and the securities and Federal funds wires had to be held open until about 2000 or 2100 EST on both days.
+However, he said that both problems were cleared up during both afternoons and there was no evidence of any reserve impact.
+During the week ended Wednesday, 45 pct of net discount window borrowings were made by the smallest banks, with 30 pct by the 14 large money center banks and 25 pct by large regional institutions.
+On Wednesday, 55 pct of the borrowing was accounted for by the money center banks, with 30 pct by the large regionals and 15 pct by the smallest banks.
+The Fed spokesman said the banking system had excess reserves on Thursday, Monday and Tuesday and a deficit on Friday and Wedndsday.
+That produced a small daily average deficit for the week as a whole.
+For the two-week period, he said there were relatively high excess reserves on a daily avearge, almost all of which were at the smallest banks.
+
+American Express Co remained silent on market rumors it would spinoff all or part of its Shearson Lehman Brothers Inc, but some analysts said the company may be considering such a move because it is unhappy with the market value of its stock.
+American Express stock got a lift from the rumor, as the market calculated a partially public Shearson may command a good market value, thereby boosting the total value of American Express.
+The rumor also was accompanied by talk the financial services firm would split its stock and boost its dividend.
+American Express closed on the New York Stock Exchange at 72-5/8, up 4-1/8 on heavy volume.
+American Express would not comment on the rumors or its stock activity.
+Analysts said comments by the company at an analysts' meeting Tuesday helped fuel the rumors as did an announcement yesterday of management changes.
+At the meeting, company officials said American Express stock is undervalued and does not fully reflect the performance of Shearson, according to analysts.
+Yesterday, Shearson said it was elevating its chief operating officer, Jeffery Lane, to the added position of president, which had been vacant.
+It also created four new positions for chairmen of its operating divisions.
+Analysts speculated a partial spinoff would make most sense, contrary to one variation on market rumors of a total spinoff.
+Some analysts, however, disagreed that any spinoff of Shearson would be good since it is a strong profit center for American Express, contributing about 20 pct of earnings last year.
+"I think it is highly unlikely that American Express is going to sell shearson," said Perrin Long of Lipper Analytical.
+He questioned what would be a better investment than "a very profitable securities firm."
+Several analysts said American Express is not in need of cash, which might be the only reason to sell a part of a strong asset.
+But others believe the company could very well of considered the option of spinning out part of Shearson, and one rumor suggests selling about 20 pct of it in the market.
+Larry Eckenfelder of Prudential-Bache Securities said he believes American Express could have considered a partial spinoff in the past.
+"Shearson being as profitable as it is would have fetched a big premium in the market place.
+Shearson's book value is in the 1.4 mln dlr range.
+Shearson in the market place would probably be worth three to 3.5 bilion dlrs in terms of market capitalization," said Eckenfelder.
+Some analysts said American Express could use capital since it plans to expand globally.
+"They have enormous internal growth plans that takes capital.
+You want your stock to reflect realistic valuations to enhance your ability to make all kinds of endeavors down the road," said E.F. Hutton Group analyst Michael Lewis.
+"They've outlined the fact that they're investing heavily in the future, which goes heavily into the international arena," said Lewis.
+"...That does not preclude acquisitions and divestitures along the way," he said.
+Lewis said if American Express reduced its exposure to the brokerage business by selling part of shearson, its stock might better reflect other assets, such as the travel related services business.
+"It could find its true water mark with a lesser exposure to brokerage.
+The value of the other components could command a higher multiple because they constitute a higher percentage of the total operating earnings of the company," he said.
+Lewis said Shearson contributed 316 mln in after-tax operating earnings, up from about 200 mln dlrs in 1985.
+Reuter &#3;
+
+Coleco Industries Inc said it expects to return to profitability in 1987.
+Earlier, Coleco reported a net loss of 111.2 mln dlrs for the year ended December 31 compared to a profit of 64.2 mln dlrs in the year earlier.
+In a prepared statement, the company said the dramatic swing in operating results was due primarily to the steep decline in sales of Cabbage Patch Kids products from 600 mln dlrs to 230 mln dlrs.
+Coleco said it changed from a single product company to a more diversified organization through four major acquisitions last year.
+Products from the new acquisitions and other new product introductions are expected to enable it to return to profitability, it said.
+At the annual Toy Fair earlier this month, vice president Morton Handel said analysts' 1987 projected earnings of 90 cts a share on sales of 600 mln dlrs are reasonable.
+Venezuela is seeking a 'constructive and flexible' attitude from its creditor banks in current talks to reschedule 21 billion dlrs in foreign debt, finance minister manuel azpurua told a press conference.
+He declined to comment on meetings this week in new york between public finances director jorge marcano and venezuela's 13-bank advisory committee except to say, "they are progressing."
+Azpurua said venezuela has shown solidarity with brazil's decision to suspend payments, but each country must negotiate according to its own interest.
+Asked to comment on chile's agreement with its creditors today, which includes an interest rate margin of one pct over libor, azpurua said only, "that is good news."
+According to banking sources, the banks' latest offer to venezuela is also a one pct margin as against the last february's 1-1/8 pct rescheduling accord and the 7/8 pct Venezuela wants.
+Azpurua said four basic elements are being negotiated with the banks now: spread reduction, deferral of principal payments due in 1987 and 1988, lenghtening the 12-1/2 year repayment schedule, and debt capitalization schemes.
+Azpurua said the governent plans to pay 2.1 billion dlrs in public and private debt principal this year.
+It was due to amortize 1.05 billion dlrs under the rescheduling, and pay 420 mln dlrs in non-restructured principal, both public sector.
+He said venezuela's original proposal was to pay no principal on restructured debt this year, but is now insisting that if it makes payments they be compensated by new bank loans.
+The banking sources said the committee has been prepared to lower amortizations to around 400 mln dlrs this year, but that no direct commitment was likely on new loans.
+"debtors and bank creditors have a joint responsibility and there will be no lasting solution unless a positive flow of financing is guaranteed," azpurua said.
+However, he appeared to discard earlier venezuelan proposals for a direct link between oil income and debt payments, "because circumstances change too quickly."
+At the same time, he said the government is presently studying possible mechanisms for capitlizing public and private sector foreign debt, based on experience in other countries.
+The rules would be published by the finance ministry and the central bank.
+
+Thomson McKinnon Mortgage Assets Corp, a unit of Thomson McKinnon Inc, is offering 100 mln dlrs of collateralized mortgage obligations in three tranches that include floating rate and inverse floating rate CMOS.
+The floating rate class amounts to 60 mln dlrs.
+It has an average life of 7.11 years and matures 2018.
+The CMOs have an initial coupon of 7.0375 pct, which will be reset 60 basis points above LIBOR, said sole manager Thomson McKinnon.
+The inverse floater totals 4.8 mln dlrs.
+It has an average life of 13.49 years and matures 2018.
+These CMOs were given an initial coupon of 11-1/2 pct and priced at 104.40.
+Subsequent rates on the inverse floater will equal 11-1/2 pct minus the product of three times (LIBOR minus 6-1/2 pct).
+A Thomson officer explained that the coupon of the inverse floating rate tranche would increase if LIBOR declined.
+"The yield floats opposite of LIBOR," he said.
+The fixed-rate tranche totals 35.2 mln dlrs.
+It has an average life of 3.5 years and matures 2016.
+The CMOs were assigned a 7.65 pct coupon and par pricing.
+The issue is rated AAA by Standard and Poor's and secured by Federal Home Loan Mortgage Corp, Freddie Mac, certificates.
+
+
+OPEC may be forced to meet before a scheduled June session to readdress its production cutting agreement if the organization wants to halt the current slide in oil prices, oil industry analysts said.
+"The movement to higher oil prices was never to be as easy as OPEC thought.
+They may need an emergency meeting to sort out the problems," said Daniel Yergin, director of Cambridge Energy Research Associates, CERA.
+Analysts and oil industry sources said the problem OPEC faces is excess oil supply in world oil markets.
+"OPEC's problem is not a price problem but a production issue and must be addressed in that way," said Paul Mlotok, oil analyst with Salomon Brothers Inc.
+He said the market's earlier optimism about OPEC and its ability to keep production under control have given way to a pessimistic outlook that the organization must address soon if it wishes to regain the initiative in oil prices.
+But some other analysts were uncertain that even an emergency meeting would address the problem of OPEC production above the 15.8 mln bpd quota set last December.
+"OPEC has to learn that in a buyers market you cannot have deemed quotas, fixed prices and set differentials," said the regional manager for one of the major oil companies who spoke on condition that he not be named.
+"The market is now trying to teach them that lesson again," he added.
+David T. Mizrahi, editor of Mideast reports, expects OPEC to meet before June, although not immediately.
+However, he is not optimistic that OPEC can address its principal problems.
+"They will not meet now as they try to take advantage of the winter demand to sell their oil, but in late March and April when demand slackens," Mizrahi said.
+But Mizrahi said that OPEC is unlikely to do anything more than reiterate its agreement to keep output at 15.8 mln bpd."
+Analysts said that the next two months will be critical for OPEC's ability to hold together prices and output.
+"OPEC must hold to its pact for the next six to eight weeks since buyers will come back into the market then," said Dillard Spriggs of Petroleum Analysis Ltd in New York.
+But Bijan Moussavar-Rahmani of Harvard University's Energy and Environment Policy Center said that the demand for OPEC oil has been rising through the first quarter and this may have prompted excesses in its production.
+"Demand for their (OPEC) oil is clearly above 15.8 mln bpd and is probably closer to 17 mln bpd or higher now so what we are seeing characterized as cheating is OPEC meeting this demand through current production," he told Reuters in a telephone interview.
+
+BankAmerica Corp is not under pressure to act quickly on its proposed equity offering and would do well to delay it because of the stock's recent poor performance, banking analysts said.
+Some analysts said they have recommended BankAmerica delay its up to one-billion-dlr equity offering, which has yet to be approved by the Securities and Exchange Commission.
+BankAmerica stock fell this week, along with other banking issues, on the news that Brazil has suspended interest payments on a large portion of its foreign debt.
+The stock traded around 12, down 1/8, this afternoon, after falling to 11-1/2 earlier this week on the news.
+Banking analysts said that with the immediate threat of the First Interstate Bancorp <I> takeover bid gone, BankAmerica is under no pressure to sell the securities into a market that will be nervous on bank stocks in the near term.
+BankAmerica filed the offer on January 26.
+It was seen as one of the major factors leading the First Interstate withdrawing its takeover bid on February 9.
+A BankAmerica spokesman said SEC approval is taking longer than expected and market conditions must now be re-evaluated.
+"The circumstances at the time will determine what we do," said Arthur Miller, BankAmerica's Vice President for Financial Communications, when asked if BankAmerica would proceed with the offer immediately after it receives SEC approval.
+"I'd put it off as long as they conceivably could," said Lawrence Cohn, analyst with Merrill Lynch, Pierce, Fenner and Smith.
+Cohn said the longer BankAmerica waits, the longer they have to show the market an improved financial outlook.
+Although BankAmerica has yet to specify the types of equities it would offer, most analysts believed a convertible preferred stock would encompass at least part of it.
+Such an offering at a depressed stock price would mean a lower conversion price and more dilution to BankAmerica stock holders, noted Daniel Williams, analyst with Sutro Group.
+Several analysts said that while they believe the Brazilian debt problem will continue to hang over the banking industry through the quarter, the initial shock reaction is likely to ease over the coming weeks.
+Nevertheless, BankAmerica, which holds about 2.70 billion dlrs in Brazilian loans, stands to lose 15-20 mln dlrs if the interest rate is reduced on the debt, and as much as 200 mln dlrs if Brazil pays no interest for a year, said Joseph Arsenio, analyst with Birr, Wilson and Co.
+He noted, however, that any potential losses would not show up in the current quarter.
+
+The Federal Deposit Insurance Corp (FDIC) said three troubled banks in Texas and Louisiana were merged with healthy financial institutions.
+The FDIC said it subsidized the merger of Central Bank and Trust Co, Glenmora, La., with the healthy Peoples Bank and Trust Co, Natchitoches, La., after state regulators notified it that Central was in danger of failing.
+Central had assets of 28.3 mln dlrs.
+The FDIC said the deposits of the failed Farmers State Bank, Hart, Tex., were assumed by Hale County State Bank, Plainview, Tex.
+Farmers, with 9.6 mln dlrs in assets, was closed by Texas bank regulators.
+The deposits of the failed First National Bank of Crosby, Crosby, Tex., with total assets of 8.2 mln dlrs, were assumed by Central Bancshares of the South Inc, Birmingham, Ala., after First National was closed by federal bank regulators, the FDIC said.
+Brazil's 14-bank advisory committee expressed "grave concern" to chief debt negotiator Antonio Padua de Seixas over the country's suspension of interest payments, according to a telex from committee chairman Citibank to creditor banks worldwide.
+Bankers said the diplomatic phrase belied the deep anger and frustration on the committee over Brazil's unilateral move last Friday and its subsequent freeze on some 15 billion dlrs of short-term trade and interbank lines.
+Seixas, director of the Brazilian central bank's foreign debt department, met the full panel on Tuesday and Wednesday.
+Seixas, who met again this morning with senior Citibank executive William Rhodes and representatives from committee vice-chairmen Morgan Guaranty Trust Co and Lloyds Bank Plc, told the banks that the government was preparing a telex to explain and clarify the freeze on short-term credits.
+The telex could be sent to creditors as early as today, bankers said.
+Despite the rising tempers, bankers said there are no plans for Brazilian finance minister Dilson Funaro to meet commercial bankers during his trip to Washington on Friday and Saturday.
+Funaro will be explaining Brazil's actions to U.S. Treasury Secretary James Baker, Federal Reserve Board chairman Paul Volcker and International Monetary Fund managing director Michel Camdessus before travelling to Europe at the weekend.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/analysis/opennlp/src/tools/test-model-data/tokenizer.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/opennlp/src/tools/test-model-data/tokenizer.txt b/lucene/analysis/opennlp/src/tools/test-model-data/tokenizer.txt
new file mode 100644
index 0000000..e301d62
--- /dev/null
+++ b/lucene/analysis/opennlp/src/tools/test-model-data/tokenizer.txt
@@ -0,0 +1,69 @@
+Iran announced tonight that its major offensive against Iraq in the Gulf war had ended after dealing savage blows against the Baghdad government<SPLIT>.
+The Iranian news agency IRNA<SPLIT>, in a report received in London<SPLIT>, said the operation code-named Karbala-5 launched into Iraq on January 9 was now over<SPLIT>.
+It quoted a joint statewment by the Iranian Army and Revolutionary Guards Corps as saying that their forces had "<SPLIT>dealt one of the severest blows on the Iraqi war machine in the history of the Iraq-imposed war<SPLIT>.<SPLIT>"
+The statement by the Iranian High Command appeared to herald the close of an assault on the port city of Basra in southern Iraq<SPLIT>.
+"<SPLIT>The operation was launched at a time when the Baghdad government was spreading extensive propaganda on the resistance power of its army<SPLIT>...<SPLIT>,<SPLIT>" said the statement quoted by IRNA<SPLIT>.
+It claimed massive victories in the seven-week offensive and called on supporters of Baghdad to "<SPLIT>come to their senses<SPLIT>" and discontinue support for what it called the tottering regime in Iraq<SPLIT>.
+Iran said its forces had "<SPLIT>liberated<SPLIT>" 155 square kilometers of enemy-occupied territory during the 1987 offensive and taken over islands<SPLIT>, townships<SPLIT>, rivers and part of a road leading into Basra<SPLIT>.
+The Iranian forces "<SPLIT>are in full control of these areas<SPLIT>,<SPLIT>" the statement said<SPLIT>.
+It said 81 Iraqi brigades and battalions were totally destroyed<SPLIT>, along with 700 tanks and 1,500 other vehicles<SPLIT>.
+
+U.S. bank discount window borrowings less extended credits averaged 310 mln dlrs in the week to Wednesday February 25<SPLIT>, the Federal Reserve said<SPLIT>.
+The Fed said that overall borrowings in the week fell 131 mln dlrs to 614 mln dlrs<SPLIT>, with extended credits up 10 mln dlrs at 304 mln dlrs<SPLIT>.
+The week was the second half of a two-week statement period<SPLIT>.
+Net borrowings in the prior week averaged 451 mln dlrs<SPLIT>.
+Commenting on the two-week statement period ended February 25<SPLIT>, the Fed said that banks had average net free reserves of 644 mln dlrs a day<SPLIT>, down from 1.34 billion two weeks earlier<SPLIT>.
+A Federal Reserve spokesman told a press briefing that there were no large single day net misses in the Fed's reserve projections in the week to Wednesday<SPLIT>.
+He said that natural float had been "<SPLIT>acting a bit strangely<SPLIT>" for this time of year<SPLIT>, noting that there had been poor weather during the latest week<SPLIT>.
+The spokesman said that natural float ranged from under 500 mln dlrs on Friday<SPLIT>, for which he could give no reason<SPLIT>, to nearly one billion dlrs on both Thursday and Wednesday<SPLIT>.
+The Fed spokeman could give no reason for Thursday's high float<SPLIT>, but he said that about 750 mln dlrs of Wednesday's float figure was due to holdover and transportation float at two widely separated Fed districts<SPLIT>.
+For the week as a whole<SPLIT>, he said that float related as of adjustments were "<SPLIT>small<SPLIT>,<SPLIT>" adding that they fell to a negative 750 mln dlrs on Tuesday due to a number of corrections for unrelated cash letter errors in six districts around the country<SPLIT>.
+The spokesman said that on both Tuesday and Wednesday<SPLIT>, two different clearing banks had system problems and the securities and Federal funds wires had to be held open until about 2000 or 2100 EST on both days<SPLIT>.
+However<SPLIT>, he said that both problems were cleared up during both afternoons and there was no evidence of any reserve impact<SPLIT>.
+During the week ended Wednesday<SPLIT>, 45 pct of net discount window borrowings were made by the smallest banks<SPLIT>, with 30 pct by the 14 large money center banks and 25 pct by large regional institutions<SPLIT>.
+On Wednesday<SPLIT>, 55 pct of the borrowing was accounted for by the money center banks<SPLIT>, with 30 pct by the large regionals and 15 pct by the smallest banks<SPLIT>.
+The Fed spokesman said the banking system had excess reserves on Thursday<SPLIT>, Monday and Tuesday and a deficit on Friday and Wedndsday<SPLIT>.
+That produced a small daily average deficit for the week as a whole<SPLIT>.
+For the two-week period<SPLIT>, he said there were relatively high excess reserves on a daily avearge<SPLIT>, almost all of which were at the smallest banks<SPLIT>.
+American Express Co remained silent on market rumors it would spinoff all or part of its Shearson Lehman Brothers Inc<SPLIT>, but some analysts said the company may be considering such a move because it is unhappy with the market value of its stock<SPLIT>.
+American Express stock got a lift from the rumor<SPLIT>, as the market calculated a partially public Shearson may command a good market value<SPLIT>, thereby boosting the total value of American Express<SPLIT>.
+The rumor also was accompanied by talk the financial services firm would split its stock and boost its dividend<SPLIT>.
+American Express closed on the New York Stock Exchange at 72-5/8<SPLIT>, up 4-1/8 on heavy volume<SPLIT>.
+American Express would not comment on the rumors or its stock activity<SPLIT>.
+Analysts said comments by the company at an analysts' meeting Tuesday helped fuel the rumors as did an announcement yesterday of management changes<SPLIT>.
+At the meeting<SPLIT>, company officials said American Express stock is undervalued and does not fully reflect the performance of Shearson<SPLIT>, according to analysts<SPLIT>.
+Yesterday<SPLIT>, Shearson said it was elevating its chief operating officer<SPLIT>, Jeffery Lane<SPLIT>, to the added position of president<SPLIT>, which had been vacant<SPLIT>.
+It also created four new positions for chairmen of its operating divisions<SPLIT>.
+Analysts speculated a partial spinoff would make most sense<SPLIT>, contrary to one variation on market rumors of a total spinoff<SPLIT>.
+Some analysts<SPLIT>, however<SPLIT>, disagreed that any spinoff of Shearson would be good since it is a strong profit center for American Express<SPLIT>, contributing about 20 pct of earnings last year<SPLIT>.
+"<SPLIT>I think it is highly unlikely that American Express is going to sell shearson<SPLIT>,<SPLIT>" said Perrin Long of Lipper Analytical<SPLIT>.
+He questioned what would be a better investment than "<SPLIT>a very profitable securities firm<SPLIT>.<SPLIT>"
+Several analysts said American Express is not in need of cash<SPLIT>, which might be the only reason to sell a part of a strong asset<SPLIT>.
+But others believe the company could very well of considered the option of spinning out part of Shearson<SPLIT>, and one rumor suggests selling about 20 pct of it in the market<SPLIT>.
+Larry Eckenfelder of Prudential-Bache Securities said he believes American Express could have considered a partial spinoff in the past<SPLIT>.
+"<SPLIT>Shearson being as profitable as it is would have fetched a big premium in the market place<SPLIT>.
+Some analysts said American Express could use capital since it plans to expand globally<SPLIT>.
+"<SPLIT>They've outlined the fact that they're investing heavily in the future<SPLIT>, which goes heavily into the international arena<SPLIT>,<SPLIT>" said Lewis<SPLIT>.
+Lewis said if American Express reduced its exposure to the brokerage business by selling part of shearson<SPLIT>, its stock might better reflect other assets<SPLIT>, such as the travel related services business<SPLIT>.
+Lewis said Shearson contributed 316 mln in after-tax operating earnings<SPLIT>, up from about 200 mln dlrs in 1985<SPLIT>.
+Coleco Industries Inc said it expects to return to profitability in 1987<SPLIT>.
+Earlier<SPLIT>, Coleco reported a net loss of 111.2 mln dlrs for the year ended December 31 compared to a profit of 64.2 mln dlrs in the year earlier<SPLIT>.
+In a prepared statement<SPLIT>, the company said the dramatic swing in operating results was due primarily to the steep decline in sales of Cabbage Patch Kids products from 600 mln dlrs to 230 mln dlrs<SPLIT>.
+Coleco said it changed from a single product company to a more diversified organization through four major acquisitions last year<SPLIT>.
+Products from the new acquisitions and other new product introductions are expected to enable it to return to profitability<SPLIT>, it said<SPLIT>.
+At the annual Toy Fair earlier this month<SPLIT>, vice president Morton Handel said analysts' 1987 projected earnings of 90 cts a share on sales of 600 mln dlrs are reasonable<SPLIT>.
+Azpurua said venezuela has shown solidarity with brazil's decision to suspend payments<SPLIT>, but each country must negotiate according to its own interest<SPLIT>.
+Azpurua said the governent plans to pay 2.1 billion dlrs in public and private debt principal this year<SPLIT>.
+It was due to amortize 1.05 billion dlrs under the rescheduling<SPLIT>, and pay 420 mln dlrs in non-restructured principal<SPLIT>, both public sector<SPLIT>.
+He said venezuela's original proposal was to pay no principal on restructured debt this year<SPLIT>, but is now insisting that if it makes payments they be compensated by new bank loans<SPLIT>.
+The banking sources said the committee has been prepared to lower amortizations to around 400 mln dlrs this year<SPLIT>, but that no direct commitment was likely on new loans<SPLIT>.
+At the same time<SPLIT>, he said the government is presently studying possible mechanisms for capitlizing public and private sector foreign debt<SPLIT>, based on experience in other countries<SPLIT>.
+The rules would be published by the finance ministry and the central bank<SPLIT>.
+
+Thomson McKinnon Mortgage Assets Corp<SPLIT>, a unit of Thomson McKinnon Inc<SPLIT>, is offering 100 mln dlrs of collateralized mortgage obligations in three tranches that include floating rate and inverse floating rate CMOS<SPLIT>.
+The floating rate class amounts to 60 mln dlrs<SPLIT>.
+The inverse floater totals 4.8 mln dlrs<SPLIT>.
+Subsequent rates on the inverse floater will equal 11-1/2 pct minus the product of three times (<SPLIT>LIBOR minus 6-1/2 pct<SPLIT>)<SPLIT>.
+A Thomson officer explained that the coupon of the inverse floating rate tranche would increase if LIBOR declined<SPLIT>.
+The fixed-rate tranche totals 35.2 mln dlrs<SPLIT>.
+The issue is rated AAA by Standard and Poor's and secured by Federal Home Loan Mortgage Corp<SPLIT>, Freddie Mac<SPLIT>, certificates<SPLIT>.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/core/src/test/org/apache/lucene/analysis/TestStopFilter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/analysis/TestStopFilter.java b/lucene/core/src/test/org/apache/lucene/analysis/TestStopFilter.java
index 3e26965..f17cd51 100644
--- a/lucene/core/src/test/org/apache/lucene/analysis/TestStopFilter.java
+++ b/lucene/core/src/test/org/apache/lucene/analysis/TestStopFilter.java
@@ -20,12 +20,8 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.util.ArrayList;
 
-import org.apache.lucene.analysis.BaseTokenStreamTestCase;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.util.English;
 
 public class TestStopFilter extends BaseTokenStreamTestCase {
@@ -111,9 +107,10 @@ public class TestStopFilter extends BaseTokenStreamTestCase {
                               7,
                               1,
                               null,
-                              true);    
+                              true,
+                              null);
   }
-  
+
   private void doTestStopPositons(StopFilter stpf) throws IOException {
     CharTermAttribute termAtt = stpf.getAttribute(CharTermAttribute.class);
     PositionIncrementAttribute posIncrAtt = stpf.getAttribute(PositionIncrementAttribute.class);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 2478f85..35df7ae 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -161,6 +161,9 @@ org.apache.james.apache.mime4j.version = 0.7.2
 
 /org.apache.mina/mina-core = 2.0.0-M5
 
+/org.apache.opennlp/opennlp-maxent = 3.0.3
+/org.apache.opennlp/opennlp-tools = 1.8.3
+
 org.apache.pdfbox.version = 2.0.6
 /org.apache.pdfbox/fontbox = ${org.apache.pdfbox.version}
 /org.apache.pdfbox/jempbox = 1.8.13

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/licenses/opennlp-maxent-3.0.3.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/opennlp-maxent-3.0.3.jar.sha1 b/lucene/licenses/opennlp-maxent-3.0.3.jar.sha1
new file mode 100644
index 0000000..c3c412f
--- /dev/null
+++ b/lucene/licenses/opennlp-maxent-3.0.3.jar.sha1
@@ -0,0 +1 @@
+55e39e6b46e71f35229cdd6950e72d8cce3b5fd4

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/licenses/opennlp-maxent-NOTICE.txt
----------------------------------------------------------------------
diff --git a/lucene/licenses/opennlp-maxent-NOTICE.txt b/lucene/licenses/opennlp-maxent-NOTICE.txt
new file mode 100644
index 0000000..9b97287
--- /dev/null
+++ b/lucene/licenses/opennlp-maxent-NOTICE.txt
@@ -0,0 +1,6 @@
+
+Apache OpenNLP Maxent
+Copyright 2013 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/lucene/licenses/opennlp-tools-1.8.3.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/opennlp-tools-1.8.3.jar.sha1 b/lucene/licenses/opennlp-tools-1.8.3.jar.sha1
new file mode 100644
index 0000000..c6a7549
--- /dev/null
+++ b/lucene/licenses/opennlp-tools-1.8.3.jar.sha1
@@ -0,0 +1 @@
+3ce7c9056048f55478d983248cf18c7e02b1d072


[18/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Fix hyperlink

Posted by da...@apache.org.
LUCENE-2899: Fix hyperlink


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

Branch: refs/heads/jira/solr-11702
Commit: f5c4276163d1211f33dc0f27e947e7dc78aa0444
Parents: 3e2f9e6
Author: Steve Rowe <sa...@apache.org>
Authored: Fri Dec 15 11:41:03 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Dec 15 11:41:15 2017 -0500

----------------------------------------------------------------------
 .../OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5c42761/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
index d00df2b..45bb839 100644
--- a/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
+++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
@@ -74,7 +74,7 @@ import org.slf4j.LoggerFactory;
  *   &lt;/fieldType&gt;
  * </pre>
  * 
- * <p>See the <a href="OpenNLP website">http://opennlp.apache.org/models.html</a>
+ * <p>See the <a href="http://opennlp.apache.org/models.html">OpenNLP website"></a>
  * for information on downloading pre-trained models.</p>
  *
  * <p>


[49/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11703: Solr Should Send Log Notifications if Ulimits are too low

Posted by da...@apache.org.
SOLR-11703: Solr Should Send Log Notifications if Ulimits are too low


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

Branch: refs/heads/jira/solr-11702
Commit: e82e029406b21ccbfebfd36141c2a318f7326453
Parents: 7e321d7
Author: Erick Erickson <er...@apache.org>
Authored: Mon Dec 25 13:22:19 2017 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Mon Dec 25 13:22:19 2017 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt    |  2 ++
 solr/bin/solr       | 34 ++++++++++++++++++++++++++++++++++
 solr/bin/solr.in.sh | 11 +++++++++++
 3 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82e0294/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f98d71d..1d4ac2a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -81,6 +81,8 @@ Other Changes
 
 * SOLR-11701: Upgrade to Tika 1.17 when available (Tim Allison, Karthik Ramachandran via Erick Erickson)
 
+* SOLR-11703: Solr Should Send Log Notifications if Ulimits are too low (Kevin Cowan via Erick Eickson)
+
 ==================  7.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82e0294/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index f9814c2..ef83cee 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -1414,6 +1414,40 @@ if [ "$SCRIPT_CMD" != "stop" ] && [ "$SCRIPT_CMD" != "start" ] && [ "$SCRIPT_CMD
   exit 1
 fi
 
+#Check current Ulimits for Open Files and Max Processes.  Warn if they are below the recommended values.
+
+if [ -z "$SOLR_RECOMMENDED_MAX_PROCESSES" ]; then
+  SOLR_RECOMMENDED_MAX_PROCESSES=65000
+fi
+
+if [ -z "$SOLR_RECOMMENDED_OPEN_FILES" ]; then
+  SOLR_RECOMMENDED_OPEN_FILES=65000
+fi
+
+if [ -z "$SOLR_ULIMIT_CHECKS" ] || [ "$SOLR_ULIMIT_CHECKS" != "false" ]; then
+  if [ "$SCRIPT_CMD" == "start" ] || [ "$SCRIPT_CMD" == "restart" ] || [ "$SCRIPT_CMD" == "status" ]; then
+    if hash ulimit 2>/dev/null; then
+       openFiles=$(ulimit -n)
+       maxProcs=$(ulimit -u)
+       if [ $openFiles -lt "$SOLR_RECOMMENDED_OPEN_FILES" ]; then
+           echo "*** [WARN] *** Your open file limit is currently $openFiles.  "
+           echo " It should be set to $SOLR_RECOMMENDED_OPEN_FILES to avoid operational impariment. "
+           echo " If you no longer wish to see this warning, set SOLR_ULIMIT_CHECKS to false in your profile or solr.in.sh"
+       fi
+
+       if [ $maxProcs -lt "$SOLR_RECOMMENDED_MAX_PROCESSES" ]; then
+           echo "*** [WARN] ***  Your Max Processes Limit is currently $maxProcs. "
+           echo " It should be set to $SOLR_RECOMMENDED_MAX_PROCESSES to avoid operational impariment. "
+           echo " If you no longer wish to see this warning, set SOLR_ULIMIT_CHECKS to false in your profile or solr.in.sh"
+       fi
+    else
+      echo "Could not check ulimits for processes and open files, recommended values are"
+      echo "     max processes: $SOLR_RECOMMENDED_MAX_PROCESSES "
+      echo "     open files:    $SOLR_RECOMMENDED_OPEN_FILES"
+    fi
+  fi
+fi
+
 # Run in foreground (default is to run in the background)
 FG="false"
 FORCE=false

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82e0294/solr/bin/solr.in.sh
----------------------------------------------------------------------
diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh
index 10d8b90..e7478cd 100644
--- a/solr/bin/solr.in.sh
+++ b/solr/bin/solr.in.sh
@@ -149,3 +149,14 @@
 #  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD"
 #SOLR_OPTS="$SOLR_OPTS $SOLR_ZK_CREDS_AND_ACLS"
 
+
+# Settings for common system values that may cause operational imparement when system defaults are used.
+# Solr can use many processes and many file handles. On modern operating systems the savings by leaving
+# these settings low is minuscule, while the consequence can be Solr instability. To turn these checks off, set
+# SOLR_ULIMIT_CHECKS=false either here or as part of your profile.
+
+# Different limits can be set in solr.in.sh or your profile if you prefer as well.
+#SOLR_RECOMMENDED_OPEN_FILES=
+#SOLR_RECOMMENDED_MAX_PROCESSES=
+#SOLR_ULIMIT_CHECKS=
+


[12/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Add OpenNLP Analysis capabilities as a module

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/licenses/opennlp-maxent-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/opennlp-maxent-NOTICE.txt b/solr/licenses/opennlp-maxent-NOTICE.txt
new file mode 100644
index 0000000..9b97287
--- /dev/null
+++ b/solr/licenses/opennlp-maxent-NOTICE.txt
@@ -0,0 +1,6 @@
+
+Apache OpenNLP Maxent
+Copyright 2013 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/licenses/opennlp-tools-1.8.3.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/opennlp-tools-1.8.3.jar.sha1 b/solr/licenses/opennlp-tools-1.8.3.jar.sha1
new file mode 100644
index 0000000..c6a7549
--- /dev/null
+++ b/solr/licenses/opennlp-tools-1.8.3.jar.sha1
@@ -0,0 +1 @@
+3ce7c9056048f55478d983248cf18c7e02b1d072

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/licenses/opennlp-tools-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/opennlp-tools-NOTICE.txt b/solr/licenses/opennlp-tools-NOTICE.txt
new file mode 100644
index 0000000..68a08dc
--- /dev/null
+++ b/solr/licenses/opennlp-tools-NOTICE.txt
@@ -0,0 +1,6 @@
+
+Apache OpenNLP Tools
+Copyright 2015 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/solr-ref-guide/src/filter-descriptions.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/filter-descriptions.adoc b/solr/solr-ref-guide/src/filter-descriptions.adoc
index b8ab6de..09dd0f6 100644
--- a/solr/solr-ref-guide/src/filter-descriptions.adoc
+++ b/solr/solr-ref-guide/src/filter-descriptions.adoc
@@ -1576,6 +1576,38 @@ This filter adds the token's type, as an encoded byte sequence, as its payload.
 
 *Out:* "Pay"[<ALPHANUM>], "Bob's"[<APOSTROPHE>], "I.O.U."[<ACRONYM>]
 
+== Type As Synonym Filter
+
+This filter adds the token's type, as a token at the same position as the token, optionally with a configurable prefix prepended.
+
+*Factory class:* `solr.TypeAsSynonymFilterFactory`
+
+*Arguments:*
+
+`prefix`:: (optional) The prefix to prepend to the token's type.
+
+*Examples:*
+
+With the example below, each token's type will be emitted verbatim at the same position:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.StandardTokenizerFactory"/>
+  <filter class="solr.TypeAsSynonymFilterFactory"/>
+</analyzer>
+----
+
+With the example below, for a token "example.com" with type `<URL>`, the token emitted at the same position will be "\_type_<URL>":
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.UAX29URLEmailTokenizerFactory"/>
+  <filter class="solr.TypeAsSynonymFilterFactory" prefix="_type_"/>
+</analyzer>
+----
+
 == Type Token Filter
 
 This filter blacklists or whitelists a specified list of token types, assuming the tokens have type metadata associated with them. For example, the <<tokenizers.adoc#uax29-url-email-tokenizer,UAX29 URL Email Tokenizer>> emits "<URL>" and "<EMAIL>" typed tokens, as well as other types. This filter would allow you to pull out only e-mail addresses from text as tokens, if you wish.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/solr-ref-guide/src/language-analysis.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/language-analysis.adoc b/solr/solr-ref-guide/src/language-analysis.adoc
index 8e6a214..9d00325 100644
--- a/solr/solr-ref-guide/src/language-analysis.adoc
+++ b/solr/solr-ref-guide/src/language-analysis.adoc
@@ -355,6 +355,214 @@ This can increase recall by causing more matches. On the other hand, it can redu
 </analyzer>
 ----
 
+== OpenNLP Integration
+
+The `lucene/analysis/opennlp` module provides OpenNLP integration via several analysis components: a tokenizer, a part-of-speech tagging filter, a phrase chunking filter, and a lemmatization filter.  In addition to these analysis components, Solr also provides an update request processor to extract named entities - see <<update-request-processors.adoc#update-processor-factories-that-can-be-loaded-as-plugins,Update Processor Factories That Can Be Loaded as Plugins>>.
+
+NOTE: The <<OpenNLP Tokenizer>> must be used with all other OpenNLP analysis components, for two reasons: first, the OpenNLP Tokenizer detects and marks the sentence boundaries required by all the OpenNLP filters; and second, since the pre-trained OpenNLP models used by these filters were trained using the corresponding language-specific sentence-detection/tokenization models, the same tokenization, using the same models, must be used at runtime for optimal performance.
+
+See `solr/contrib/analysis-extras/README.txt` for information on which jars you need to add to your `SOLR_HOME/lib`.
+
+=== OpenNLP Tokenizer
+
+The OpenNLP Tokenizer takes two language-specific binary model files as parameters: a sentence detector model and a tokenizer model.  The last token in each sentence is flagged, so that following OpenNLP-based filters can use this information to apply operations to tokens one sentence at a time. See the http://opennlp.apache.org/models.html[OpenNLP website] for information on downloading pre-trained models.
+
+*Factory class:* `solr.OpenNLPTokenizerFactory`
+
+*Arguments:*
+
+`sentenceModel`:: (required) The path of a language-specific OpenNLP sentence detection model file. This path may be an absolute path, or path relative to the Solr config directory.
+
+`tokenizerModel`:: (required) The path of a language-specific OpenNLP tokenization model file. This path may be an absolute path, or path relative to the Solr config directory.
+
+*Example:*
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+</analyzer>
+----
+
+=== OpenNLP Part-Of-Speech Filter
+
+This filter sets each token's type attribute to the part of speech (POS) assigned by the configured model. See the http://opennlp.apache.org/models.html[OpenNLP website] for information on downloading pre-trained models.
+
+NOTE: Lucene currently does not index token types, so if you want to keep this information, you have to preserve it either in a payload or as a synonym; see the examples below.
+
+*Factory class:* `solr.OpenNLPPOSFilterFactory`
+
+*Arguments:*
+
+`posTaggerModel`:: (required) The path of a language-specific OpenNLP POS tagger model file. This path may be an absolute path, or path relative to the Solr config directory.
+
+*Examples:*
+
+The OpenNLP tokenizer will tokenize punctuation, which is useful for following token filters, but ordinarily you don't want to include punctuation in your index, so the `TypeTokenFilter` (<<filter-descriptions.adoc#type-token-filter,described here>>) is included in the examples below, with `stop.pos.txt` containing the following:
+
+.stop.pos.txt
+[source,text]
+----
+#
+$
+''
+``
+,
+-LRB-
+-RRB-
+:
+.
+----
+
+Index the POS for each token as a payload:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.TypeAsPayloadFilterFactory"/>
+  <filter class="solr.TypeTokenFilterFactory" types="stop.pos.txt"/>
+</analyzer>
+----
+
+Index the POS for each token as a synonym, after prefixing the POS with "@" (see the <<filter-descriptions.adoc#type-as-synonym-filter,TypeAsSynonymFilter description>>):
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.TypeAsSynonymFilterFactory" prefix="@"/>
+  <filter class="solr.TypeTokenFilterFactory" types="stop.pos.txt"/>
+</analyzer>
+----
+
+Only index nouns - the `keep.pos.txt` file contains lines `NN`, `NNS`, `NNP` and `NNPS`:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.TypeTokenFilterFactory" types="keep.pos.txt" useWhitelist="true"/>
+</analyzer>
+----
+
+=== OpenNLP Phrase Chunking Filter
+
+This filter sets each token's type attribute based on the output of an OpenNLP phrase chunking model.  The chunk labels replace the POS tags that previously were in each token's type attribute. See the http://opennlp.apache.org/models.html[OpenNLP website] for information on downloading pre-trained models.
+
+Prerequisite: the <<OpenNLP Tokenizer>> and the <<OpenNLP Part-Of-Speech Filter>> must precede this filter.
+
+NOTE: Lucene currently does not index token types, so if you want to keep this information, you have to preserve it either in a payload or as a synonym; see the examples below.
+
+*Factory class:* `solr.OpenNLPChunkerFilter`
+
+*Arguments:*
+
+`chunkerModel`:: (required) The path of a language-specific OpenNLP phrase chunker model file. This path may be an absolute path, or path relative to the Solr config directory.
+
+*Examples*:
+
+Index the phrase chunk label for each token as a payload:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.OpenNLPChunkerFactory" chunkerModel="en-chunker.bin"/>
+  <filter class="solr.TypeAsPayloadFilterFactory"/>
+</analyzer>
+----
+
+Index the phrase chunk label for each token as a synonym, after prefixing it with "#" (see the <<filter-descriptions.adoc#type-as-synonym-filter,TypeAsSynonymFilter description>>):
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.OpenNLPChunkerFactory" chunkerModel="en-chunker.bin"/>
+  <filter class="solr.TypeAsSynonymFilterFactory" prefix="#"/>
+</analyzer>
+----
+
+=== OpenNLP Lemmatizer Filter
+
+This filter replaces the text of each token with its lemma. Both a dictionary-based lemmatizer and a model-based lemmatizer are supported. If both are configured, the dictionary-based lemmatizer is tried first, and then the model-based lemmatizer is consulted for out-of-vocabulary tokens. See the http://opennlp.apache.org/models.html[OpenNLP website] for information on downloading pre-trained models.
+
+*Factory class:* `solr.OpenNLPLemmatizerFilter`
+
+*Arguments:*
+
+Either `dictionary` or `lemmatizerModel` must be provided, and both may be provided - see the examples below:
+
+`dictionary`:: (optional) The path of a lemmatization dictionary file. This path may be an absolute path, or path relative to the Solr config directory. The dictionary file must be encoded as UTF-8, with one entry per line, in the form `word[tab]lemma[tab]part-of-speech`, e.g. `wrote[tab]write[tab]VBD`.
+
+`lemmatizerModel`:: (optional) The path of a language-specific OpenNLP lemmatizer model file. This path may be an absolute path, or path relative to the Solr config directory.
+
+*Examples:*
+
+Perform dictionary-based lemmatization, and fall back to model-based lemmatization for out-of-vocabulary tokens (see the <<OpenNLP Part-Of-Speech Filter>> section above for information about using `TypeTokenFilter` to avoid indexing punctuation):
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.OpenNLPLemmatizerFilterFactory"
+          dictionary="lemmas.txt"
+          lemmatizerModel="en-lemmatizer.bin"/>
+  <filter class="solr.TypeTokenFilterFactory" types="stop.pos.txt"/>
+</analyzer>
+----
+
+Perform dictionary-based lemmatization only:
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.OpenNLPLemmatizerFilterFactory" dictionary="lemmas.txt"/>
+  <filter class="solr.TypeTokenFilterFactory" types="stop.pos.txt"/>
+</analyzer>
+----
+
+Perform model-based lemmatization only, preserving the original token and emitting the lemma as a synonym (see the <<KeywordRepeatFilterFactory,KeywordRepeatFilterFactory description>>)):
+
+[source,xml]
+----
+<analyzer>
+  <tokenizer class="solr.OpenNLPTokenizerFactory"
+             sentenceModel="en-sent.bin"
+             tokenizerModel="en-tokenizer.bin"/>
+  <filter class="solr.OpenNLPPOSFilterFactory" posTaggerModel="en-pos-maxent.bin"/>
+  <filter class="solr.KeywordRepeatFilterFactory"/>
+  <filter class="solr.OpenNLPLemmatizerFilterFactory" lemmatizerModel="en-lemmatizer.bin"/>
+  <filter class="solr.RemoveDuplicatesTokenFilterFactory"/>
+  <filter class="solr.TypeTokenFilterFactory" types="stop.pos.txt"/>
+</analyzer>
+----
+
 == Language-Specific Factories
 
 These factories are each designed to work with specific languages. The languages covered here are:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/solr-ref-guide/src/tokenizers.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/tokenizers.adoc b/solr/solr-ref-guide/src/tokenizers.adoc
index 89dad34..c2fcf60 100644
--- a/solr/solr-ref-guide/src/tokenizers.adoc
+++ b/solr/solr-ref-guide/src/tokenizers.adoc
@@ -502,3 +502,7 @@ Specifies how to define whitespace for the purpose of tokenization. Valid values
 *In:* "To be, or what?"
 
 *Out:* "To", "be,", "or", "what?"
+
+== OpenNLP Tokenizer and OpenNLP Filters
+
+See <<language-analysis.adoc#opennlp-integration,OpenNLP Integration>> for information about using the OpenNLP Tokenizer, along with information about available OpenNLP token filters.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/solr-ref-guide/src/update-request-processors.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/update-request-processors.adoc b/solr/solr-ref-guide/src/update-request-processors.adoc
index 09c49d1..921677a 100644
--- a/solr/solr-ref-guide/src/update-request-processors.adoc
+++ b/solr/solr-ref-guide/src/update-request-processors.adoc
@@ -275,6 +275,8 @@ What follows are brief descriptions of the currently available update request pr
 
 {solr-javadocs}/solr-core/org/apache/solr/update/processor/IgnoreCommitOptimizeUpdateProcessorFactory.html[IgnoreCommitOptimizeUpdateProcessorFactory]:: Allows you to ignore commit and/or optimize requests from client applications when running in SolrCloud mode, for more information, see: Shards and Indexing Data in SolrCloud
 
+{solr-javadocs}/solr-core/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.html[CloneFieldUpdateProcessorFactory]:: Clones the values found in any matching _source_ field into the configured _dest_ field.
+
 {solr-javadocs}/solr-core/org/apache/solr/update/processor/RegexpBoostProcessorFactory.html[RegexpBoostProcessorFactory]:: A processor which will match content of "inputField" against regular expressions found in "boostFilename", and if it matches will return the corresponding boost value from the file and output this to "boostField" as a double value.
 
 {solr-javadocs}/solr-core/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.html[SignatureUpdateProcessorFactory]:: Uses a defined set of fields to generate a hash "signature" for the document. Useful for only indexing one copy of "similar" documents.
@@ -351,6 +353,10 @@ The {solr-javadocs}/solr-uima/index.html[`uima`] contrib provides::
 
 {solr-javadocs}/solr-uima/org/apache/solr/uima/processor/UIMAUpdateRequestProcessorFactory.html[UIMAUpdateRequestProcessorFactory]::: Update document(s) to be indexed with UIMA extracted information.
 
+The {solr-javadocs}/solr-analysis-extras/index.html[`analysis-extras`] contrib provides::
+
+{solr-javadocs}/solr-analysis-extras/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesProcessorFactory.html[OpenNLPExtractNamedEntitiesProcessorFactory]::: Update document(s) to be indexed with named entities extracted using an OpenNLP NER model.
+
 === Update Processor Factories You Should _Not_ Modify or Remove
 
 These are listed for completeness, but are part of the Solr infrastructure, particularly SolrCloud. Other than insuring you do _not_ remove them when modifying the update request handlers (or any copies you make), you will rarely, if ever, need to change these.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3e2f9e62/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java b/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java
new file mode 100644
index 0000000..d3aa979
--- /dev/null
+++ b/solr/test-framework/src/java/org/apache/solr/update/processor/UpdateProcessorTestBase.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.update.processor;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+
+import java.io.IOException;
+
+public class UpdateProcessorTestBase extends SolrTestCaseJ4 {
+
+  /**
+   * Runs a document through the specified chain, and returns the final
+   * document used when the chain is completed (NOTE: some chains may
+   * modify the document in place
+   */
+  protected SolrInputDocument processAdd(final String chain,
+                                         final SolrInputDocument docIn)
+    throws IOException {
+
+    return processAdd(chain, new ModifiableSolrParams(), docIn);
+  }
+
+  /**
+   * Runs a document through the specified chain, and returns the final
+   * document used when the chain is completed (NOTE: some chains may
+   * modify the document in place
+   */
+  protected SolrInputDocument processAdd(final String chain,
+                                         final SolrParams requestParams,
+                                         final SolrInputDocument docIn)
+    throws IOException {
+
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, requestParams);
+    try {
+      SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
+      AddUpdateCommand cmd = new AddUpdateCommand(req);
+      cmd.solrDoc = docIn;
+
+      UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+      if (null != processor) {
+        // test chain might be empty or short circuited.
+        processor.processAdd(cmd);
+      }
+
+      return cmd.solrDoc;
+    } finally {
+      SolrRequestInfo.clearRequestInfo();
+      req.close();
+    }
+  }
+
+  protected void processCommit(final String chain) throws IOException {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+
+    CommitUpdateCommand cmd = new CommitUpdateCommand(req,false);
+    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+    try {
+      processor.processCommit(cmd);
+    } finally {
+      req.close();
+    }
+  }
+
+  protected void processDeleteById(final String chain, String id) throws IOException {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+
+    DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
+    cmd.setId(id);
+    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+    try {
+      processor.processDelete(cmd);
+    } finally {
+      req.close();
+    }
+  }
+
+  protected void finish(final String chain) throws IOException {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+
+    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+    try {
+      processor.finish();
+    } finally {
+      IOUtils.closeQuietly(processor);
+      req.close();
+    }
+  }
+
+
+  /**
+   * Convenience method for building up SolrInputDocuments
+   */
+  final SolrInputDocument doc(SolrInputField... fields) {
+    SolrInputDocument d = new SolrInputDocument();
+    for (SolrInputField f : fields) {
+      d.put(f.getName(), f);
+    }
+    return d;
+  }
+
+  /**
+   * Convenience method for building up SolrInputFields
+   */
+  final SolrInputField field(String name, Object... values) {
+    SolrInputField f = new SolrInputField(name);
+    for (Object v : values) {
+      f.addValue(v);
+    }
+    return f;
+  }
+
+  /**
+   * Convenience method for building up SolrInputFields with default boost
+   */
+  final SolrInputField f(String name, Object... values) {
+    return field(name, values);
+  }
+}


[54/54] [abbrv] lucene-solr:jira/solr-11702: SOLR-11702: Fix precommit

Posted by da...@apache.org.
SOLR-11702: Fix precommit


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

Branch: refs/heads/jira/solr-11702
Commit: a7b32b0800d2455092642cc6ebf7da3a2d27da4a
Parents: 2352449
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed Dec 27 22:03:30 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed Dec 27 22:03:30 2017 +0700

----------------------------------------------------------------------
 .../java/org/apache/solr/cloud/CreateCollectionCmd.java  | 11 ++++++++---
 .../src/test/org/apache/solr/cloud/ZkShardTermsTest.java |  7 ++++---
 2 files changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a7b32b08/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index 543f1e2..b63e91d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
 import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
@@ -399,10 +400,14 @@ public class CreateCollectionCmd implements Cmd {
     String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
     String termsPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/terms";
     try {
-      if (zkClient.exists(termsPath, true)) {
-        zkClient.clean(termsPath);
+      if (stateManager.hasData(termsPath)) {
+        List<String> paths = stateManager.listData(termsPath);
+        for (String path : paths) {
+          stateManager.removeData(termsPath + "/" + path, -1);
+        }
+        stateManager.removeData(termsPath, -1);
       }
-    } catch (KeeperException | InterruptedException e) {
+    } catch (KeeperException | InterruptedException | IOException | BadVersionException e) {
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error deleting old term nodes for collection from Zookeeper", e);
     }
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a7b32b08/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
index 83f3fb0..074bc9e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
@@ -33,6 +33,7 @@ import java.util.function.Supplier;
 
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -101,7 +102,7 @@ public class ZkShardTermsTest extends SolrCloudTestCase {
     expectedTerms.put("rep1", 1L);
     expectedTerms.put("rep2", 1L);
 
-    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, new TimeSource.CurrentTimeSource());
     while (!timeOut.hasTimedOut()) {
       if (Objects.equals(expectedTerms, rep1Terms.getTerms()) && Objects.equals(expectedTerms, rep2Terms.getTerms())) break;
     }
@@ -148,7 +149,7 @@ public class ZkShardTermsTest extends SolrCloudTestCase {
     long maxTerm = 0;
     try (ZkShardTerms shardTerms = new ZkShardTerms(collection, "shard1", cluster.getZkClient())) {
       shardTerms.registerTerm("leader");
-      TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
+      TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, new TimeSource.CurrentTimeSource());
       while (!timeOut.hasTimedOut()) {
         maxTerm++;
         assertEquals(shardTerms.getTerms().get("leader"), Collections.max(shardTerms.getTerms().values()));
@@ -194,7 +195,7 @@ public class ZkShardTermsTest extends SolrCloudTestCase {
   }
 
   private <T> void waitFor(T expected, Supplier<T> supplier) throws InterruptedException {
-    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
+    TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, new TimeSource.CurrentTimeSource());
     while (!timeOut.hasTimedOut()) {
       if (expected == supplier.get()) return;
       Thread.sleep(100);


[19/54] [abbrv] lucene-solr:jira/solr-11702: LUCENE-2899: Fix hyperlink text

Posted by da...@apache.org.
LUCENE-2899: Fix hyperlink text


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

Branch: refs/heads/jira/solr-11702
Commit: 565d13c96d89064214f74a81739eaf6b9fb7be18
Parents: f5c4276
Author: Steve Rowe <sa...@apache.org>
Authored: Fri Dec 15 11:42:08 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Dec 15 11:42:08 2017 -0500

----------------------------------------------------------------------
 .../OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/565d13c9/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
index 45bb839..a3df416 100644
--- a/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
+++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/update/processor/OpenNLPExtractNamedEntitiesUpdateProcessorFactory.java
@@ -74,7 +74,7 @@ import org.slf4j.LoggerFactory;
  *   &lt;/fieldType&gt;
  * </pre>
  * 
- * <p>See the <a href="http://opennlp.apache.org/models.html">OpenNLP website"></a>
+ * <p>See the <a href="http://opennlp.apache.org/models.html">OpenNLP website</a>
  * for information on downloading pre-trained models.</p>
  *
  * <p>